You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2017/05/25 20:49:40 UTC

[01/31] hive git commit: HIVE-16746: Reduce number of index lookups for same table in IndexWhereTaskDispatcher (Rajesh Balamohan, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 21c209e32 -> 52e0f8f34


HIVE-16746: Reduce number of index lookups for same table in IndexWhereTaskDispatcher (Rajesh Balamohan, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 3fe65a3c5b0f3bb60da0a866b9c00ec7a6448055
Parents: a74c1e7
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu May 25 04:03:54 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Thu May 25 04:03:54 2017 +0530

----------------------------------------------------------------------
 .../index/IndexWhereTaskDispatcher.java         | 34 ++++++++++++++++----
 1 file changed, 27 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3fe65a3c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java
index ae96def..7e3fb1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java
@@ -20,14 +20,16 @@ package org.apache.hadoop.hive.ql.optimizer.physical.index;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Stack;
 
+import com.google.common.collect.Maps;
 import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.cache.CacheUtils;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -49,6 +51,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.MapredWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hive.common.util.HiveStringUtils;
 
 /**
  *
@@ -60,10 +63,17 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 public class IndexWhereTaskDispatcher implements Dispatcher {
 
   private final PhysicalContext physicalContext;
+  // To store table to index mapping
+  private final Map<String, List<Index>> indexMap;
+  private final List<String> supportedIndexes;
 
   public IndexWhereTaskDispatcher(PhysicalContext context) {
     super();
     physicalContext = context;
+    indexMap = Maps.newHashMap();
+    supportedIndexes = new ArrayList<String>();
+    supportedIndexes.add(CompactIndexHandler.class.getName());
+    supportedIndexes.add(BitmapIndexHandler.class.getName());
   }
 
   @Override
@@ -104,6 +114,21 @@ public class IndexWhereTaskDispatcher implements Dispatcher {
     return null;
   }
 
+  private List<Index> getIndex(Table table) throws SemanticException {
+    String indexCacheKey = CacheUtils.buildKey(
+        HiveStringUtils.normalizeIdentifier(table.getDbName()),
+        HiveStringUtils.normalizeIdentifier(table.getTableName()));
+    List<Index>indexList = indexMap.get(indexCacheKey);
+    if (indexList == null) {
+      indexList =  IndexUtils.getIndexes(table, supportedIndexes);
+      if (indexList == null) {
+        indexList = Collections.emptyList();
+      }
+      indexMap.put(indexCacheKey, indexList);
+    }
+    return indexList;
+  }
+
   /**
    * Create a set of rules that only matches WHERE predicates on columns we have
    * an index on.
@@ -112,16 +137,11 @@ public class IndexWhereTaskDispatcher implements Dispatcher {
   private Map<Rule, NodeProcessor> createOperatorRules(ParseContext pctx) throws SemanticException {
     Map<Rule, NodeProcessor> operatorRules = new LinkedHashMap<Rule, NodeProcessor>();
 
-    List<String> supportedIndexes = new ArrayList<String>();
-    supportedIndexes.add(CompactIndexHandler.class.getName());
-    supportedIndexes.add(BitmapIndexHandler.class.getName());
-
     // query the metastore to know what columns we have indexed
     Map<TableScanOperator, List<Index>> indexes = new HashMap<TableScanOperator, List<Index>>();
     for (Operator<? extends OperatorDesc> op : pctx.getTopOps().values()) {
       if (op instanceof TableScanOperator) {
-        List<Index> tblIndexes = IndexUtils.getIndexes(((TableScanOperator) op).getConf()
-            .getTableMetadata(), supportedIndexes);
+        List<Index> tblIndexes = getIndex(((TableScanOperator) op).getConf().getTableMetadata());
         if (tblIndexes.size() > 0) {
           indexes.put((TableScanOperator) op, tblIndexes);
         }


[09/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 218fa8a..22998c4 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
@@ -105,12 +105,15 @@ TOK_IMPORT;
 TOK_REPLICATION;
 TOK_METADATA;
 TOK_NULL;
+TOK_NOT_NULL;
+TOK_UNIQUE;
 TOK_PRIMARY_KEY;
 TOK_FOREIGN_KEY;
 TOK_VALIDATE;
 TOK_NOVALIDATE;
 TOK_RELY;
 TOK_NORELY;
+TOK_CONSTRAINT_NAME;
 TOK_TINYINT;
 TOK_SMALLINT;
 TOK_INT;
@@ -549,6 +552,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
     xlateMap.put("KW_UPDATE", "UPDATE");
     xlateMap.put("KW_VALUES", "VALUES");
     xlateMap.put("KW_PURGE", "PURGE");
+    xlateMap.put("KW_UNIQUE", "UNIQUE");
     xlateMap.put("KW_PRIMARY", "PRIMARY");
     xlateMap.put("KW_FOREIGN", "FOREIGN");
     xlateMap.put("KW_KEY", "KEY");
@@ -1004,7 +1008,7 @@ createTableStatement
          tableFileFormat?
          tableLocation?
          tablePropertiesPrefixed?
-       | (LPAREN columnNameTypeOrPKOrFKList RPAREN)?
+       | (LPAREN columnNameTypeOrConstraintList RPAREN)?
          tableComment?
          tablePartition?
          tableBuckets?
@@ -1017,7 +1021,7 @@ createTableStatement
       )
     -> ^(TOK_CREATETABLE $name $temp? $ext? ifNotExists?
          ^(TOK_LIKETABLE $likeName?)
-         columnNameTypeOrPKOrFKList?
+         columnNameTypeOrConstraintList?
          tableComment?
          tablePartition?
          tableBuckets?
@@ -1232,9 +1236,9 @@ alterStatementSuffixAddCol
 alterStatementSuffixAddConstraint
 @init { pushMsg("add constraint statement", state); }
 @after { popMsg(state); }
-   :  KW_ADD (fk=foreignKeyWithName | primaryKeyWithName)
-   -> {fk != null}? ^(TOK_ALTERTABLE_ADDCONSTRAINT foreignKeyWithName)
-   ->               ^(TOK_ALTERTABLE_ADDCONSTRAINT primaryKeyWithName)
+   :  KW_ADD (fk=alterForeignKeyWithName | alterConstraintWithName)
+   -> {fk != null}? ^(TOK_ALTERTABLE_ADDCONSTRAINT alterForeignKeyWithName)
+   ->               ^(TOK_ALTERTABLE_ADDCONSTRAINT alterConstraintWithName)
    ;
 
 alterStatementSuffixDropConstraint
@@ -1247,8 +1251,8 @@ alterStatementSuffixDropConstraint
 alterStatementSuffixRenameCol
 @init { pushMsg("rename column name", state); }
 @after { popMsg(state); }
-    : KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition? restrictOrCascade?
-    ->^(TOK_ALTERTABLE_RENAMECOL $oldName $newName colType $comment? alterStatementChangeColPosition? restrictOrCascade?)
+    : KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType alterColumnConstraint[$newName.tree]? (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition? restrictOrCascade?
+    ->^(TOK_ALTERTABLE_RENAMECOL $oldName $newName colType $comment? alterColumnConstraint? alterStatementChangeColPosition? restrictOrCascade?)
     ;
 
 alterStatementSuffixUpdateStatsCol
@@ -2106,10 +2110,10 @@ columnNameTypeList
 @after { popMsg(state); }
     : columnNameType (COMMA columnNameType)* -> ^(TOK_TABCOLLIST columnNameType+)
     ;
-columnNameTypeOrPKOrFKList
+columnNameTypeOrConstraintList
 @init { pushMsg("column name type list with PK and FK", state); }
 @after { popMsg(state); }
-    : columnNameTypeOrPKOrFK (COMMA columnNameTypeOrPKOrFK)* -> ^(TOK_TABCOLLIST columnNameTypeOrPKOrFK+)
+    : columnNameTypeOrConstraint (COMMA columnNameTypeOrConstraint)* -> ^(TOK_TABCOLLIST columnNameTypeOrConstraint+)
     ;
 
 columnNameColonTypeList
@@ -2150,6 +2154,12 @@ columnParenthesesList
     : LPAREN! columnNameList RPAREN!
     ;
 
+enableValidateSpecification
+@init { pushMsg("enable specification", state); }
+@after { popMsg(state); }
+    : enableSpecification validateSpecification?
+    ;
+
 enableSpecification
 @init { pushMsg("enable specification", state); }
 @after { popMsg(state); }
@@ -2171,32 +2181,36 @@ relySpecification
     |  (KW_NORELY)? -> ^(TOK_NORELY)
     ;
 
-primaryKeyWithoutName
-@init { pushMsg("primary key without key name", state); }
+createConstraint
+@init { pushMsg("pk or uk or nn constraint", state); }
 @after { popMsg(state); }
-    : KW_PRIMARY KW_KEY columnParenthesesList enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    -> ^(TOK_PRIMARY_KEY columnParenthesesList $relySpec $enableSpec $validateSpec)
+    : (KW_CONSTRAINT constraintName=identifier)? tableConstraintType pkCols=columnParenthesesList constraintOptsCreate?
+    -> {$constraintName.tree != null}?
+            ^(tableConstraintType $pkCols ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsCreate?)
+    -> ^(tableConstraintType $pkCols constraintOptsCreate?)
     ;
 
-primaryKeyWithName
-@init { pushMsg("primary key with key name", state); }
+alterConstraintWithName
+@init { pushMsg("pk or uk or nn constraint with name", state); }
 @after { popMsg(state); }
-    : KW_CONSTRAINT idfr=identifier KW_PRIMARY KW_KEY pkCols=columnParenthesesList enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    -> ^(TOK_PRIMARY_KEY $pkCols $idfr $relySpec $enableSpec $validateSpec)
+    : KW_CONSTRAINT constraintName=identifier tableConstraintType pkCols=columnParenthesesList constraintOptsAlter?
+    -> ^(tableConstraintType $pkCols ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsAlter?)
     ;
 
-foreignKeyWithName
-@init { pushMsg("foreign key with key name", state); }
+createForeignKey
+@init { pushMsg("foreign key", state); }
 @after { popMsg(state); }
-    : KW_CONSTRAINT idfr=identifier KW_FOREIGN KW_KEY fkCols=columnParenthesesList  KW_REFERENCES tabName=tableName parCols=columnParenthesesList enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    -> ^(TOK_FOREIGN_KEY $idfr $fkCols $tabName $parCols $relySpec $enableSpec $validateSpec)
+    : (KW_CONSTRAINT constraintName=identifier)? KW_FOREIGN KW_KEY fkCols=columnParenthesesList  KW_REFERENCES tabName=tableName parCols=columnParenthesesList constraintOptsCreate?
+    -> {$constraintName.tree != null}?
+            ^(TOK_FOREIGN_KEY ^(TOK_CONSTRAINT_NAME $constraintName) $fkCols $tabName $parCols constraintOptsCreate?)
+    -> ^(TOK_FOREIGN_KEY $fkCols $tabName $parCols constraintOptsCreate?)
     ;
 
-foreignKeyWithoutName
-@init { pushMsg("foreign key without key name", state); }
+alterForeignKeyWithName
+@init { pushMsg("foreign key with key name", state); }
 @after { popMsg(state); }
-    : KW_FOREIGN KW_KEY fkCols=columnParenthesesList  KW_REFERENCES tabName=tableName parCols=columnParenthesesList enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    -> ^(TOK_FOREIGN_KEY $fkCols  $tabName $parCols $relySpec $enableSpec $validateSpec)
+    : KW_CONSTRAINT constraintName=identifier KW_FOREIGN KW_KEY fkCols=columnParenthesesList  KW_REFERENCES tabName=tableName parCols=columnParenthesesList constraintOptsAlter?
+    -> ^(TOK_FOREIGN_KEY ^(TOK_CONSTRAINT_NAME $constraintName) $fkCols $tabName $parCols constraintOptsAlter?)
     ;
 
 skewedValueElement
@@ -2310,14 +2324,94 @@ columnNameType
     ->                     ^(TOK_TABCOL $colName colType $comment)
     ;
 
-columnNameTypeOrPKOrFK
-@init { pushMsg("column name or primary key or foreign key", state); }
+columnNameTypeOrConstraint
+@init { pushMsg("column name or constraint", state); }
+@after { popMsg(state); }
+    : ( tableConstraint )
+    | ( columnNameTypeConstraint )
+    ;
+
+tableConstraint
+@init { pushMsg("table constraint", state); }
+@after { popMsg(state); }
+    : ( createForeignKey )
+    | ( createConstraint )
+    ;
+
+columnNameTypeConstraint
+@init { pushMsg("column specification", state); }
+@after { popMsg(state); }
+    : colName=identifier colType columnConstraint[$colName.tree]? (KW_COMMENT comment=StringLiteral)?
+    -> {containExcludedCharForCreateTableColumnName($colName.text)}? {throwColumnNameException()}
+    -> ^(TOK_TABCOL $colName colType $comment? columnConstraint?)
+    ;
+
+columnConstraint[CommonTree fkColName]
+@init { pushMsg("column constraint", state); }
+@after { popMsg(state); }
+    : ( foreignKeyConstraint[$fkColName] )
+    | ( colConstraint )
+    ;
+
+foreignKeyConstraint[CommonTree fkColName]
+@init { pushMsg("column constraint", state); }
 @after { popMsg(state); }
-    : ( foreignKeyWithName )
-    | ( primaryKeyWithName )
-    | ( primaryKeyWithoutName )
-    | ( foreignKeyWithoutName )
-    | ( columnNameType )
+    : (KW_CONSTRAINT constraintName=identifier)? KW_REFERENCES tabName=tableName LPAREN colName=columnName RPAREN constraintOptsCreate?
+    -> {$constraintName.tree != null}?
+            ^(TOK_FOREIGN_KEY ^(TOK_CONSTRAINT_NAME $constraintName) ^(TOK_TABCOLNAME {$fkColName}) $tabName ^(TOK_TABCOLNAME $colName) constraintOptsCreate?)
+    -> ^(TOK_FOREIGN_KEY ^(TOK_TABCOLNAME {$fkColName}) $tabName ^(TOK_TABCOLNAME $colName) constraintOptsCreate?)
+    ;
+
+colConstraint
+@init { pushMsg("column constraint", state); }
+@after { popMsg(state); }
+    : (KW_CONSTRAINT constraintName=identifier)? columnConstraintType constraintOptsCreate?
+    -> {$constraintName.tree != null}?
+            ^(columnConstraintType ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsCreate?)
+    -> ^(columnConstraintType constraintOptsCreate?)
+    ;
+
+alterColumnConstraint[CommonTree fkColName]
+@init { pushMsg("alter column constraint", state); }
+@after { popMsg(state); }
+    : ( alterForeignKeyConstraint[$fkColName] )
+    | ( alterColConstraint )
+    ;
+
+alterForeignKeyConstraint[CommonTree fkColName]
+@init { pushMsg("alter column constraint", state); }
+@after { popMsg(state); }
+    : (KW_CONSTRAINT constraintName=identifier)? KW_REFERENCES tabName=tableName LPAREN colName=columnName RPAREN constraintOptsAlter?
+    -> {$constraintName.tree != null}?
+            ^(TOK_FOREIGN_KEY ^(TOK_CONSTRAINT_NAME $constraintName) ^(TOK_TABCOLNAME {$fkColName}) $tabName ^(TOK_TABCOLNAME $colName) constraintOptsAlter?)
+    -> ^(TOK_FOREIGN_KEY ^(TOK_TABCOLNAME {$fkColName}) $tabName ^(TOK_TABCOLNAME $colName) constraintOptsAlter?)
+    ;
+
+alterColConstraint
+@init { pushMsg("alter column constraint", state); }
+@after { popMsg(state); }
+    : (KW_CONSTRAINT constraintName=identifier)? columnConstraintType constraintOptsAlter?
+    -> {$constraintName.tree != null}?
+            ^(columnConstraintType ^(TOK_CONSTRAINT_NAME $constraintName) constraintOptsAlter?)
+    -> ^(columnConstraintType constraintOptsAlter?)
+    ;
+
+columnConstraintType
+    : KW_NOT KW_NULL       ->    TOK_NOT_NULL
+    | tableConstraintType
+    ;
+
+tableConstraintType
+    : KW_PRIMARY KW_KEY    ->    TOK_PRIMARY_KEY
+    | KW_UNIQUE            ->    TOK_UNIQUE
+    ;
+
+constraintOptsCreate
+    : enableValidateSpecification relySpecification
+    ;
+
+constraintOptsAlter
+    : enableValidateSpecification relySpecification
     ;
 
 columnNameColonType

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 43a61ce..f2b0bd0 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
@@ -73,7 +73,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -11961,6 +11963,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     List<String> bucketCols = new ArrayList<String>();
     List<SQLPrimaryKey> primaryKeys = new ArrayList<SQLPrimaryKey>();
     List<SQLForeignKey> foreignKeys = new ArrayList<SQLForeignKey>();
+    List<SQLUniqueConstraint> uniqueConstraints = new ArrayList<>();
+    List<SQLNotNullConstraint> notNullConstraints = new ArrayList<>();
     List<Order> sortCols = new ArrayList<Order>();
     int numBuckets = -1;
     String comment = null;
@@ -12053,7 +12057,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         selectStmt = child;
         break;
       case HiveParser.TOK_TABCOLLIST:
-        cols = getColumns(child, true, primaryKeys, foreignKeys);
+        cols = getColumns(child, true, primaryKeys, foreignKeys,
+            uniqueConstraints, notNullConstraints);
         break;
       case HiveParser.TOK_TABLECOMMENT:
         comment = unescapeSQLString(child.getChild(0).getText());
@@ -12167,7 +12172,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           comment,
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists, skewedColNames,
-          skewedValues, primaryKeys, foreignKeys);
+          skewedValues, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
       crtTblDesc.setStoredAsSubDirectories(storedAsDirs);
       crtTblDesc.setNullFormat(rowFormatParams.nullFormat);
 
@@ -12264,7 +12269,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           rowFormatParams.lineDelim, comment, storageFormat.getInputFormat(),
           storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
-	  skewedColNames, skewedValues, true, primaryKeys, foreignKeys);
+	        skewedColNames, skewedValues, true, primaryKeys, foreignKeys,
+	        uniqueConstraints, notNullConstraints);
       tableDesc.setMaterialization(isMaterialization);
       tableDesc.setStoredAsSubDirectories(storedAsDirs);
       tableDesc.setNullFormat(rowFormatParams.nullFormat);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 d5a6679..6cfde18 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
@@ -22,7 +22,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
@@ -127,6 +129,8 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   String dropConstraintName;
   List<SQLPrimaryKey> primaryKeyCols;
   List<SQLForeignKey> foreignKeyCols;
+  List<SQLUniqueConstraint> uniqueConstraintCols;
+  List<SQLNotNullConstraint> notNullConstraintCols;
 
   public AlterTableDesc() {
   }
@@ -157,6 +161,28 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     this.isCascade = isCascade;
   }
 
+  public AlterTableDesc(String tblName, HashMap<String, String> partSpec,
+      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) {
+    super();
+    oldName = tblName;
+    this.partSpec = partSpec;
+    this.oldColName = oldColName;
+    this.newColName = newColName;
+    newColType = newType;
+    newColComment = newComment;
+    this.first = first;
+    this.afterCol = afterCol;
+    op = AlterTableTypes.RENAMECOLUMN;
+    this.isCascade = isCascade;
+    this.primaryKeyCols = primaryKeyCols;
+    this.foreignKeyCols = foreignKeyCols;
+    this.uniqueConstraintCols = uniqueConstraintCols;
+    this.notNullConstraintCols = notNullConstraintCols;
+  }
+
   /**
    * @param oldName
    *          old name of the table
@@ -280,10 +306,12 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     op = AlterTableTypes.DROPCONSTRAINT;
   }
 
-  public AlterTableDesc(String tableName, List<SQLPrimaryKey> primaryKeyCols, List<SQLForeignKey> foreignKeyCols) {
+  public AlterTableDesc(String tableName, List<SQLPrimaryKey> primaryKeyCols,
+          List<SQLForeignKey> foreignKeyCols, List<SQLUniqueConstraint> uniqueConstraintCols) {
     this.oldName = tableName;
     this.primaryKeyCols = primaryKeyCols;
     this.foreignKeyCols = foreignKeyCols;
+    this.uniqueConstraintCols = uniqueConstraintCols;
     op = AlterTableTypes.ADDCONSTRAINT;
   }
 
@@ -462,6 +490,20 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
   }
 
   /**
+   * @return the unique constraint cols
+   */
+  public List<SQLUniqueConstraint> getUniqueConstraintCols() {
+    return uniqueConstraintCols;
+  }
+
+  /**
+   * @return the not null constraint cols
+   */
+  public List<SQLNotNullConstraint> getNotNullConstraintCols() {
+    return notNullConstraintCols;
+  }
+
+  /**
    * @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/696be9f5/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 d971c73..7b46fcd 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
@@ -32,7 +32,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -96,6 +98,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
   private boolean isCTAS = false;
   List<SQLPrimaryKey> primaryKeys;
   List<SQLForeignKey> foreignKeys;
+  List<SQLUniqueConstraint> uniqueConstraints;
+  List<SQLNotNullConstraint> notNullConstraints;
 
   public CreateTableDesc() {
   }
@@ -110,13 +114,15 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
       boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
-      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
 
     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);
+        tblProps, ifNotExists, skewedColNames, skewedColValues,
+        primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
 
     this.databaseName = databaseName;
   }
@@ -131,12 +137,14 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
                          Map<String, String> serdeProps,
                          Map<String, String> tblProps,
                          boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
-                         boolean isCTAS, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
+                         boolean isCTAS, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+                         List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
     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);
+            tblProps, ifNotExists, skewedColNames, skewedColValues,
+            primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
     this.isCTAS = isCTAS;
 
   }
@@ -152,7 +160,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
       boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
-      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints) {
     this.tableName = tableName;
     this.isExternal = isExternal;
     this.isTemporary = isTemporary;
@@ -177,16 +186,10 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     this.ifNotExists = ifNotExists;
     this.skewedColNames = copyList(skewedColNames);
     this.skewedColValues = copyList(skewedColValues);
-    if (primaryKeys == null) {
-      this.primaryKeys = new ArrayList<SQLPrimaryKey>();
-    } else {
-      this.primaryKeys = new ArrayList<SQLPrimaryKey>(primaryKeys);
-    }
-    if (foreignKeys == null) {
-      this.foreignKeys = new ArrayList<SQLForeignKey>();
-    } else {
-      this.foreignKeys = new ArrayList<SQLForeignKey>(foreignKeys);
-    }
+    this.primaryKeys = copyList(primaryKeys);
+    this.foreignKeys = copyList(foreignKeys);
+    this.uniqueConstraints = copyList(uniqueConstraints);
+    this.notNullConstraints = copyList(notNullConstraints);
   }
 
   private static <T> List<T> copyList(List<T> copy) {
@@ -257,6 +260,14 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     this.foreignKeys = foreignKeys;
   }
 
+  public List<SQLUniqueConstraint> getUniqueConstraints() {
+    return uniqueConstraints;
+  }
+
+  public List<SQLNotNullConstraint> getNotNullConstraints() {
+    return notNullConstraints;
+  }
+
   @Explain(displayName = "bucket columns")
   public List<String> getBucketCols() {
     return bucketCols;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 f43627d..bb02c26 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
@@ -74,7 +74,11 @@ public class ImportTableDesc {
                 (null == table.getSd().getSkewedInfo()) ? null : table.getSd().getSkewedInfo()
                         .getSkewedColNames(),
                 (null == table.getSd().getSkewedInfo()) ? null : table.getSd().getSkewedInfo()
-                        .getSkewedColValues(), null, null);
+                        .getSkewedColValues(),
+                null,
+                null,
+                null,
+                null);
         this.createTblDesc.setStoredAsSubDirectories(table.getSd().isStoredAsSubDirectories());
         break;
       case VIEW:

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/org/apache/hadoop/hive/ql/parse/TestHiveDecimalParse.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestHiveDecimalParse.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestHiveDecimalParse.java
index 7073374..9f155e1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestHiveDecimalParse.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestHiveDecimalParse.java
@@ -104,7 +104,7 @@ public class TestHiveDecimalParse {
     int rc = driver.compile(query);
     Assert.assertTrue("Got " + rc + ", expected not zero", rc != 0);
     Assert.assertTrue(driver.getErrorMsg(),
-      driver.getErrorMsg().contains("missing ) at ',' near ',' in column name or primary key or foreign key"));
+      driver.getErrorMsg().contains("missing ) at ',' near ',' in column name or constraint"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
index 8be8583..1a89eb1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSQL11ReservedKeyWordsNegative.java
@@ -1072,6 +1072,19 @@ public class TestSQL11ReservedKeyWordsNegative {
 		}
 	}
 
+  @Test
+  public void testSQL11ReservedKeyWords_UNIQUE() {
+    try {
+      parse("CREATE TABLE UNIQUE (col STRING)");
+      Assert.assertFalse("Expected ParseException", true);
+    } catch (ParseException ex) {
+      Assert.assertEquals(
+          "Failure didn't match.",
+          "line 1:13 cannot recognize input near 'UNIQUE' '(' 'col' in table name",
+          ex.getMessage());
+    }
+  }
+
 	@Test
 	public void testSQL11ReservedKeyWords_UPDATE() {
 		try {

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_duplicate_pk.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_duplicate_pk.q b/ql/src/test/queries/clientnegative/alter_table_constraint_duplicate_pk.q
index f77eb29..82de06d 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_duplicate_pk.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_duplicate_pk.q
@@ -1,2 +1,2 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
 alter table table1 add constraint pk4 primary key (b) disable novalidate rely;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col1.q b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col1.q
index e12808d..88be76a 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col1.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col1.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
+CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely);
 alter table table2 add constraint fk1 foreign key (c) references table1(a) disable novalidate;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col2.q b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col2.q
index 97703de..cfc0757 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col2.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_col2.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
+CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely);
 alter table table2 add constraint fk1 foreign key (b) references table1(c) disable novalidate;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl1.q b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl1.q
index dcd7839..0cc7c87 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl1.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl1.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
+CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely);
 alter table table3 add constraint fk1 foreign key (c) references table1(a) disable novalidate;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl2.q b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl2.q
index c18247b..f019eb5 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl2.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_fk_tbl2.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
+CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely);
 alter table table2 add constraint fk1 foreign key (b) references table3(a) disable novalidate;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_pk_tbl.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_pk_tbl.q b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_pk_tbl.q
index b6850fa..ced99f5 100644
--- a/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_pk_tbl.q
+++ b/ql/src/test/queries/clientnegative/alter_table_constraint_invalid_pk_tbl.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable);
 CREATE TABLE table2 (a STRING, b STRING);
 alter table table3 add constraint pk3 primary key (a) disable novalidate rely;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q b/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q
index b6b30c5..a0bc7f6 100644
--- a/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q
+++ b/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q
@@ -1,2 +1,2 @@
-create table t1(x int, constraint pk1 primary key (x) disable novalidate);
-create table t2(x int, constraint pk1 primary key (x) disable novalidate);
+create table t1(x int, constraint pk1 primary key (x) disable);
+create table t2(x int, constraint pk1 primary key (x) disable);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/create_with_constraints_enable.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_with_constraints_enable.q b/ql/src/test/queries/clientnegative/create_with_constraints_enable.q
index 4ce3cbc..59ebb1e 100644
--- a/ql/src/test/queries/clientnegative/create_with_constraints_enable.q
+++ b/ql/src/test/queries/clientnegative/create_with_constraints_enable.q
@@ -1 +1 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) enable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) enable);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/create_with_fk_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_with_fk_constraint.q b/ql/src/test/queries/clientnegative/create_with_fk_constraint.q
new file mode 100644
index 0000000..ea77d84
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_with_fk_constraint.q
@@ -0,0 +1,2 @@
+CREATE TABLE table2 (a STRING PRIMARY KEY DISABLE, b STRING);
+CREATE TABLE table1 (a STRING, b STRING, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/create_with_multi_pk_constraint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_with_multi_pk_constraint.q b/ql/src/test/queries/clientnegative/create_with_multi_pk_constraint.q
new file mode 100644
index 0000000..46306fc
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_with_multi_pk_constraint.q
@@ -0,0 +1 @@
+CREATE TABLE table1 (a STRING PRIMARY KEY DISABLE, b STRING PRIMARY KEY DISABLE);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
index 2055f9e..128eb2c 100644
--- a/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint1.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable);
 ALTER TABLE table1 DROP CONSTRAINT pk1;
 ALTER TABLE table1 DROP CONSTRAINT pk1;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
index d253617..871bd3e 100644
--- a/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint2.q
@@ -1,2 +1,2 @@
-CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable);
 ALTER TABLE table1 DROP CONSTRAINT pk1;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
index 04eb1fb..ca1f599 100644
--- a/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint3.q
@@ -1,2 +1,2 @@
-CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
+CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable);
 ALTER TABLE table2 DROP CONSTRAINT pk2;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q b/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
index 3cf2d2a..25026b0 100644
--- a/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
+++ b/ql/src/test/queries/clientnegative/drop_invalid_constraint4.q
@@ -1,3 +1,3 @@
-CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable);
+CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable);
 ALTER TABLE table1 DROP CONSTRAINT pk2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 7dc15c1..9700b5c 100644
--- a/ql/src/test/queries/clientpositive/create_with_constraints.q
+++ b/ql/src/test/queries/clientpositive/create_with_constraints.q
@@ -1,17 +1,24 @@
-CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate);
-CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate);
-CREATE TABLE table3 (x string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE); 
-CREATE TABLE table4 (x string, y string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE, 
-CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a)  DISABLE NOVALIDATE);
-CREATE TABLE table5 (x string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE);
-CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE,
-CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE);
-CREATE TABLE table7 (a STRING, b STRING, primary key (a) disable novalidate rely);
-CREATE TABLE table8 (a STRING, b STRING, constraint pk8 primary key (a) disable novalidate norely);
-CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely);
-CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate);
-CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, constraint fk11_1 foreign key (a, b) references table9(a, b) disable novalidate,
-constraint fk11_2 foreign key (c) references table4(x) disable novalidate);
+CREATE TABLE table1 (a STRING, b STRING, PRIMARY KEY (a) DISABLE);
+CREATE TABLE table2 (a STRING, b STRING, CONSTRAINT pk1 PRIMARY KEY (a) DISABLE);
+CREATE TABLE table3 (x string NOT NULL DISABLE, PRIMARY KEY (x) DISABLE, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE); 
+CREATE TABLE table4 (x string CONSTRAINT nn4_1 NOT NULL DISABLE, y string CONSTRAINT nn4_2 NOT NULL DISABLE, UNIQUE (x) DISABLE, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a) DISABLE, 
+CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a) DISABLE);
+CREATE TABLE table5 (x string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE);
+CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE,
+CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE);
+CREATE TABLE table7 (a STRING, b STRING, PRIMARY KEY (a) DISABLE RELY);
+CREATE TABLE table8 (a STRING, b STRING, CONSTRAINT pk8 PRIMARY KEY (a) DISABLE NORELY);
+CREATE TABLE table9 (a STRING, b STRING, PRIMARY KEY (a, b) DISABLE RELY);
+CREATE TABLE table10 (a STRING, b STRING, CONSTRAINT pk10 PRIMARY KEY (a) DISABLE NORELY, FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE);
+CREATE TABLE table11 (a STRING, b STRING, c STRING, CONSTRAINT pk11 PRIMARY KEY (a) DISABLE RELY, CONSTRAINT fk11_1 FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE,
+CONSTRAINT fk11_2 FOREIGN KEY (c) REFERENCES table4(x) DISABLE);
+CREATE TABLE table12 (a STRING CONSTRAINT nn12_1 NOT NULL DISABLE NORELY, b STRING);
+CREATE TABLE table13 (a STRING NOT NULL DISABLE RELY, b STRING);
+CREATE TABLE table14 (a STRING CONSTRAINT nn14_1 NOT NULL DISABLE RELY, b STRING);
+CREATE TABLE table15 (a STRING REFERENCES table4(x) DISABLE, b STRING);
+CREATE TABLE table16 (a STRING CONSTRAINT nn16_1 REFERENCES table4(x) DISABLE RELY, b STRING);
+CREATE TABLE table17 (a STRING CONSTRAINT uk17_1 UNIQUE DISABLE RELY, b STRING);
+CREATE TABLE table18 (a STRING, b STRING, CONSTRAINT uk18_1 UNIQUE (b) DISABLE RELY);
 
 DESCRIBE EXTENDED table1;
 DESCRIBE EXTENDED table2;
@@ -24,6 +31,13 @@ DESCRIBE EXTENDED table8;
 DESCRIBE EXTENDED table9;
 DESCRIBE EXTENDED table10;
 DESCRIBE EXTENDED table11;
+DESCRIBE EXTENDED table12;
+DESCRIBE EXTENDED table13;
+DESCRIBE EXTENDED table14;
+DESCRIBE EXTENDED table15;
+DESCRIBE EXTENDED table16;
+DESCRIBE EXTENDED table17;
+DESCRIBE EXTENDED table18;
 
 DESCRIBE FORMATTED table1;
 DESCRIBE FORMATTED table2;
@@ -36,30 +50,60 @@ DESCRIBE FORMATTED table8;
 DESCRIBE FORMATTED table9;
 DESCRIBE FORMATTED table10;
 DESCRIBE FORMATTED table11;
+DESCRIBE FORMATTED table12;
+DESCRIBE FORMATTED table13;
+DESCRIBE FORMATTED table14;
+DESCRIBE FORMATTED table15;
+DESCRIBE FORMATTED table16;
+DESCRIBE FORMATTED table17;
+DESCRIBE FORMATTED table18;
 
 ALTER TABLE table2 DROP CONSTRAINT pk1;
 ALTER TABLE table3 DROP CONSTRAINT fk1;
+ALTER TABLE table4 DROP CONSTRAINT nn4_1;
 ALTER TABLE table6 DROP CONSTRAINT fk4;
+ALTER TABLE table16 DROP CONSTRAINT nn16_1;
+ALTER TABLE table18 DROP CONSTRAINT uk18_1;
 
 DESCRIBE EXTENDED table2;
 DESCRIBE EXTENDED table3;
+DESCRIBE EXTENDED table4;
 DESCRIBE EXTENDED table6;
+DESCRIBE EXTENDED table16;
+DESCRIBE EXTENDED table18;
 
 DESCRIBE FORMATTED table2;
 DESCRIBE FORMATTED table3;
+DESCRIBE FORMATTED table4;
 DESCRIBE FORMATTED table6;
+DESCRIBE FORMATTED table16;
+DESCRIBE FORMATTED table18;
 
-ALTER TABLE table2 ADD CONSTRAINT pkt2 primary key (a) disable novalidate;
-ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE RELY;
-ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE;
+ALTER TABLE table2 ADD CONSTRAINT pkt2 PRIMARY KEY (a) DISABLE NOVALIDATE;
+ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE NOVALIDATE RELY;
+ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE NOVALIDATE;
+ALTER TABLE table16 CHANGE a a STRING REFERENCES table4(x) DISABLE NOVALIDATE;
+ALTER TABLE table18 ADD CONSTRAINT uk18_2 UNIQUE (a, b) DISABLE NOVALIDATE;
 
 DESCRIBE FORMATTED table2;
 DESCRIBE FORMATTED table3;
 DESCRIBE FORMATTED table6;
+DESCRIBE FORMATTED table16;
+DESCRIBE FORMATTED table18;
+
+ALTER TABLE table12 CHANGE COLUMN b b STRING CONSTRAINT nn12_2 NOT NULL DISABLE NOVALIDATE;
+ALTER TABLE table13 CHANGE b b STRING NOT NULL DISABLE NOVALIDATE;
+
+DESCRIBE FORMATTED table12;
+DESCRIBE FORMATTED table13;
+
+ALTER TABLE table12 DROP CONSTRAINT nn12_2;
+
+DESCRIBE FORMATTED table12;
 
 CREATE DATABASE DbConstraint;
 USE DbConstraint;
-CREATE TABLE Table2 (a STRING, b STRING, constraint Pk1 primary key (a) disable novalidate);
+CREATE TABLE Table2 (a STRING, b STRING NOT NULL DISABLE, CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE);
 USE default;
 
 DESCRIBE EXTENDED DbConstraint.Table2;
@@ -70,7 +114,7 @@ ALTER TABLE DbConstraint.Table2 DROP CONSTRAINT Pk1;
 DESCRIBE EXTENDED DbConstraint.Table2;
 DESCRIBE FORMATTED DbConstraint.Table2;
 
-ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 primary key (a) disable novalidate;
+ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE NOVALIDATE;
 DESCRIBE FORMATTED DbConstraint.Table2;
-ALTER TABLE DbConstraint.Table2  ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a)  DISABLE NOVALIDATE;
+ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a) DISABLE NOVALIDATE;
 DESCRIBE FORMATTED DbConstraint.Table2;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
index d1bb637..30ad841 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col1.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col1.q.out
index 2cd85c4..1617609 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col1.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col1.q.out
@@ -1,16 +1,16 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col2.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col2.q.out
index 86c38c1..47166ac 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col2.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_col2.q.out
@@ -1,16 +1,16 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl1.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl1.q.out
index 16edd44..49bc928 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl1.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl1.q.out
@@ -1,16 +1,16 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl2.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl2.q.out
index 31dfcd1..f5ac4ac 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl2.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_fk_tbl2.q.out
@@ -1,16 +1,16 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, primary key (a) disable rely)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/alter_table_constraint_invalid_pk_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_pk_tbl.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_pk_tbl.q.out
index 0207d8c..792134c 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_invalid_pk_tbl.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_invalid_pk_tbl.q.out
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out b/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out
index 989ff38..e4747d4 100644
--- a/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out
+++ b/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out
@@ -1,12 +1,12 @@
-PREHOOK: query: create table t1(x int, constraint pk1 primary key (x) disable novalidate)
+PREHOOK: query: create table t1(x int, constraint pk1 primary key (x) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t1
-POSTHOOK: query: create table t1(x int, constraint pk1 primary key (x) disable novalidate)
+POSTHOOK: query: create table t1(x int, constraint pk1 primary key (x) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t1
-PREHOOK: query: create table t2(x int, constraint pk1 primary key (x) disable novalidate)
+PREHOOK: query: create table t2(x int, constraint pk1 primary key (x) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out b/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out
index f5dcf85..65d3bfe 100644
--- a/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out
+++ b/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out
@@ -1 +1 @@
-FAILED: SemanticException [Error 10326]: Invalid Primary Key syntax  ENABLE feature not supported yet
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax ENABLE feature not supported yet. Please use DISABLE instead.

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out b/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out
index 307e8ec..b2442e8 100644
--- a/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out
+++ b/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out
@@ -1 +1 @@
-FAILED: SemanticException [Error 10326]: Invalid Primary Key syntax  VALIDATE feature not supported yet
+FAILED: SemanticException [Error 10326]: Invalid Constraint syntax VALIDATE feature not supported yet. Please use NOVALIDATE instead.

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/create_with_fk_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_with_fk_constraint.q.out b/ql/src/test/results/clientnegative/create_with_fk_constraint.q.out
new file mode 100644
index 0000000..6598d6c
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_with_fk_constraint.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: CREATE TABLE table2 (a STRING PRIMARY KEY DISABLE, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table2
+POSTHOOK: query: CREATE TABLE table2 (a STRING PRIMARY KEY DISABLE, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table2
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidObjectException(message:Child column not found: x)

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/create_with_multi_pk_constraint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_with_multi_pk_constraint.q.out b/ql/src/test/results/clientnegative/create_with_multi_pk_constraint.q.out
new file mode 100644
index 0000000..200fdf5
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_with_multi_pk_constraint.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10332]: Invalid constraint definition Cannot exist more than one primary key definition for the same table

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
index 4568ccb..2cb3996 100644
--- a/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint1.q.out
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
index 0051131..04352b4 100644
--- a/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint2.q.out
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
index 9c60e94..03e4bd6 100644
--- a/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint3.q.out
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out b/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
index 1d93c42..473dec7 100644
--- a/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
+++ b/ql/src/test/results/clientnegative/drop_invalid_constraint4.q.out
@@ -1,16 +1,16 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, constraint pk1 primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk2 primary key (a) disable)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2


[12/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index f26cb5b..1e2b890 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -821,28 +821,40 @@ class SQLForeignKey:
   def __ne__(self, other):
     return not (self == other)
 
-class Type:
+class SQLUniqueConstraint:
   """
   Attributes:
-   - name
-   - type1
-   - type2
-   - fields
+   - table_db
+   - table_name
+   - column_name
+   - key_seq
+   - uk_name
+   - enable_cstr
+   - validate_cstr
+   - rely_cstr
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'name', None, None, ), # 1
-    (2, TType.STRING, 'type1', None, None, ), # 2
-    (3, TType.STRING, 'type2', None, None, ), # 3
-    (4, TType.LIST, 'fields', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 4
+    (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.I32, 'key_seq', None, None, ), # 4
+    (5, TType.STRING, 'uk_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, name=None, type1=None, type2=None, fields=None,):
-    self.name = name
-    self.type1 = type1
-    self.type2 = type2
-    self.fields = fields
+  def __init__(self, table_db=None, table_name=None, column_name=None, key_seq=None, uk_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.key_seq = key_seq
+    self.uk_name = uk_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:
@@ -855,28 +867,42 @@ class Type:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.name = iprot.readString()
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.type1 = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.type2 = iprot.readString()
+          self.column_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
-        if ftype == TType.LIST:
-          self.fields = []
-          (_etype3, _size0) = iprot.readListBegin()
-          for _i4 in xrange(_size0):
-            _elem5 = FieldSchema()
-            _elem5.read(iprot)
-            self.fields.append(_elem5)
-          iprot.readListEnd()
+        if ftype == TType.I32:
+          self.key_seq = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.uk_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:
@@ -888,25 +914,38 @@ class Type:
     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('Type')
-    if self.name is not None:
-      oprot.writeFieldBegin('name', TType.STRING, 1)
-      oprot.writeString(self.name)
+    oprot.writeStructBegin('SQLUniqueConstraint')
+    if self.table_db is not None:
+      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
-    if self.type1 is not None:
-      oprot.writeFieldBegin('type1', TType.STRING, 2)
-      oprot.writeString(self.type1)
+    if self.table_name is not None:
+      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
-    if self.type2 is not None:
-      oprot.writeFieldBegin('type2', TType.STRING, 3)
-      oprot.writeString(self.type2)
+    if self.column_name is not None:
+      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
-    if self.fields is not None:
-      oprot.writeFieldBegin('fields', TType.LIST, 4)
-      oprot.writeListBegin(TType.STRUCT, len(self.fields))
-      for iter6 in self.fields:
-        iter6.write(oprot)
-      oprot.writeListEnd()
+    if self.key_seq is not None:
+      oprot.writeFieldBegin('key_seq', TType.I32, 4)
+      oprot.writeI32(self.key_seq)
+      oprot.writeFieldEnd()
+    if self.uk_name is not None:
+      oprot.writeFieldBegin('uk_name', TType.STRING, 5)
+      oprot.writeString(self.uk_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()
@@ -917,10 +956,14 @@ class Type:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.name)
-    value = (value * 31) ^ hash(self.type1)
-    value = (value * 31) ^ hash(self.type2)
-    value = (value * 31) ^ hash(self.fields)
+    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.key_seq)
+    value = (value * 31) ^ hash(self.uk_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):
@@ -934,31 +977,37 @@ class Type:
   def __ne__(self, other):
     return not (self == other)
 
-class HiveObjectRef:
+class SQLNotNullConstraint:
   """
   Attributes:
-   - objectType
-   - dbName
-   - objectName
-   - partValues
-   - columnName
+   - table_db
+   - table_name
+   - column_name
+   - nn_name
+   - enable_cstr
+   - validate_cstr
+   - rely_cstr
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.I32, 'objectType', None, None, ), # 1
-    (2, TType.STRING, 'dbName', None, None, ), # 2
-    (3, TType.STRING, 'objectName', None, None, ), # 3
-    (4, TType.LIST, 'partValues', (TType.STRING,None), None, ), # 4
-    (5, TType.STRING, 'columnName', None, None, ), # 5
+    (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, 'nn_name', None, None, ), # 4
+    (5, TType.BOOL, 'enable_cstr', None, None, ), # 5
+    (6, TType.BOOL, 'validate_cstr', None, None, ), # 6
+    (7, TType.BOOL, 'rely_cstr', None, None, ), # 7
   )
 
-  def __init__(self, objectType=None, dbName=None, objectName=None, partValues=None, columnName=None,):
-    self.objectType = objectType
-    self.dbName = dbName
-    self.objectName = objectName
-    self.partValues = partValues
-    self.columnName = columnName
+  def __init__(self, table_db=None, table_name=None, column_name=None, nn_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.nn_name = nn_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:
@@ -970,33 +1019,38 @@ class HiveObjectRef:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.I32:
-          self.objectType = iprot.readI32()
+        if ftype == TType.STRING:
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.dbName = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.objectName = iprot.readString()
+          self.column_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
-        if ftype == TType.LIST:
-          self.partValues = []
-          (_etype10, _size7) = iprot.readListBegin()
-          for _i11 in xrange(_size7):
-            _elem12 = iprot.readString()
-            self.partValues.append(_elem12)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.nn_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 5:
-        if ftype == TType.STRING:
-          self.columnName = iprot.readString()
+        if ftype == TType.BOOL:
+          self.enable_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.BOOL:
+          self.validate_cstr = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.rely_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
       else:
@@ -1008,29 +1062,34 @@ class HiveObjectRef:
     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('HiveObjectRef')
-    if self.objectType is not None:
-      oprot.writeFieldBegin('objectType', TType.I32, 1)
-      oprot.writeI32(self.objectType)
+    oprot.writeStructBegin('SQLNotNullConstraint')
+    if self.table_db is not None:
+      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
-    if self.dbName is not None:
-      oprot.writeFieldBegin('dbName', TType.STRING, 2)
-      oprot.writeString(self.dbName)
+    if self.table_name is not None:
+      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
-    if self.objectName is not None:
-      oprot.writeFieldBegin('objectName', TType.STRING, 3)
-      oprot.writeString(self.objectName)
+    if self.column_name is not None:
+      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
-    if self.partValues is not None:
-      oprot.writeFieldBegin('partValues', TType.LIST, 4)
-      oprot.writeListBegin(TType.STRING, len(self.partValues))
-      for iter13 in self.partValues:
-        oprot.writeString(iter13)
-      oprot.writeListEnd()
+    if self.nn_name is not None:
+      oprot.writeFieldBegin('nn_name', TType.STRING, 4)
+      oprot.writeString(self.nn_name)
       oprot.writeFieldEnd()
-    if self.columnName is not None:
-      oprot.writeFieldBegin('columnName', TType.STRING, 5)
-      oprot.writeString(self.columnName)
+    if self.enable_cstr is not None:
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 5)
+      oprot.writeBool(self.enable_cstr)
+      oprot.writeFieldEnd()
+    if self.validate_cstr is not None:
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 6)
+      oprot.writeBool(self.validate_cstr)
+      oprot.writeFieldEnd()
+    if self.rely_cstr is not None:
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 7)
+      oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -1041,11 +1100,13 @@ class HiveObjectRef:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.objectType)
-    value = (value * 31) ^ hash(self.dbName)
-    value = (value * 31) ^ hash(self.objectName)
-    value = (value * 31) ^ hash(self.partValues)
-    value = (value * 31) ^ hash(self.columnName)
+    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.nn_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):
@@ -1059,30 +1120,268 @@ class HiveObjectRef:
   def __ne__(self, other):
     return not (self == other)
 
-class PrivilegeGrantInfo:
+class Type:
   """
   Attributes:
-   - privilege
-   - createTime
-   - grantor
-   - grantorType
-   - grantOption
+   - name
+   - type1
+   - type2
+   - fields
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'privilege', None, None, ), # 1
-    (2, TType.I32, 'createTime', None, None, ), # 2
-    (3, TType.STRING, 'grantor', None, None, ), # 3
-    (4, TType.I32, 'grantorType', None, None, ), # 4
-    (5, TType.BOOL, 'grantOption', None, None, ), # 5
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRING, 'type1', None, None, ), # 2
+    (3, TType.STRING, 'type2', None, None, ), # 3
+    (4, TType.LIST, 'fields', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 4
   )
 
-  def __init__(self, privilege=None, createTime=None, grantor=None, grantorType=None, grantOption=None,):
-    self.privilege = privilege
-    self.createTime = createTime
-    self.grantor = grantor
-    self.grantorType = grantorType
+  def __init__(self, name=None, type1=None, type2=None, fields=None,):
+    self.name = name
+    self.type1 = type1
+    self.type2 = type2
+    self.fields = fields
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.type1 = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.type2 = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.fields = []
+          (_etype3, _size0) = iprot.readListBegin()
+          for _i4 in xrange(_size0):
+            _elem5 = FieldSchema()
+            _elem5.read(iprot)
+            self.fields.append(_elem5)
+          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('Type')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.type1 is not None:
+      oprot.writeFieldBegin('type1', TType.STRING, 2)
+      oprot.writeString(self.type1)
+      oprot.writeFieldEnd()
+    if self.type2 is not None:
+      oprot.writeFieldBegin('type2', TType.STRING, 3)
+      oprot.writeString(self.type2)
+      oprot.writeFieldEnd()
+    if self.fields is not None:
+      oprot.writeFieldBegin('fields', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRUCT, len(self.fields))
+      for iter6 in self.fields:
+        iter6.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.type1)
+    value = (value * 31) ^ hash(self.type2)
+    value = (value * 31) ^ hash(self.fields)
+    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 HiveObjectRef:
+  """
+  Attributes:
+   - objectType
+   - dbName
+   - objectName
+   - partValues
+   - columnName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'objectType', None, None, ), # 1
+    (2, TType.STRING, 'dbName', None, None, ), # 2
+    (3, TType.STRING, 'objectName', None, None, ), # 3
+    (4, TType.LIST, 'partValues', (TType.STRING,None), None, ), # 4
+    (5, TType.STRING, 'columnName', None, None, ), # 5
+  )
+
+  def __init__(self, objectType=None, dbName=None, objectName=None, partValues=None, columnName=None,):
+    self.objectType = objectType
+    self.dbName = dbName
+    self.objectName = objectName
+    self.partValues = partValues
+    self.columnName = columnName
+
+  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.I32:
+          self.objectType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.objectName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.partValues = []
+          (_etype10, _size7) = iprot.readListBegin()
+          for _i11 in xrange(_size7):
+            _elem12 = iprot.readString()
+            self.partValues.append(_elem12)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.columnName = 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('HiveObjectRef')
+    if self.objectType is not None:
+      oprot.writeFieldBegin('objectType', TType.I32, 1)
+      oprot.writeI32(self.objectType)
+      oprot.writeFieldEnd()
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.objectName is not None:
+      oprot.writeFieldBegin('objectName', TType.STRING, 3)
+      oprot.writeString(self.objectName)
+      oprot.writeFieldEnd()
+    if self.partValues is not None:
+      oprot.writeFieldBegin('partValues', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.partValues))
+      for iter13 in self.partValues:
+        oprot.writeString(iter13)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.columnName is not None:
+      oprot.writeFieldBegin('columnName', TType.STRING, 5)
+      oprot.writeString(self.columnName)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.objectType)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.objectName)
+    value = (value * 31) ^ hash(self.partValues)
+    value = (value * 31) ^ hash(self.columnName)
+    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 PrivilegeGrantInfo:
+  """
+  Attributes:
+   - privilege
+   - createTime
+   - grantor
+   - grantorType
+   - grantOption
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'privilege', None, None, ), # 1
+    (2, TType.I32, 'createTime', None, None, ), # 2
+    (3, TType.STRING, 'grantor', None, None, ), # 3
+    (4, TType.I32, 'grantorType', None, None, ), # 4
+    (5, TType.BOOL, 'grantOption', None, None, ), # 5
+  )
+
+  def __init__(self, privilege=None, createTime=None, grantor=None, grantorType=None, grantOption=None,):
+    self.privilege = privilege
+    self.createTime = createTime
+    self.grantor = grantor
+    self.grantorType = grantorType
     self.grantOption = grantOption
 
   def read(self, iprot):
@@ -5379,28 +5678,423 @@ class ColumnStatisticsObj:
       oprot.writeFieldBegin('colType', TType.STRING, 2)
       oprot.writeString(self.colType)
       oprot.writeFieldEnd()
-    if self.statsData is not None:
-      oprot.writeFieldBegin('statsData', TType.STRUCT, 3)
-      self.statsData.write(oprot)
+    if self.statsData is not None:
+      oprot.writeFieldBegin('statsData', TType.STRUCT, 3)
+      self.statsData.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.colName is None:
+      raise TProtocol.TProtocolException(message='Required field colName is unset!')
+    if self.colType is None:
+      raise TProtocol.TProtocolException(message='Required field colType is unset!')
+    if self.statsData is None:
+      raise TProtocol.TProtocolException(message='Required field statsData is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.colName)
+    value = (value * 31) ^ hash(self.colType)
+    value = (value * 31) ^ hash(self.statsData)
+    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 ColumnStatisticsDesc:
+  """
+  Attributes:
+   - isTblLevel
+   - dbName
+   - tableName
+   - partName
+   - lastAnalyzed
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.BOOL, 'isTblLevel', None, None, ), # 1
+    (2, TType.STRING, 'dbName', None, None, ), # 2
+    (3, TType.STRING, 'tableName', None, None, ), # 3
+    (4, TType.STRING, 'partName', None, None, ), # 4
+    (5, TType.I64, 'lastAnalyzed', None, None, ), # 5
+  )
+
+  def __init__(self, isTblLevel=None, dbName=None, tableName=None, partName=None, lastAnalyzed=None,):
+    self.isTblLevel = isTblLevel
+    self.dbName = dbName
+    self.tableName = tableName
+    self.partName = partName
+    self.lastAnalyzed = lastAnalyzed
+
+  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.BOOL:
+          self.isTblLevel = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.partName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I64:
+          self.lastAnalyzed = iprot.readI64()
+        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('ColumnStatisticsDesc')
+    if self.isTblLevel is not None:
+      oprot.writeFieldBegin('isTblLevel', TType.BOOL, 1)
+      oprot.writeBool(self.isTblLevel)
+      oprot.writeFieldEnd()
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tableName is not None:
+      oprot.writeFieldBegin('tableName', TType.STRING, 3)
+      oprot.writeString(self.tableName)
+      oprot.writeFieldEnd()
+    if self.partName is not None:
+      oprot.writeFieldBegin('partName', TType.STRING, 4)
+      oprot.writeString(self.partName)
+      oprot.writeFieldEnd()
+    if self.lastAnalyzed is not None:
+      oprot.writeFieldBegin('lastAnalyzed', TType.I64, 5)
+      oprot.writeI64(self.lastAnalyzed)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.isTblLevel is None:
+      raise TProtocol.TProtocolException(message='Required field isTblLevel is unset!')
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tableName is None:
+      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.isTblLevel)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tableName)
+    value = (value * 31) ^ hash(self.partName)
+    value = (value * 31) ^ hash(self.lastAnalyzed)
+    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 ColumnStatistics:
+  """
+  Attributes:
+   - statsDesc
+   - statsObj
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'statsDesc', (ColumnStatisticsDesc, ColumnStatisticsDesc.thrift_spec), None, ), # 1
+    (2, TType.LIST, 'statsObj', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 2
+  )
+
+  def __init__(self, statsDesc=None, statsObj=None,):
+    self.statsDesc = statsDesc
+    self.statsObj = statsObj
+
+  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.statsDesc = ColumnStatisticsDesc()
+          self.statsDesc.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.LIST:
+          self.statsObj = []
+          (_etype242, _size239) = iprot.readListBegin()
+          for _i243 in xrange(_size239):
+            _elem244 = ColumnStatisticsObj()
+            _elem244.read(iprot)
+            self.statsObj.append(_elem244)
+          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('ColumnStatistics')
+    if self.statsDesc is not None:
+      oprot.writeFieldBegin('statsDesc', TType.STRUCT, 1)
+      self.statsDesc.write(oprot)
+      oprot.writeFieldEnd()
+    if self.statsObj is not None:
+      oprot.writeFieldBegin('statsObj', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
+      for iter245 in self.statsObj:
+        iter245.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.statsDesc is None:
+      raise TProtocol.TProtocolException(message='Required field statsDesc is unset!')
+    if self.statsObj is None:
+      raise TProtocol.TProtocolException(message='Required field statsObj is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.statsDesc)
+    value = (value * 31) ^ hash(self.statsObj)
+    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 AggrStats:
+  """
+  Attributes:
+   - colStats
+   - partsFound
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 1
+    (2, TType.I64, 'partsFound', None, None, ), # 2
+  )
+
+  def __init__(self, colStats=None, partsFound=None,):
+    self.colStats = colStats
+    self.partsFound = partsFound
+
+  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.colStats = []
+          (_etype249, _size246) = iprot.readListBegin()
+          for _i250 in xrange(_size246):
+            _elem251 = ColumnStatisticsObj()
+            _elem251.read(iprot)
+            self.colStats.append(_elem251)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.partsFound = iprot.readI64()
+        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('AggrStats')
+    if self.colStats is not None:
+      oprot.writeFieldBegin('colStats', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
+      for iter252 in self.colStats:
+        iter252.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.partsFound is not None:
+      oprot.writeFieldBegin('partsFound', TType.I64, 2)
+      oprot.writeI64(self.partsFound)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.colStats is None:
+      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
+    if self.partsFound is None:
+      raise TProtocol.TProtocolException(message='Required field partsFound is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.colStats)
+    value = (value * 31) ^ hash(self.partsFound)
+    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 SetPartitionsStatsRequest:
+  """
+  Attributes:
+   - colStats
+   - needMerge
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatistics, ColumnStatistics.thrift_spec)), None, ), # 1
+    (2, TType.BOOL, 'needMerge', None, None, ), # 2
+  )
+
+  def __init__(self, colStats=None, needMerge=None,):
+    self.colStats = colStats
+    self.needMerge = needMerge
+
+  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.colStats = []
+          (_etype256, _size253) = iprot.readListBegin()
+          for _i257 in xrange(_size253):
+            _elem258 = ColumnStatistics()
+            _elem258.read(iprot)
+            self.colStats.append(_elem258)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.needMerge = 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('SetPartitionsStatsRequest')
+    if self.colStats is not None:
+      oprot.writeFieldBegin('colStats', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
+      for iter259 in self.colStats:
+        iter259.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.needMerge is not None:
+      oprot.writeFieldBegin('needMerge', TType.BOOL, 2)
+      oprot.writeBool(self.needMerge)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.colName is None:
-      raise TProtocol.TProtocolException(message='Required field colName is unset!')
-    if self.colType is None:
-      raise TProtocol.TProtocolException(message='Required field colType is unset!')
-    if self.statsData is None:
-      raise TProtocol.TProtocolException(message='Required field statsData is unset!')
+    if self.colStats is None:
+      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.colName)
-    value = (value * 31) ^ hash(self.colType)
-    value = (value * 31) ^ hash(self.statsData)
+    value = (value * 31) ^ hash(self.colStats)
+    value = (value * 31) ^ hash(self.needMerge)
     return value
 
   def __repr__(self):
@@ -5414,31 +6108,22 @@ class ColumnStatisticsObj:
   def __ne__(self, other):
     return not (self == other)
 
-class ColumnStatisticsDesc:
+class Schema:
   """
   Attributes:
-   - isTblLevel
-   - dbName
-   - tableName
-   - partName
-   - lastAnalyzed
+   - fieldSchemas
+   - properties
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.BOOL, 'isTblLevel', None, None, ), # 1
-    (2, TType.STRING, 'dbName', None, None, ), # 2
-    (3, TType.STRING, 'tableName', None, None, ), # 3
-    (4, TType.STRING, 'partName', None, None, ), # 4
-    (5, TType.I64, 'lastAnalyzed', None, None, ), # 5
+    (1, TType.LIST, 'fieldSchemas', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 1
+    (2, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 2
   )
 
-  def __init__(self, isTblLevel=None, dbName=None, tableName=None, partName=None, lastAnalyzed=None,):
-    self.isTblLevel = isTblLevel
-    self.dbName = dbName
-    self.tableName = tableName
-    self.partName = partName
-    self.lastAnalyzed = lastAnalyzed
+  def __init__(self, fieldSchemas=None, properties=None,):
+    self.fieldSchemas = fieldSchemas
+    self.properties = properties
 
   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:
@@ -5450,28 +6135,25 @@ class ColumnStatisticsDesc:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.BOOL:
-          self.isTblLevel = iprot.readBool()
+        if ftype == TType.LIST:
+          self.fieldSchemas = []
+          (_etype263, _size260) = iprot.readListBegin()
+          for _i264 in xrange(_size260):
+            _elem265 = FieldSchema()
+            _elem265.read(iprot)
+            self.fieldSchemas.append(_elem265)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.STRING:
-          self.dbName = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRING:
-          self.tableName = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.STRING:
-          self.partName = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 5:
-        if ftype == TType.I64:
-          self.lastAnalyzed = iprot.readI64()
+        if ftype == TType.MAP:
+          self.properties = {}
+          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
+          for _i270 in xrange(_size266):
+            _key271 = iprot.readString()
+            _val272 = iprot.readString()
+            self.properties[_key271] = _val272
+          iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -5483,47 +6165,33 @@ class ColumnStatisticsDesc:
     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('ColumnStatisticsDesc')
-    if self.isTblLevel is not None:
-      oprot.writeFieldBegin('isTblLevel', TType.BOOL, 1)
-      oprot.writeBool(self.isTblLevel)
-      oprot.writeFieldEnd()
-    if self.dbName is not None:
-      oprot.writeFieldBegin('dbName', TType.STRING, 2)
-      oprot.writeString(self.dbName)
-      oprot.writeFieldEnd()
-    if self.tableName is not None:
-      oprot.writeFieldBegin('tableName', TType.STRING, 3)
-      oprot.writeString(self.tableName)
-      oprot.writeFieldEnd()
-    if self.partName is not None:
-      oprot.writeFieldBegin('partName', TType.STRING, 4)
-      oprot.writeString(self.partName)
+    oprot.writeStructBegin('Schema')
+    if self.fieldSchemas is not None:
+      oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
+      for iter273 in self.fieldSchemas:
+        iter273.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
-    if self.lastAnalyzed is not None:
-      oprot.writeFieldBegin('lastAnalyzed', TType.I64, 5)
-      oprot.writeI64(self.lastAnalyzed)
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 2)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter274,viter275 in self.properties.items():
+        oprot.writeString(kiter274)
+        oprot.writeString(viter275)
+      oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.isTblLevel is None:
-      raise TProtocol.TProtocolException(message='Required field isTblLevel is unset!')
-    if self.dbName is None:
-      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
-    if self.tableName is None:
-      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.isTblLevel)
-    value = (value * 31) ^ hash(self.dbName)
-    value = (value * 31) ^ hash(self.tableName)
-    value = (value * 31) ^ hash(self.partName)
-    value = (value * 31) ^ hash(self.lastAnalyzed)
+    value = (value * 31) ^ hash(self.fieldSchemas)
+    value = (value * 31) ^ hash(self.properties)
     return value
 
   def __repr__(self):
@@ -5537,22 +6205,19 @@ class ColumnStatisticsDesc:
   def __ne__(self, other):
     return not (self == other)
 
-class ColumnStatistics:
+class EnvironmentContext:
   """
   Attributes:
-   - statsDesc
-   - statsObj
+   - properties
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRUCT, 'statsDesc', (ColumnStatisticsDesc, ColumnStatisticsDesc.thrift_spec), None, ), # 1
-    (2, TType.LIST, 'statsObj', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 2
+    (1, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 1
   )
 
-  def __init__(self, statsDesc=None, statsObj=None,):
-    self.statsDesc = statsDesc
-    self.statsObj = statsObj
+  def __init__(self, properties=None,):
+    self.properties = properties
 
   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:
@@ -5564,20 +6229,14 @@ class ColumnStatistics:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.STRUCT:
-          self.statsDesc = ColumnStatisticsDesc()
-          self.statsDesc.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.LIST:
-          self.statsObj = []
-          (_etype242, _size239) = iprot.readListBegin()
-          for _i243 in xrange(_size239):
-            _elem244 = ColumnStatisticsObj()
-            _elem244.read(iprot)
-            self.statsObj.append(_elem244)
-          iprot.readListEnd()
+        if ftype == TType.MAP:
+          self.properties = {}
+          (_ktype277, _vtype278, _size276 ) = iprot.readMapBegin()
+          for _i280 in xrange(_size276):
+            _key281 = iprot.readString()
+            _val282 = iprot.readString()
+            self.properties[_key281] = _val282
+          iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -5589,33 +6248,25 @@ class ColumnStatistics:
     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('ColumnStatistics')
-    if self.statsDesc is not None:
-      oprot.writeFieldBegin('statsDesc', TType.STRUCT, 1)
-      self.statsDesc.write(oprot)
-      oprot.writeFieldEnd()
-    if self.statsObj is not None:
-      oprot.writeFieldBegin('statsObj', TType.LIST, 2)
-      oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
-      for iter245 in self.statsObj:
-        iter245.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('EnvironmentContext')
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter283,viter284 in self.properties.items():
+        oprot.writeString(kiter283)
+        oprot.writeString(viter284)
+      oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.statsDesc is None:
-      raise TProtocol.TProtocolException(message='Required field statsDesc is unset!')
-    if self.statsObj is None:
-      raise TProtocol.TProtocolException(message='Required field statsObj is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.statsDesc)
-    value = (value * 31) ^ hash(self.statsObj)
+    value = (value * 31) ^ hash(self.properties)
     return value
 
   def __repr__(self):
@@ -5629,22 +6280,22 @@ class ColumnStatistics:
   def __ne__(self, other):
     return not (self == other)
 
-class AggrStats:
+class PrimaryKeysRequest:
   """
   Attributes:
-   - colStats
-   - partsFound
+   - db_name
+   - tbl_name
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatisticsObj, ColumnStatisticsObj.thrift_spec)), None, ), # 1
-    (2, TType.I64, 'partsFound', None, None, ), # 2
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
   )
 
-  def __init__(self, colStats=None, partsFound=None,):
-    self.colStats = colStats
-    self.partsFound = partsFound
+  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:
@@ -5656,19 +6307,13 @@ class AggrStats:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.LIST:
-          self.colStats = []
-          (_etype249, _size246) = iprot.readListBegin()
-          for _i250 in xrange(_size246):
-            _elem251 = ColumnStatisticsObj()
-            _elem251.read(iprot)
-            self.colStats.append(_elem251)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.db_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.I64:
-          self.partsFound = iprot.readI64()
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -5680,33 +6325,30 @@ class AggrStats:
     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('AggrStats')
-    if self.colStats is not None:
-      oprot.writeFieldBegin('colStats', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter252 in self.colStats:
-        iter252.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('PrimaryKeysRequest')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
       oprot.writeFieldEnd()
-    if self.partsFound is not None:
-      oprot.writeFieldBegin('partsFound', TType.I64, 2)
-      oprot.writeI64(self.partsFound)
+    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.colStats is None:
-      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
-    if self.partsFound is None:
-      raise TProtocol.TProtocolException(message='Required field partsFound is unset!')
+    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.colStats)
-    value = (value * 31) ^ hash(self.partsFound)
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
     return value
 
   def __repr__(self):
@@ -5720,22 +6362,19 @@ class AggrStats:
   def __ne__(self, other):
     return not (self == other)
 
-class SetPartitionsStatsRequest:
+class PrimaryKeysResponse:
   """
   Attributes:
-   - colStats
-   - needMerge
+   - primaryKeys
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatistics, ColumnStatistics.thrift_spec)), None, ), # 1
-    (2, TType.BOOL, 'needMerge', None, None, ), # 2
+    (1, TType.LIST, 'primaryKeys', (TType.STRUCT,(SQLPrimaryKey, SQLPrimaryKey.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, colStats=None, needMerge=None,):
-    self.colStats = colStats
-    self.needMerge = needMerge
+  def __init__(self, primaryKeys=None,):
+    self.primaryKeys = primaryKeys
 
   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:
@@ -5748,20 +6387,15 @@ class SetPartitionsStatsRequest:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.colStats = []
-          (_etype256, _size253) = iprot.readListBegin()
-          for _i257 in xrange(_size253):
-            _elem258 = ColumnStatistics()
-            _elem258.read(iprot)
-            self.colStats.append(_elem258)
+          self.primaryKeys = []
+          (_etype288, _size285) = iprot.readListBegin()
+          for _i289 in xrange(_size285):
+            _elem290 = SQLPrimaryKey()
+            _elem290.read(iprot)
+            self.primaryKeys.append(_elem290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.BOOL:
-          self.needMerge = iprot.readBool()
-        else:
-          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -5771,31 +6405,26 @@ class SetPartitionsStatsRequest:
     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('SetPartitionsStatsRequest')
-    if self.colStats is not None:
-      oprot.writeFieldBegin('colStats', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter259 in self.colStats:
-        iter259.write(oprot)
+    oprot.writeStructBegin('PrimaryKeysResponse')
+    if self.primaryKeys is not None:
+      oprot.writeFieldBegin('primaryKeys', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
+      for iter291 in self.primaryKeys:
+        iter291.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
-    if self.needMerge is not None:
-      oprot.writeFieldBegin('needMerge', TType.BOOL, 2)
-      oprot.writeBool(self.needMerge)
-      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.colStats is None:
-      raise TProtocol.TProtocolException(message='Required field colStats is unset!')
+    if self.primaryKeys is None:
+      raise TProtocol.TProtocolException(message='Required field primaryKeys is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.colStats)
-    value = (value * 31) ^ hash(self.needMerge)
+    value = (value * 31) ^ hash(self.primaryKeys)
     return value
 
   def __repr__(self):
@@ -5809,22 +6438,28 @@ class SetPartitionsStatsRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class Schema:
+class ForeignKeysRequest:
   """
   Attributes:
-   - fieldSchemas
-   - properties
+   - parent_db_name
+   - parent_tbl_name
+   - foreign_db_name
+   - foreign_tbl_name
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'fieldSchemas', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 1
-    (2, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 2
+    (1, TType.STRING, 'parent_db_name', None, None, ), # 1
+    (2, TType.STRING, 'parent_tbl_name', None, None, ), # 2
+    (3, TType.STRING, 'foreign_db_name', None, None, ), # 3
+    (4, TType.STRING, 'foreign_tbl_name', None, None, ), # 4
   )
 
-  def __init__(self, fieldSchemas=None, properties=None,):
-    self.fieldSchemas = fieldSchemas
-    self.properties = properties
+  def __init__(self, parent_db_name=None, parent_tbl_name=None, foreign_db_name=None, foreign_tbl_name=None,):
+    self.parent_db_name = parent_db_name
+    self.parent_tbl_name = parent_tbl_name
+    self.foreign_db_name = foreign_db_name
+    self.foreign_tbl_name = foreign_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:
@@ -5836,25 +6471,23 @@ class Schema:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.LIST:
-          self.fieldSchemas = []
-          (_etype263, _size260) = iprot.readListBegin()
-          for _i264 in xrange(_size260):
-            _elem265 = FieldSchema()
-            _elem265.read(iprot)
-            self.fieldSchemas.append(_elem265)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.parent_db_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.MAP:
-          self.properties = {}
-          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
-          for _i270 in xrange(_size266):
-            _key271 = iprot.readString()
-            _val272 = iprot.readString()
-            self.properties[_key271] = _val272
-          iprot.readMapEnd()
+        if ftype == TType.STRING:
+          self.parent_tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.foreign_db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.foreign_tbl_name = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -5866,21 +6499,22 @@ class Schema:
     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('Schema')
-    if self.fieldSchemas is not None:
-      oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
-      for iter273 in self.fieldSchemas:
-        iter273.write(oprot)
-      oprot.writeListEnd()
+    oprot.writeStructBegin('ForeignKeysRequest')
+    if self.parent_db_name is not None:
+      oprot.writeFieldBegin('parent_db_name', TType.STRING, 1)
+      oprot.writeString(self.parent_db_name)
       oprot.writeFieldEnd()
-    if self.properties is not None:
-      oprot.writeFieldBegin('properties', TType.MAP, 2)
-      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter274,viter275 in self.properties.items():
-        oprot.writeString(kiter274)
-        oprot.writeString(viter275)
-      oprot.writeMapEnd()
+    if self.parent_tbl_name is not None:
+      oprot.writeFieldBegin('parent_tbl_name', TType.STRING, 2)
+      oprot.writeString(self.parent_tbl_name)
+      oprot.writeFieldEnd()
+    if self.foreign_db_name is not None:
+      oprot.writeFieldBegin('foreign_db_name', TType.STRING, 3)
+      oprot.writeString(self.foreign_db_name)
+      oprot.writeFieldEnd()
+    if self.foreign_tbl_name is not None:
+      oprot.writeFieldBegin('foreign_tbl_name', TType.STRING, 4)
+      oprot.writeString(self.foreign_tbl_name)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -5891,8 +6525,10 @@ class Schema:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.fieldSchemas)
-    value = (value * 31) ^ hash(self.properties)
+    value = (value * 31) ^ hash(self.parent_db_name)
+    value = (value * 31) ^ hash(self.parent_tbl_name)
+    value = (value * 31) ^ hash(self.foreign_db_name)
+    value = (value * 31) ^ hash(self.foreign_tbl_name)
     return value
 
   def __repr__(self):
@@ -5906,19 +6542,19 @@ class Schema:
   def __ne__(self, other):
     return not (self == other)
 
-class EnvironmentContext:
+class ForeignKeysResponse:
   """
   Attributes:
-   - properties
+   - foreignKeys
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 1
+    (1, TType.LIST, 'foreignKeys', (TType.STRUCT,(SQLForeignKey, SQLForeignKey.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, properties=None,):
-    self.properties = properties
+  def __init__(self, foreignKeys=None,):
+    self.foreignKeys = foreignKeys
 
   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:
@@ -5930,14 +6566,14 @@ class EnvironmentContext:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.MAP:
-          self.properties = {}
-          (_ktype277, _vtype278, _size276 ) = iprot.readMapBegin()
-          for _i280 in xrange(_size276):
-            _key281 = iprot.readString()
-            _val282 = iprot.readString()
-            self.properties[_key281] = _val282
-          iprot.readMapEnd()
+        if ftype == TType.LIST:
+          self.foreignKeys = []
+          (_etype295, _size292) = iprot.readListBegin()
+          for _i296 in xrange(_size292):
+            _elem297 = SQLForeignKey()
+            _elem297.read(iprot)
+            self.foreignKeys.append(_elem297)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -5949,25 +6585,26 @@ class EnvironmentContext:
     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('EnvironmentContext')
-    if self.properties is not None:
-      oprot.writeFieldBegin('properties', TType.MAP, 1)
-      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter283,viter284 in self.properties.items():
-        oprot.writeString(kiter283)
-        oprot.writeString(viter284)
-      oprot.writeMapEnd()
+    oprot.writeStructBegin('ForeignKeysResponse')
+    if self.foreignKeys is not None:
+      oprot.writeFieldBegin('foreignKeys', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
+      for iter298 in self.foreignKeys:
+        iter298.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
+    if self.foreignKeys is None:
+      raise TProtocol.TProtocolException(message='Required field foreignKeys is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.properties)
+    value = (value * 31) ^ hash(self.foreignKeys)
     return value
 
   def __repr__(self):
@@ -5981,7 +6618,7 @@ class EnvironmentContext:
   def __ne__(self, other):
     return not (self == other)
 
-class PrimaryKeysRequest:
+class UniqueConstraintsRequest:
   """
   Attributes:
    - db_name
@@ -6026,7 +6663,7 @@ class PrimaryKeysRequest:
     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('PrimaryKeysRequest')
+    oprot.writeStructBegin('UniqueConstraintsRequest')
     if self.db_name is not None:
       oprot.writeFieldBegin('db_name', TType.STRING, 1)
       oprot.writeString(self.db_name)
@@ -6063,19 +6700,19 @@ class PrimaryKeysRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class PrimaryKeysResponse:
+class UniqueConstraintsResponse:
   """
   Attributes:
-   - primaryKeys
+   - uniqueConstraints
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'primaryKeys', (TType.STRUCT,(SQLPrimaryKey, SQLPrimaryKey.thrift_spec)), None, ), # 1
+    (1, TType.LIST, 'uniqueConstraints', (TType.STRUCT,(SQLUniqueConstraint, SQLUniqueConstraint.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, primaryKeys=None,):
-    self.primaryKeys = primaryKeys
+  def __init__(self, uniqueConstraints=None,):
+    self.uniqueConstraints = uniqueConstraints
 
   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:
@@ -6088,12 +6725,12 @@ class PrimaryKeysResponse:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.primaryKeys = []
-          (_etype288, _size285) = iprot.readListBegin()
-          for _i289 in xrange(_size285):
-            _elem290 = SQLPrimaryKey()
-            _elem290.read(iprot)
-            self.primaryKeys.append(_elem290)
+          self.uniqueConstraints = []
+          (_etype302, _size299) = iprot.readListBegin()
+          for _i303 in xrange(_size299):
+            _elem304 = SQLUniqueConstraint()
+            _elem304.read(iprot)
+            self.uniqueConstraints.append(_elem304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6106,26 +6743,26 @@ class PrimaryKeysResponse:
     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('PrimaryKeysResponse')
-    if self.primaryKeys is not None:
-      oprot.writeFieldBegin('primaryKeys', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter291 in self.primaryKeys:
-        iter291.write(oprot)
+    oprot.writeStructBegin('UniqueConstraintsResponse')
+    if self.uniqueConstraints is not None:
+      oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
+      for iter305 in self.uniqueConstraints:
+        iter305.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.primaryKeys is None:
-      raise TProtocol.TProtocolException(message='Required field primaryKeys is unset!')
+    if self.uniqueConstraints is None:
+      raise TProtocol.TProtocolException(message='Required field uniqueConstraints is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.primaryKeys)
+    value = (value * 31) ^ hash(self.uniqueConstraints)
     return value
 
   def __repr__(self):
@@ -6139,28 +6776,22 @@ class PrimaryKeysResponse:
   def __ne__(self, other):
     return not (self == other)
 
-class ForeignKeysRequest:
+class NotNullConstraintsRequest:
   """
   Attributes:
-   - parent_db_name
-   - parent_tbl_name
-   - foreign_db_name
-   - foreign_tbl_name
+   - db_name
+   - tbl_name
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'parent_db_name', None, None, ), # 1
-    (2, TType.STRING, 'parent_tbl_name', None, None, ), # 2
-    (3, TType.STRING, 'foreign_db_name', None, None, ), # 3
-    (4, TType.STRING, 'foreign_tbl_name', None, None, ), # 4
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
   )
 
-  def __init__(self, parent_db_name=None, parent_tbl_name=None, foreign_db_name=None, foreign_tbl_name=None,):
-    self.parent_db_name = parent_db_name
-    self.parent_tbl_name = parent_tbl_name
-    self.foreign_db_name = foreign_db_name
-    self.foreign_tbl_name = foreign_tbl_name
+  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:
@@ -6173,22 +6804,12 @@ class ForeignKeysRequest:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.parent_db_name = iprot.readString()
+          self.db_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.parent_tbl_name = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRING:
-          self.foreign_db_name = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.STRING:
-          self.foreign_tbl_name = iprot.readString()
+          self.tbl_name = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -6200,36 +6821,30 @@ class ForeignKeysRequest:
     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('ForeignKeysRequest')
-    if self.parent_db_name is not None:
-      oprot.writeFieldBegin('parent_db_name', TType.STRING, 1)
-      oprot.writeString(self.parent_db_name)
-      oprot.writeFieldEnd()
-    if self.parent_tbl_name is not None:
-      oprot.writeFieldBegin('parent_tbl_name', TType.STRING, 2)
-      oprot.writeString(self.parent_tbl_name)
-      oprot.writeFieldEnd()
-    if self.foreign_db_name is not None:
-      oprot.writeFieldBegin('foreign_db_name', TType.STRING, 3)
-      oprot.writeString(self.foreign_db_name)
+    oprot.writeStructBegin('NotNullConstraintsRequest')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
       oprot.writeFieldEnd()
-    if self.foreign_tbl_name is not None:
-      oprot.writeFieldBegin('foreign_tbl_name', TType.STRING, 4)
-      oprot.writeString(self.foreign_tbl_name)
+    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.parent_db_name)
-    value = (value * 31) ^ hash(self.parent_tbl_name)
-    value = (value * 31) ^ hash(self.foreign_db_name)
-    value = (value * 31) ^ hash(self.foreign_tbl_name)
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
     return value
 
   def __repr__(self):
@@ -6243,19 +6858,19 @@ class ForeignKeysRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class ForeignKeysResponse:
+class NotNullConstraintsResponse:
   """
   Attributes:
-   - foreignKeys
+   - notNullConstraints
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'foreignKeys', (TType.STRUCT,(SQLForeignKey, SQLForeignKey.thrift_spec)), None, ), # 1
+    (1, TType.LIST, 'notNullConstraints', (TType.STRUCT,(SQLNotNullConstraint, SQLNotNullConstraint.thrift_spec)), None, ), # 1
   )
 
-  def __init__(self, foreignKeys=None,):
-    self.foreignKeys = foreignKeys
+  def __init__(self, notNullConstraints=None,):
+    self.notNullConstraints = notNullConstraints
 
   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:
@@ -6268,12 +6883,12 @@ class ForeignKeysResponse:
         break
       if fid == 1:
         if ftype == TType.LIST:
-          self.foreignKeys = []
-          (_etype295, _size292) = iprot.readListBegin()
-          for _i296 in xrange(_size292):
-            _elem297 = SQLForeignKey()
-            _elem297.read(iprot)
-            self.foreignKeys.append(_elem297)
+          self.notNullConstraints = []
+          (_etype309, _size306) = iprot.readListBegin()
+          for _i310 in xrange(_size306):
+            _elem311 = SQLNotNullConstraint()
+            _elem311.read(iprot)
+            self.notNullConstraints.append(_elem311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6286,26 +6901,26 @@ class ForeignKeysResponse:
     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('ForeignKeysResponse')
-    if self.foreignKeys is not None:
-      oprot.writeFieldBegin('foreignKeys', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter298 in self.foreignKeys:
-        iter298.write(oprot)
+    oprot.writeStructBegin('NotNullConstraintsResponse')
+    if self.notNullConstraints is not None:
+      oprot.writeFieldBegin('notNullConstraints', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
+      for iter312 in self.notNullConstraints:
+        iter312.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.foreignKeys is None:
-      raise TProtocol.TProtocolException(message='Required field foreignKeys is unset!')
+    if self.notNullConstraints is None:
+      raise TProtocol.TProtocolException(message='Required field notNullConstraints is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.foreignKeys)
+    value = (value * 31) ^ hash(self.notNullConstraints)
     return value
 
   def __repr__(self):
@@ -6442,11 +7057,11 @@ class AddPrimaryKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.primaryKeyCols = []
-          (_etype302, _size299) = iprot.readListBegin()
-          for _i303 in xrange(_size299):
-            _elem304 = SQLPrimaryKey()
-            _elem304.read(iprot)
-            self.primaryKeyCols.append(_elem304)
+          (_etype316, _size313) = iprot.readListBegin()
+          for _i317 in xrange(_size313):
+            _elem318 = SQLPrimaryKey()
+            _elem318.read(iprot)
+            self.primaryKeyCols.append(_elem318)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6463,8 +7078,8 @@ class AddPrimaryKeyRequest:
     if self.primaryKeyCols is not None:
       oprot.writeFieldBegin('primaryKeyCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeyCols))
-      for iter305 in self.primaryKeyCols:
-        iter305.write(oprot)
+      for iter319 in self.primaryKeyCols:
+        iter319.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6518,11 +7133,11 @@ class AddForeignKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.foreignKeyCols = []
-          (_etype309, _size306) = iprot.readListBegin()
-          for _i310 in xrange(_size306):
-            _elem311 = SQLForeignKey()
-            _elem311.read(iprot)
-            self.foreignKeyCols.append(_elem311)
+          (_etype323, _size320) = iprot.readListBegin()
+          for _i324 in xrange(_size320):
+            _elem325 = SQLForeignKey()
+            _elem325.read(iprot)
+            self.foreignKeyCols.append(_elem325)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6539,8 +7154,8 @@ class AddForeignKeyRequest:
     if self.foreignKeyCols is not None:
       oprot.writeFieldBegin('foreignKeyCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeyCols))
-      for iter312 in self.foreignKeyCols:
-        iter312.write(oprot)
+      for iter326 in self.foreignKeyCols:
+        iter326.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6568,6 +7183,158 @@ class AddForeignKeyRequest:
   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 = []
+          (_etype330, _size327) = iprot.readListBegin()
+          for _i331 in xrange(_size327):
+            _elem332 = SQLUniqueConstraint()
+            _elem332.read(iprot)
+            self.uniqueConstraintCols.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('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.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    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.uniqueConstraintCols)
+    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 AddNotNullConstraintRequest:
+  """
+  Attributes:
+   - notNullConstraintCols
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'notNullConstraintCols', (TType.STRUCT,(SQLNotNullConstraint, SQLNotNullConstraint.thrift_spec)), None, ), # 1
+  )
+
+  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:
+      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.notNullConstraintCols = []
+          (_etype337, _size334) = iprot.readListBegin()
+          for _i338 in xrange(_size334):
+            _elem339 = SQLNotNullConstraint()
+            _elem339.read(iprot)
+            self.notNullConstraintCols.append(_elem339)
+          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('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.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    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.notNullConstraintCols)
+    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 PartitionsByExprResult:
   """
   Attributes:
@@ -6597,11 +7364,11 @@ class PartitionsByExprResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype316, _size313) = iprot.readListBegin()
-          for _i317 in xrange(_size313):
-            _elem318 = Partition()
-            _elem318.read(iprot)
-            self.partitions.append(_elem318)
+          (_etype344, _size341) = iprot.readListBegin()
+          for _i345 in xrange(_size341):
+            _elem346 = Partition()
+            _elem346.read(iprot)
+            self.partitions.append(_elem346)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6623,8 +7390,8 @@ class PartitionsByExprResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter319 in self.partitions:
-        iter319.write(oprot)
+      for iter347 in self.partitions:
+        iter347.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.hasUnknownPartitions is not None:
@@ -6808,11 +7575,11 @@ class TableStatsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tableStats = []
-          (_etype323, _size320) = iprot.readListBegin()
-          for _i324 in xrange(_size320):
-            _elem325 = ColumnStatisticsObj()
-            _elem325.read(iprot)
-            self.tableStats.append(_elem325)
+          (_etype351, _size348) = iprot.readListBegin()
+          for _i352 in xrange(_size348):
+            _elem353 = ColumnStatisticsObj()
+            _elem353.read(iprot)
+            self.tableStats.append(_elem353)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6829,8 +7596,8 @@ class TableStatsResult:
     if self.tableStats is not None:
       oprot.writeFieldBegin('tableStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tableStats))
-      for iter326 in self.tableStats:
-        iter326.write(oprot)
+      for iter354 in self.tableStats:
+        iter354.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6884,17 +7651,17 @@ class PartitionsStatsResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.partStats = {}
-          (_ktype328, _vtype329, _size327 ) = iprot.readMapBegin()
-          for _i331 in xrange(_size327):
-            _key332 = iprot.readString()
-            _val333 = []
-            (_etype337, _size334) = iprot.readListBegin()
-            for _i338 in xrange(_size334):
-              _elem339 = ColumnStatisticsObj()
-              _elem339.read(iprot)
-              _val333.append(_elem339)
+          (_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)
             iprot.readListEnd()
-            self.partStats[_key332] = _val333
+            self.partStats[_key360] = _val361
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6911,11 +7678,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 kiter340,viter341 in self.partStats.items():
-        oprot.writeString(kiter340)
-        oprot.writeListBegin(TType.STRUCT, len(viter341))
-        for iter342 in viter341:
-          iter342.write(oprot)
+      for kiter368,viter369 in self.partStats.items():
+        oprot.writeString(kiter368)
+        oprot.writeListBegin(TType.STRUCT, len(viter369))
+        for iter370 in viter369:
+          iter370.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -6986,10 +7753,10 @@ class TableStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype346, _size343) = iprot.readListBegin()
-          for _i347 in xrange(_size343):
-            _elem348 = iprot.readString()
-            self.colNames.append(_elem348)
+          (_etype374, _size371) = iprot.readListBegin()
+          for _i375 in xrange(_size371):
+            _elem376 = iprot.readString()
+            self.colNames.append(_elem376)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7014,8 +7781,8 @@ class TableStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter349 in self.colNames:
-        oprot.writeString(iter349)
+      for iter377 in self.colNames:
+        oprot.writeString(iter377)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7094,20 +7861,20 @@ class PartitionsStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype353, _size350) = iprot.readListBegin()
-          for _i354 in xrange(_size350):
-            _elem355 = iprot.readString()
-            self.colNames.append(_elem355)
+          (_etype381, _size378) = iprot.readListBegin()
+          for _i382 in xrange(_size378):
+            _elem383 = iprot.readString()
+            self.colNames.append(_elem383)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype359, _size356) = iprot.readListBegin()
-          for _i360 in xrange(_size356):
-            _elem361 = iprot.readString()
-            self.partNames.append(_elem361)
+          (_etype387, _size384) = iprot.readListBegin()
+          for _i388 in xrange(_size384):
+            _elem389 = iprot.readString()
+            self.partNames.append(_elem389)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7132,15 +7899,15 @@ class PartitionsStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter362 in self.colNames:
-        oprot.writeString(iter362)
+      for iter390 in self.colNames:
+        oprot.writeString(iter390)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter363 in self.partNames:
-        oprot.writeString(iter363)
+      for iter391 in self.partNames:
+        oprot.writeString(iter391)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7203,11 +7970,11 @@ class AddPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype367, _size364) = iprot.readListBegin()
-          for _i368 in xrange(_size364):
-            _elem369 = Partition()
-            _elem369.read(iprot)
-            self.partitions.append(_elem369)
+          (_etype395, _size392) = iprot.readListBegin()
+          for _i396 in xrange(_size392):
+            _elem397 = Partition()
+            _elem397.read(iprot)
+            self.partitions.append(_elem397)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7224,8 +7991,8 @@ class AddPartitionsResu

<TRUNCATED>

[16/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 2bb4755..709eea9 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -160,12 +160,14 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @param \metastore\Table $tbl
    * @param \metastore\SQLPrimaryKey[] $primaryKeys
    * @param \metastore\SQLForeignKey[] $foreignKeys
+   * @param \metastore\SQLUniqueConstraint[] $uniqueConstraints
+   * @param \metastore\SQLNotNullConstraint[] $notNullConstraints
    * @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);
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints);
   /**
    * @param \metastore\DropConstraintRequest $req
    * @throws \metastore\NoSuchObjectException
@@ -185,6 +187,18 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function add_foreign_key(\metastore\AddForeignKeyRequest $req);
   /**
+   * @param \metastore\AddUniqueConstraintRequest $req
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function add_unique_constraint(\metastore\AddUniqueConstraintRequest $req);
+  /**
+   * @param \metastore\AddNotNullConstraintRequest $req
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function add_not_null_constraint(\metastore\AddNotNullConstraintRequest $req);
+  /**
    * @param string $dbname
    * @param string $name
    * @param bool $deleteData
@@ -768,6 +782,20 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_foreign_keys(\metastore\ForeignKeysRequest $request);
   /**
+   * @param \metastore\UniqueConstraintsRequest $request
+   * @return \metastore\UniqueConstraintsResponse
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_unique_constraints(\metastore\UniqueConstraintsRequest $request);
+  /**
+   * @param \metastore\NotNullConstraintsRequest $request
+   * @return \metastore\NotNullConstraintsResponse
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_not_null_constraints(\metastore\NotNullConstraintsRequest $request);
+  /**
    * @param \metastore\ColumnStatistics $stats_obj
    * @return bool
    * @throws \metastore\NoSuchObjectException
@@ -2243,18 +2271,20 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
-  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys)
+  public function create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints)
   {
-    $this->send_create_table_with_constraints($tbl, $primaryKeys, $foreignKeys);
+    $this->send_create_table_with_constraints($tbl, $primaryKeys, $foreignKeys, $uniqueConstraints, $notNullConstraints);
     $this->recv_create_table_with_constraints();
   }
 
-  public function send_create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys)
+  public function send_create_table_with_constraints(\metastore\Table $tbl, array $primaryKeys, array $foreignKeys, array $uniqueConstraints, array $notNullConstraints)
   {
     $args = new \metastore\ThriftHiveMetastore_create_table_with_constraints_args();
     $args->tbl = $tbl;
     $args->primaryKeys = $primaryKeys;
     $args->foreignKeys = $foreignKeys;
+    $args->uniqueConstraints = $uniqueConstraints;
+    $args->notNullConstraints = $notNullConstraints;
     $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
     if ($bin_accel)
     {
@@ -2467,6 +2497,114 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function add_unique_constraint(\metastore\AddUniqueConstraintRequest $req)
+  {
+    $this->send_add_unique_constraint($req);
+    $this->recv_add_unique_constraint();
+  }
+
+  public function send_add_unique_constraint(\metastore\AddUniqueConstraintRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_unique_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_unique_constraint', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_unique_constraint', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_unique_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_unique_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_unique_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 add_not_null_constraint(\metastore\AddNotNullConstraintRequest $req)
+  {
+    $this->send_add_not_null_constraint($req);
+    $this->recv_add_not_null_constraint();
+  }
+
+  public function send_add_not_null_constraint(\metastore\AddNotNullConstraintRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_not_null_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_not_null_constraint', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_not_null_constraint', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_not_null_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_not_null_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_not_null_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);
@@ -6296,6 +6434,120 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_foreign_keys failed: unknown result");
   }
 
+  public function get_unique_constraints(\metastore\UniqueConstraintsRequest $request)
+  {
+    $this->send_get_unique_constraints($request);
+    return $this->recv_get_unique_constraints();
+  }
+
+  public function send_get_unique_constraints(\metastore\UniqueConstraintsRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_unique_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_unique_constraints', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_unique_constraints', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_unique_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_unique_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_unique_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_unique_constraints failed: unknown result");
+  }
+
+  public function get_not_null_constraints(\metastore\NotNullConstraintsRequest $request)
+  {
+    $this->send_get_not_null_constraints($request);
+    return $this->recv_get_not_null_constraints();
+  }
+
+  public function send_get_not_null_constraints(\metastore\NotNullConstraintsRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_not_null_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_not_null_constraints', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_not_null_constraints', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_not_null_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_not_null_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_not_null_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_not_null_constraints failed: unknown result");
+  }
+
   public function update_table_column_statistics(\metastore\ColumnStatistics $stats_obj)
   {
     $this->send_update_table_column_statistics($stats_obj);
@@ -11150,14 +11402,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size596 = 0;
-            $_etype599 = 0;
-            $xfer += $input->readListBegin($_etype599, $_size596);
-            for ($_i600 = 0; $_i600 < $_size596; ++$_i600)
+            $_size624 = 0;
+            $_etype627 = 0;
+            $xfer += $input->readListBegin($_etype627, $_size624);
+            for ($_i628 = 0; $_i628 < $_size624; ++$_i628)
             {
-              $elem601 = null;
-              $xfer += $input->readString($elem601);
-              $this->success []= $elem601;
+              $elem629 = null;
+              $xfer += $input->readString($elem629);
+              $this->success []= $elem629;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11193,9 +11445,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter602)
+          foreach ($this->success as $iter630)
           {
-            $xfer += $output->writeString($iter602);
+            $xfer += $output->writeString($iter630);
           }
         }
         $output->writeListEnd();
@@ -11326,14 +11578,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size603 = 0;
-            $_etype606 = 0;
-            $xfer += $input->readListBegin($_etype606, $_size603);
-            for ($_i607 = 0; $_i607 < $_size603; ++$_i607)
+            $_size631 = 0;
+            $_etype634 = 0;
+            $xfer += $input->readListBegin($_etype634, $_size631);
+            for ($_i635 = 0; $_i635 < $_size631; ++$_i635)
             {
-              $elem608 = null;
-              $xfer += $input->readString($elem608);
-              $this->success []= $elem608;
+              $elem636 = null;
+              $xfer += $input->readString($elem636);
+              $this->success []= $elem636;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11369,9 +11621,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter609)
+          foreach ($this->success as $iter637)
           {
-            $xfer += $output->writeString($iter609);
+            $xfer += $output->writeString($iter637);
           }
         }
         $output->writeListEnd();
@@ -12372,18 +12624,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size610 = 0;
-            $_ktype611 = 0;
-            $_vtype612 = 0;
-            $xfer += $input->readMapBegin($_ktype611, $_vtype612, $_size610);
-            for ($_i614 = 0; $_i614 < $_size610; ++$_i614)
+            $_size638 = 0;
+            $_ktype639 = 0;
+            $_vtype640 = 0;
+            $xfer += $input->readMapBegin($_ktype639, $_vtype640, $_size638);
+            for ($_i642 = 0; $_i642 < $_size638; ++$_i642)
             {
-              $key615 = '';
-              $val616 = new \metastore\Type();
-              $xfer += $input->readString($key615);
-              $val616 = new \metastore\Type();
-              $xfer += $val616->read($input);
-              $this->success[$key615] = $val616;
+              $key643 = '';
+              $val644 = new \metastore\Type();
+              $xfer += $input->readString($key643);
+              $val644 = new \metastore\Type();
+              $xfer += $val644->read($input);
+              $this->success[$key643] = $val644;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -12419,10 +12671,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter617 => $viter618)
+          foreach ($this->success as $kiter645 => $viter646)
           {
-            $xfer += $output->writeString($kiter617);
-            $xfer += $viter618->write($output);
+            $xfer += $output->writeString($kiter645);
+            $xfer += $viter646->write($output);
           }
         }
         $output->writeMapEnd();
@@ -12626,15 +12878,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size619 = 0;
-            $_etype622 = 0;
-            $xfer += $input->readListBegin($_etype622, $_size619);
-            for ($_i623 = 0; $_i623 < $_size619; ++$_i623)
+            $_size647 = 0;
+            $_etype650 = 0;
+            $xfer += $input->readListBegin($_etype650, $_size647);
+            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
             {
-              $elem624 = null;
-              $elem624 = new \metastore\FieldSchema();
-              $xfer += $elem624->read($input);
-              $this->success []= $elem624;
+              $elem652 = null;
+              $elem652 = new \metastore\FieldSchema();
+              $xfer += $elem652->read($input);
+              $this->success []= $elem652;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12686,9 +12938,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter625)
+          foreach ($this->success as $iter653)
           {
-            $xfer += $iter625->write($output);
+            $xfer += $iter653->write($output);
           }
         }
         $output->writeListEnd();
@@ -12930,15 +13182,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size626 = 0;
-            $_etype629 = 0;
-            $xfer += $input->readListBegin($_etype629, $_size626);
-            for ($_i630 = 0; $_i630 < $_size626; ++$_i630)
+            $_size654 = 0;
+            $_etype657 = 0;
+            $xfer += $input->readListBegin($_etype657, $_size654);
+            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
             {
-              $elem631 = null;
-              $elem631 = new \metastore\FieldSchema();
-              $xfer += $elem631->read($input);
-              $this->success []= $elem631;
+              $elem659 = null;
+              $elem659 = new \metastore\FieldSchema();
+              $xfer += $elem659->read($input);
+              $this->success []= $elem659;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12990,9 +13242,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter632)
+          foreach ($this->success as $iter660)
           {
-            $xfer += $iter632->write($output);
+            $xfer += $iter660->write($output);
           }
         }
         $output->writeListEnd();
@@ -13206,15 +13458,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size633 = 0;
-            $_etype636 = 0;
-            $xfer += $input->readListBegin($_etype636, $_size633);
-            for ($_i637 = 0; $_i637 < $_size633; ++$_i637)
+            $_size661 = 0;
+            $_etype664 = 0;
+            $xfer += $input->readListBegin($_etype664, $_size661);
+            for ($_i665 = 0; $_i665 < $_size661; ++$_i665)
             {
-              $elem638 = null;
-              $elem638 = new \metastore\FieldSchema();
-              $xfer += $elem638->read($input);
-              $this->success []= $elem638;
+              $elem666 = null;
+              $elem666 = new \metastore\FieldSchema();
+              $xfer += $elem666->read($input);
+              $this->success []= $elem666;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13266,9 +13518,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter639)
+          foreach ($this->success as $iter667)
           {
-            $xfer += $iter639->write($output);
+            $xfer += $iter667->write($output);
           }
         }
         $output->writeListEnd();
@@ -13510,15 +13762,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size640 = 0;
-            $_etype643 = 0;
-            $xfer += $input->readListBegin($_etype643, $_size640);
-            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
+            $_size668 = 0;
+            $_etype671 = 0;
+            $xfer += $input->readListBegin($_etype671, $_size668);
+            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
             {
-              $elem645 = null;
-              $elem645 = new \metastore\FieldSchema();
-              $xfer += $elem645->read($input);
-              $this->success []= $elem645;
+              $elem673 = null;
+              $elem673 = new \metastore\FieldSchema();
+              $xfer += $elem673->read($input);
+              $this->success []= $elem673;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13570,9 +13822,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter646)
+          foreach ($this->success as $iter674)
           {
-            $xfer += $iter646->write($output);
+            $xfer += $iter674->write($output);
           }
         }
         $output->writeListEnd();
@@ -14108,6 +14360,14 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
    * @var \metastore\SQLForeignKey[]
    */
   public $foreignKeys = null;
+  /**
+   * @var \metastore\SQLUniqueConstraint[]
+   */
+  public $uniqueConstraints = null;
+  /**
+   * @var \metastore\SQLNotNullConstraint[]
+   */
+  public $notNullConstraints = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -14135,6 +14395,24 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
             'class' => '\metastore\SQLForeignKey',
             ),
           ),
+        4 => array(
+          'var' => 'uniqueConstraints',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLUniqueConstraint',
+            ),
+          ),
+        5 => array(
+          'var' => 'notNullConstraints',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SQLNotNullConstraint',
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -14147,6 +14425,12 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       if (isset($vals['foreignKeys'])) {
         $this->foreignKeys = $vals['foreignKeys'];
       }
+      if (isset($vals['uniqueConstraints'])) {
+        $this->uniqueConstraints = $vals['uniqueConstraints'];
+      }
+      if (isset($vals['notNullConstraints'])) {
+        $this->notNullConstraints = $vals['notNullConstraints'];
+      }
     }
   }
 
@@ -14180,15 +14464,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size647 = 0;
-            $_etype650 = 0;
-            $xfer += $input->readListBegin($_etype650, $_size647);
-            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
+            $_size675 = 0;
+            $_etype678 = 0;
+            $xfer += $input->readListBegin($_etype678, $_size675);
+            for ($_i679 = 0; $_i679 < $_size675; ++$_i679)
             {
-              $elem652 = null;
-              $elem652 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem652->read($input);
-              $this->primaryKeys []= $elem652;
+              $elem680 = null;
+              $elem680 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem680->read($input);
+              $this->primaryKeys []= $elem680;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14198,15 +14482,51 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size681 = 0;
+            $_etype684 = 0;
+            $xfer += $input->readListBegin($_etype684, $_size681);
+            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
+            {
+              $elem686 = null;
+              $elem686 = new \metastore\SQLForeignKey();
+              $xfer += $elem686->read($input);
+              $this->foreignKeys []= $elem686;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::LST) {
+            $this->uniqueConstraints = array();
+            $_size687 = 0;
+            $_etype690 = 0;
+            $xfer += $input->readListBegin($_etype690, $_size687);
+            for ($_i691 = 0; $_i691 < $_size687; ++$_i691)
+            {
+              $elem692 = null;
+              $elem692 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem692->read($input);
+              $this->uniqueConstraints []= $elem692;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::LST) {
+            $this->notNullConstraints = array();
+            $_size693 = 0;
+            $_etype696 = 0;
+            $xfer += $input->readListBegin($_etype696, $_size693);
+            for ($_i697 = 0; $_i697 < $_size693; ++$_i697)
             {
-              $elem658 = null;
-              $elem658 = new \metastore\SQLForeignKey();
-              $xfer += $elem658->read($input);
-              $this->foreignKeys []= $elem658;
+              $elem698 = null;
+              $elem698 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem698->read($input);
+              $this->notNullConstraints []= $elem698;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14242,9 +14562,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter659)
+          foreach ($this->primaryKeys as $iter699)
           {
-            $xfer += $iter659->write($output);
+            $xfer += $iter699->write($output);
           }
         }
         $output->writeListEnd();
@@ -14259,9 +14579,43 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter660)
+          foreach ($this->foreignKeys as $iter700)
           {
-            $xfer += $iter660->write($output);
+            $xfer += $iter700->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->uniqueConstraints !== null) {
+      if (!is_array($this->uniqueConstraints)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('uniqueConstraints', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
+        {
+          foreach ($this->uniqueConstraints as $iter701)
+          {
+            $xfer += $iter701->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->notNullConstraints !== null) {
+      if (!is_array($this->notNullConstraints)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('notNullConstraints', TType::LST, 5);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
+        {
+          foreach ($this->notNullConstraints as $iter702)
+          {
+            $xfer += $iter702->write($output);
           }
         }
         $output->writeListEnd();
@@ -14973,54 +15327,125 @@ class ThriftHiveMetastore_add_foreign_key_result {
 
 }
 
-class ThriftHiveMetastore_drop_table_args {
+class ThriftHiveMetastore_add_unique_constraint_args {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var \metastore\AddUniqueConstraintRequest
    */
-  public $dbname = null;
+  public $req = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AddUniqueConstraintRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_add_unique_constraint_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::STRUCT) {
+            $this->req = new \metastore\AddUniqueConstraintRequest();
+            $xfer += $this->req->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_unique_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();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_add_unique_constraint_result {
+  static $_TSPEC;
+
   /**
-   * @var string
+   * @var \metastore\NoSuchObjectException
    */
-  public $name = null;
+  public $o1 = null;
   /**
-   * @var bool
+   * @var \metastore\MetaException
    */
-  public $deleteData = null;
+  public $o2 = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
-          'type' => TType::STRING,
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
           ),
         2 => array(
-          'var' => 'name',
-          'type' => TType::STRING,
-          ),
-        3 => array(
-          'var' => 'deleteData',
-          'type' => TType::BOOL,
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['dbname'])) {
-        $this->dbname = $vals['dbname'];
-      }
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
       }
-      if (isset($vals['deleteData'])) {
-        $this->deleteData = $vals['deleteData'];
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_args';
+    return 'ThriftHiveMetastore_add_unique_constraint_result';
   }
 
   public function read($input)
@@ -15039,22 +15464,17 @@ class ThriftHiveMetastore_drop_table_args {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
+          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::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->o2 = new \metastore\MetaException();
+            $xfer += $this->o2->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -15071,20 +15491,95 @@ 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->writeStructBegin('ThriftHiveMetastore_add_unique_constraint_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
-      $xfer += $output->writeString($this->name);
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->deleteData !== null) {
-      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->deleteData);
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_add_not_null_constraint_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\AddNotNullConstraintRequest
+   */
+  public $req = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AddNotNullConstraintRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_add_not_null_constraint_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::STRUCT) {
+            $this->req = new \metastore\AddNotNullConstraintRequest();
+            $xfer += $this->req->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_not_null_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();
@@ -15094,7 +15589,7 @@ class ThriftHiveMetastore_drop_table_args {
 
 }
 
-class ThriftHiveMetastore_drop_table_result {
+class ThriftHiveMetastore_add_not_null_constraint_result {
   static $_TSPEC;
 
   /**
@@ -15104,7 +15599,7 @@ class ThriftHiveMetastore_drop_table_result {
   /**
    * @var \metastore\MetaException
    */
-  public $o3 = null;
+  public $o2 = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -15115,7 +15610,7 @@ class ThriftHiveMetastore_drop_table_result {
           'class' => '\metastore\NoSuchObjectException',
           ),
         2 => array(
-          'var' => 'o3',
+          'var' => 'o2',
           'type' => TType::STRUCT,
           'class' => '\metastore\MetaException',
           ),
@@ -15125,14 +15620,14 @@ 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_not_null_constraint_result';
   }
 
   public function read($input)
@@ -15160,8 +15655,8 @@ class ThriftHiveMetastore_drop_table_result {
           break;
         case 2:
           if ($ftype == TType::STRUCT) {
-            $this->o3 = new \metastore\MetaException();
-            $xfer += $this->o3->read($input);
+            $this->o2 = new \metastore\MetaException();
+            $xfer += $this->o2->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -15178,15 +15673,15 @@ class ThriftHiveMetastore_drop_table_result {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result');
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_not_null_constraint_result');
     if ($this->o1 !== null) {
       $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
       $xfer += $this->o1->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->o3 !== null) {
-      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2);
-      $xfer += $this->o3->write($output);
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -15196,7 +15691,7 @@ class ThriftHiveMetastore_drop_table_result {
 
 }
 
-class ThriftHiveMetastore_drop_table_with_environment_context_args {
+class ThriftHiveMetastore_drop_table_args {
   static $_TSPEC;
 
   /**
@@ -15211,10 +15706,6 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
    * @var bool
    */
   public $deleteData = null;
-  /**
-   * @var \metastore\EnvironmentContext
-   */
-  public $environment_context = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -15231,11 +15722,6 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
           'var' => 'deleteData',
           'type' => TType::BOOL,
           ),
-        4 => array(
-          'var' => 'environment_context',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\EnvironmentContext',
-          ),
         );
     }
     if (is_array($vals)) {
@@ -15248,14 +15734,246 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args {
       if (isset($vals['deleteData'])) {
         $this->deleteData = $vals['deleteData'];
       }
-      if (isset($vals['environment_context'])) {
-        $this->environment_context = $vals['environment_context'];
-      }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_table_with_environment_context_args';
+    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)
+  {
+    $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->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->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_drop_table_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_table_with_environment_context_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var bool
+   */
+  public $deleteData = null;
+  /**
+   * @var \metastore\EnvironmentContext
+   */
+  public $environment_context = 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,
+          ),
+        4 => array(
+          'var' => 'environment_context',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\EnvironmentContext',
+          ),
+        );
+    }
+    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['environment_context'])) {
+        $this->environment_context = $vals['environment_context'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_table_with_environment_context_args';
   }
 
   public function read($input)
@@ -15533,14 +16251,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size661 = 0;
-            $_etype664 = 0;
-            $xfer += $input->readListBegin($_etype664, $_size661);
-            for ($_i665 = 0; $_i665 < $_size661; ++$_i665)
+            $_size703 = 0;
+            $_etype706 = 0;
+            $xfer += $input->readListBegin($_etype706, $_size703);
+            for ($_i707 = 0; $_i707 < $_size703; ++$_i707)
             {
-              $elem666 = null;
-              $xfer += $input->readString($elem666);
-              $this->partNames []= $elem666;
+              $elem708 = null;
+              $xfer += $input->readString($elem708);
+              $this->partNames []= $elem708;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15578,9 +16296,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter667)
+          foreach ($this->partNames as $iter709)
           {
-            $xfer += $output->writeString($iter667);
+            $xfer += $output->writeString($iter709);
           }
         }
         $output->writeListEnd();
@@ -15831,14 +16549,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size668 = 0;
-            $_etype671 = 0;
-            $xfer += $input->readListBegin($_etype671, $_size668);
-            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
+            $_size710 = 0;
+            $_etype713 = 0;
+            $xfer += $input->readListBegin($_etype713, $_size710);
+            for ($_i714 = 0; $_i714 < $_size710; ++$_i714)
             {
-              $elem673 = null;
-              $xfer += $input->readString($elem673);
-              $this->success []= $elem673;
+              $elem715 = null;
+              $xfer += $input->readString($elem715);
+              $this->success []= $elem715;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15874,9 +16592,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter674)
+          foreach ($this->success as $iter716)
           {
-            $xfer += $output->writeString($iter674);
+            $xfer += $output->writeString($iter716);
           }
         }
         $output->writeListEnd();
@@ -16078,14 +16796,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size675 = 0;
-            $_etype678 = 0;
-            $xfer += $input->readListBegin($_etype678, $_size675);
-            for ($_i679 = 0; $_i679 < $_size675; ++$_i679)
+            $_size717 = 0;
+            $_etype720 = 0;
+            $xfer += $input->readListBegin($_etype720, $_size717);
+            for ($_i721 = 0; $_i721 < $_size717; ++$_i721)
             {
-              $elem680 = null;
-              $xfer += $input->readString($elem680);
-              $this->success []= $elem680;
+              $elem722 = null;
+              $xfer += $input->readString($elem722);
+              $this->success []= $elem722;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16121,9 +16839,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter681)
+          foreach ($this->success as $iter723)
           {
-            $xfer += $output->writeString($iter681);
+            $xfer += $output->writeString($iter723);
           }
         }
         $output->writeListEnd();
@@ -16228,14 +16946,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size682 = 0;
-            $_etype685 = 0;
-            $xfer += $input->readListBegin($_etype685, $_size682);
-            for ($_i686 = 0; $_i686 < $_size682; ++$_i686)
+            $_size724 = 0;
+            $_etype727 = 0;
+            $xfer += $input->readListBegin($_etype727, $_size724);
+            for ($_i728 = 0; $_i728 < $_size724; ++$_i728)
             {
-              $elem687 = null;
-              $xfer += $input->readString($elem687);
-              $this->tbl_types []= $elem687;
+              $elem729 = null;
+              $xfer += $input->readString($elem729);
+              $this->tbl_types []= $elem729;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16273,9 +16991,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter688)
+          foreach ($this->tbl_types as $iter730)
           {
-            $xfer += $output->writeString($iter688);
+            $xfer += $output->writeString($iter730);
           }
         }
         $output->writeListEnd();
@@ -16352,15 +17070,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size689 = 0;
-            $_etype692 = 0;
-            $xfer += $input->readListBegin($_etype692, $_size689);
-            for ($_i693 = 0; $_i693 < $_size689; ++$_i693)
+            $_size731 = 0;
+            $_etype734 = 0;
+            $xfer += $input->readListBegin($_etype734, $_size731);
+            for ($_i735 = 0; $_i735 < $_size731; ++$_i735)
             {
-              $elem694 = null;
-              $elem694 = new \metastore\TableMeta();
-              $xfer += $elem694->read($input);
-              $this->success []= $elem694;
+              $elem736 = null;
+              $elem736 = new \metastore\TableMeta();
+              $xfer += $elem736->read($input);
+              $this->success []= $elem736;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16396,9 +17114,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter695)
+          foreach ($this->success as $iter737)
           {
-            $xfer += $iter695->write($output);
+            $xfer += $iter737->write($output);
           }
         }
         $output->writeListEnd();
@@ -16554,14 +17272,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size696 = 0;
-            $_etype699 = 0;
-            $xfer += $input->readListBegin($_etype699, $_size696);
-            for ($_i700 = 0; $_i700 < $_size696; ++$_i700)
+            $_size738 = 0;
+            $_etype741 = 0;
+            $xfer += $input->readListBegin($_etype741, $_size738);
+            for ($_i742 = 0; $_i742 < $_size738; ++$_i742)
             {
-              $elem701 = null;
-              $xfer += $input->readString($elem701);
-              $this->success []= $elem701;
+              $elem743 = null;
+              $xfer += $input->readString($elem743);
+              $this->success []= $elem743;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16597,9 +17315,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter702)
+          foreach ($this->success as $iter744)
           {
-            $xfer += $output->writeString($iter702);
+            $xfer += $output->writeString($iter744);
           }
         }
         $output->writeListEnd();
@@ -16914,14 +17632,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size703 = 0;
-            $_etype706 = 0;
-            $xfer += $input->readListBegin($_etype706, $_size703);
-            for ($_i707 = 0; $_i707 < $_size703; ++$_i707)
+            $_size745 = 0;
+            $_etype748 = 0;
+            $xfer += $input->readListBegin($_etype748, $_size745);
+            for ($_i749 = 0; $_i749 < $_size745; ++$_i749)
             {
-              $elem708 = null;
-              $xfer += $input->readString($elem708);
-              $this->tbl_names []= $elem708;
+              $elem750 = null;
+              $xfer += $input->readString($elem750);
+              $this->tbl_names []= $elem750;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16954,9 +17672,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter709)
+          foreach ($this->tbl_names as $iter751)
           {
-            $xfer += $output->writeString($iter709);
+            $xfer += $output->writeString($iter751);
           }
         }
         $output->writeListEnd();
@@ -17021,15 +17739,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size710 = 0;
-            $_etype713 = 0;
-            $xfer += $input->readListBegin($_etype713, $_size710);
-            for ($_i714 = 0; $_i714 < $_size710; ++$_i714)
+            $_size752 = 0;
+            $_etype755 = 0;
+            $xfer += $input->readListBegin($_etype755, $_size752);
+            for ($_i756 = 0; $_i756 < $_size752; ++$_i756)
             {
-              $elem715 = null;
-              $elem715 = new \metastore\Table();
-              $xfer += $elem715->read($input);
-              $this->success []= $elem715;
+              $elem757 = null;
+              $elem757 = new \metastore\Table();
+              $xfer += $elem757->read($input);
+              $this->success []= $elem757;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17057,9 +17775,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter716)
+          foreach ($this->success as $iter758)
           {
-            $xfer += $iter716->write($output);
+            $xfer += $iter758->write($output);
           }
         }
         $output->writeListEnd();
@@ -17725,14 +18443,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size717 = 0;
-            $_etype720 = 0;
-            $xfer += $input->readListBegin($_etype720, $_size717);
-            for ($_i721 = 0; $_i721 < $_size717; ++$_i721)
+            $_size759 = 0;
+            $_etype762 = 0;
+            $xfer += $input->readListBegin($_etype762, $_size759);
+            for ($_i763 = 0; $_i763 < $_size759; ++$_i763)
             {
-              $elem722 = null;
-              $xfer += $input->readString($elem722);
-              $this->success []= $elem722;
+              $elem764 = null;
+              $xfer += $input->readString($elem764);
+              $this->success []= $elem764;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17784,9 +18502,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter723)
+          foreach ($this->success as $iter765)
           {
-            $xfer += $output->writeString($iter723);
+            $xfer += $output->writeString($iter765);
           }
         }
         $output->writeListEnd();
@@ -19099,15 +19817,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size724 = 0;
-            $_etype727 = 0;
-            $xfer += $input->readListBegin($_etype727, $_size724);
-            for ($_i728 = 0; $_i728 < $_size724; ++$_i728)
+            $_size766 = 0;
+            $_etype769 = 0;
+            $xfer += $input->readListBegin($_etype769, $_size766);
+            for ($_i770 = 0; $_i770 < $_size766; ++$_i770)
             {
-              $elem729 = null;
-              $elem729 = new \metastore\Partition();
-              $xfer += $elem729->read($input);
-              $this->new_parts []= $elem729;
+              $elem771 = null;
+              $elem771 = new \metastore\Partition();
+              $xfer += $elem771->read($input);
+              $this->new_parts []= $elem771;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19135,9 +19853,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter730)
+          foreach ($this->new_parts as $iter772)
           {
-            $xfer += $iter730->write($output);
+            $xfer += $iter772->write($output);
           }
         }
         $output->writeListEnd();
@@ -19352,15 +20070,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size731 = 0;
-            $_etype734 = 0;
-            $xfer += $input->readListBegin($_etype734, $_size731);
-            for ($_i735 = 0; $_i735 < $_size731; ++$_i735)
+            $_size773 = 0;
+            $_etype776 = 0;
+            $xfer += $input->readListBegin($_etype776, $_size773);
+            for ($_i777 = 0; $_i777 < $_size773; ++$_i777)
             {
-              $elem736 = null;
-              $elem736 = new \metastore\PartitionSpec();
-              $xfer += $elem736->read($input);
-              $this->new_parts []= $elem736;
+              $elem778 = null;
+              $elem778 = new \metastore\PartitionSpec();
+              $xfer += $elem778->read($input);
+              $this->new_parts []= $elem778;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19388,9 +20106,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter737)
+          foreach ($this->new_parts as $iter779)
           {
-            $xfer += $iter737->write($output);
+            $xfer += $iter779->write($output);
           }
         }
         $output->writeListEnd();
@@ -19640,14 +20358,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size738 = 0;
-            $_etype741 = 0;
-            $xfer += $input->readListBegin($_etype741, $_size738);
-            for ($_i742 = 0; $_i742 < $_size738; ++$_i742)
+            $_size780 = 0;
+            $_etype783 = 0;
+            $xfer += $input->readListBegin($_etype783, $_size780);
+            for ($_i784 = 0; $_i784 < $_size780; ++$_i784)
             {
-              $elem743 = null;
-              $xfer += $input->readString($elem743);
-              $this->part_vals []= $elem743;
+              $elem785 = null;
+              $xfer += $input->readString($elem785);
+              $this->part_vals []= $elem785;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19685,9 +20403,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter744)
+          foreach ($this->part_vals as $iter786)
           {
-            $xfer += $output->writeString($iter744);
+            $xfer += $output->writeString($iter786);
           }
         }
         $output->writeListEnd();
@@ -20189,14 +20907,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size745 = 0;
-            $_etype748 = 0;
-            $xfer += $input->readListBegin($_etype748, $_size745);
-            for ($_i749 = 0; $_i749 < $_size745; ++$_i749)
+            $_size787 = 0;
+            $_etype790 = 0;
+            $xfer += $input->readListBegin($_etype790, $_size787);
+            for ($_i791 = 0; $_i791 < $_size787; ++$_i791)
             {
-              $elem750 = null;
-              $xfer += $input->readString($elem750);
-              $this->part_vals []= $elem750;
+              $elem792 = null;
+              $xfer += $input->readString($elem792);
+              $this->part_vals []= $elem792;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20242,9 +20960,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter751)
+          foreach ($this->part_vals as $iter793)
           {
-            $xfer += $output->writeString($iter751);
+            $xfer += $output->writeString($iter793);
           }
         }
         $output->writeListEnd();
@@ -21098,14 +21816,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size752 = 0;
-            $_etype755 = 0;
-            $xfer += $input->readListBegin($_etype755, $_size752);
-            for ($_i756 = 0; $_i756 < $_size752; ++$_i756)
+            $_size794 = 0;
+            $_etype797 = 0;
+            $xfer += $input->readListBegin($_etype797, $_size794);
+            for ($_i798 = 0; $_i798 < $_size794; ++$_i798)
             {
-              $elem757 = null;
-              $xfer += $input->readString($elem757);
-              $this->part_vals []= $elem757;
+              $elem799 = null;
+              $xfer += $input->readString($elem799);
+              $this->part_vals []= $elem799;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21150,9 +21868,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter758)
+          foreach ($this->part_vals as $iter800)
           {
-            $xfer += $output->writeString($iter758);
+            $xfer += $output->writeString($iter800);
           }
         }
         $output->writeListEnd();
@@ -21405,14 +22123,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size759 = 0;
-            $_etype762 = 0;
-            $xfer += $input->readListBegin($_etype762, $_size759);
-            for ($_i763 = 0; $_i763 < $_size759; ++$_i763)
+            $_size801 = 0;
+            $_etype804 = 0;
+            $xfer += $input->readListBegin($_etype804, $_size801);
+            for ($_i805 = 0; $_i805 < $_size801; ++$_i805)
             {
-              $elem764 = null;
-              $xfer += $input->readString($elem764);
-              $this->part_vals []= $elem764;
+              $elem806 = null;
+              $xfer += $input->readString($elem806);
+              $this->part_vals []= $elem806;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21465,9 +22183,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter765)
+          foreach ($this->part_vals as $iter807)
           {
-            $xfer += $output->writeString($iter765);
+            $xfer += $output->writeString($iter807);
           }
         }
         $output->writeListEnd();
@@ -22481,14 +23199,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size766 = 0;
-            $_etype769 = 0;
-            $xfer += $input->readListBegin($_etype769, $_size766);
-            for ($_i770 = 0; $_i770 < $_size766; ++$_i770)
+            $_size808 = 0;
+            $_etype811 = 0;
+            $xfer += $input->readListBegin($_etype811, $_size808);
+            for ($_i812 = 0; $_i812 < $_size808; ++$_i812)
             {
-              $elem771 = null;
-              $xfer += $input->readString($elem771);
-              $this->part_vals []= $elem771;
+              $elem813 = null;
+              $xfer += $input->readString($elem813);
+              $this->part_vals []= $elem813;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22526,9 +23244,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter772)
+          foreach ($this->part_vals as $iter814)
           {
-            $xfer += $output->writeString($iter772);
+            $xfer += $output->writeString($iter814);
           }
         }
         $output->writeListEnd();
@@ -22770,17 +23488,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size773 = 0;
-            $_ktype774 = 0;
-            $_vtype775 = 0;
-            $xfer += $input->readMapBegin($_ktype774, $_vtype775, $_size773);
-            for ($_i777 = 0; $_i777 < $_size773; ++$_i777)
+            $_size815 = 0;
+            $_ktype816 = 0;
+            $_vtype817 = 0;
+            $xfer += $input->readMapBegin($_ktype816, $_vtype817, $_size815);
+            for ($_i819 = 0; $_i819 < $_size815; ++$_i819)
             {
-              $key778 = '';
-              $val779 = '';
-              $xfer += $input->readString($key778);
-              $xfer += $input->readString($val779);
-              $this->partitionSpecs[$key778] = $val779;
+              $key820 = '';
+              $val821 = '';
+              $xfer += $input->readString($key820);
+              $xfer += $input->readString($val821);
+              $this->partitionSpecs[$key820] = $val821;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22836,10 +23554,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter780 => $viter781)
+          foreach ($this->partitionSpecs as $kiter822 => $viter823)
           {
-            $xfer += $output->writeString($kiter780);
-            $xfer += $output->writeString($viter781);
+            $xfer += $output->writeString($kiter822);
+            $xfer += $output->writeString($viter823);
           }
         }
         $output->writeMapEnd();
@@ -23151,17 +23869,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size782 = 0;
-            $_ktype783 = 0;
-            $_vtype784 = 0;
-            $xfer += $input->readMapBegin($_ktype783, $_vtype784, $_size782);
-            for ($_i786 = 0; $_i786 < $_size782; ++$_i786)
+            $_size824 = 0;
+            $_ktype825 = 0;
+            $_vtype826 = 0;
+            $xfer += $input->readMapBegin($_ktype825, $_vtype826, $_size824);
+            for ($_i828 = 0; $_i828 < $_size824; ++$_i828)
             {
-              $key787 = '';
-              $val788 = '';
-              $xfer += $input->readString($key787);
-              $xfer += $input->readString($val788);
-              $this->partitionSpecs[$key787] = $val788;
+              $key829 = '';
+              $val830 = '';
+              $xfer += $input->readString($key829);
+              $xfer += $input->readString($val830);
+              $this->partitionSpecs[$key829] = $val830;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -23217,10 +23935,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter789 => $viter790)
+          foreach ($this->partitionSpecs as $kiter831 => $viter832)
           {
-            $xfer += $output->writeString($kiter789);
-            $xfer += $output->writeString($viter790);
+            $xfer += $output->writeString($kiter831);
+            $xfer += $output->writeString($viter832);
           }
         }
         $output->writeMapEnd();
@@ -23353,15 +24071,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size791 = 0;
-            $_etype794 = 0;
-            $xfer += $input->readListBegin($_etype794, $_size791);
-            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem796 = null;
-              $elem796 = new \metastore\Partition();
-              $xfer += $elem796->read($input);
-              $this->success []= $elem796;
+              $elem838 = null;
+              $elem838 = new \metastore\Partition();
+              $xfer += $elem838->read($input);
+              $this->success []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23421,9 +24139,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter797)
+          foreach ($this->success as $iter839)
           {
-            $xfer += $iter797->write($output);
+            $xfer += $iter839->write($output);
           }
         }
         $output->writeListEnd();
@@ -23569,14 +24287,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size798 = 0;
-            $_etype801 = 0;
-            $xfer += $input->readListBegin($_etype801, $_size798);
-            for ($_i802 = 0; $_i802 < $_size798; ++$_i802)
+            $_size840 = 0;
+            $_etype843 = 0;
+            $xfer += $input->readListBegin($_etype843, $_size840);
+            for ($_i844 = 0; $_i844 < $_size840; ++$_i844)
             {
-              $elem803 = null;
-              $xfer += $input->readString($elem803);
-              $this->part_vals []= $elem803;
+              $elem845 = null;
+              $xfer += $input->readString($elem845);
+              $this->part_vals []= $elem845;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23593,14 +24311,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size846 = 0;
+            $_etype849 = 0;
+            $xfer += $input->readListBegin($_etype849, $_size846);
+            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
             {
-              $elem809 = null;
-              $xfer += $input->readString($elem809);
-              $this->group_names []= $elem809;
+              $elem851 = null;
+              $xfer += $input->readString($elem851);
+              $this->group_names []= $elem851;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23638,9 +24356,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter810)
+          foreach ($this->part_vals as $iter852)
           {
-            $xfer += $output->writeString($iter810);
+            $xfer += $output->writeString($iter852);
           }
         }
         $output->writeListEnd();
@@ -23660,9 +24378,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter811)
+          foreach ($this->group_names as $iter853)
           {
-            $xfer += $output->writeString($iter811);
+            $xfer += $output->writeString($iter853);
           }
         }
         $output->writeListEnd();
@@ -24253,15 +24971,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size812 = 0;
-            $_etype815 = 0;
-            $xfer += $input->readListBegin($_etype815, $_size812);
-            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
+            $_size854 = 0;
+            $_etype857 = 0;
+            $xfer += $input->readListBegin($_etype857, $_size854);
+            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
             {
-              $elem817 = null;
-              $elem817 = new \metastore\Partition();
-              $xfer += $elem817->read($input);
-              $this->success []= $elem817;
+              $elem859 = null;
+              $elem859 = new \metastore\Partition();
+              $xfer += $elem859->read($input);
+              $this->success []= $elem859;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24305,9 +25023,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter818)
+          foreach ($this->success as $iter860)
           {
-            $xfer += $iter818->write($output);
+            $xfer += $iter860->write($output);
           }
         }
         $output->writeListEnd();
@@ -24453,14 +25171,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size819 = 0;
-            $_etype822 = 0;
-            $xfer += $input->readListBegin($_etype822, $_size819);
-            for ($_i823 = 0; $_i823 < $_size819; ++$_i823)
+            $_size861 = 0;
+            $_etype864 = 0;
+            $xfer += $input->readListBegin($_etype864, $_size861);
+            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
             {
-              $elem824 = null;
-              $xfer += $input->readString($elem824);
-              $this->group_names []= $elem824;
+              $elem866 = null;
+              $xfer += $input->readString($elem866);
+              $this->group_names []= $elem866;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24508,9 +25226,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter825)
+          foreach ($this->group_names as $iter867)
           {
-            $xfer += $output->writeString($iter825);
+            $xfer += $output->writeString($iter867);
           }
         }
         $output->writeListEnd();
@@ -24599,15 +25317,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size826 = 0;
-            $_etype829 = 0;
-            $xfer += $input->readListBegin($_etype829, $_size826);
-            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
+            $_size868 = 0;
+            $_etype871 = 0;
+            $xfer += $input->readListBegin($_etype871, $_size868);
+            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
             {
-              $elem831 = null;
-              $elem831 = new \metastore\Partition();
-              $xfer += $elem831->read($input);
-              $this->success []= $elem831;
+              $elem873 = null;
+              $elem873 = new \metastore\Partition();
+              $xfer += $elem873->read($input);
+              $this->success []= $elem873;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24651,9 +25369,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter832)
+          foreach ($this->success as $iter874)
           {
-            $xfer += $iter832->write($output);
+            $xfer += $iter874->write($output);
           }
         }
         $output->writeListEnd();
@@ -24873,15 +25591,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size833 = 0;
-            $_etype836 = 0;
-            $xfer += $input->readListBegin($_etype836, $_size833);
-            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
+            $_size875 = 0;
+            $_etype878 = 0;
+            $xfer += $input->readListBegin($_etype878, $_size875);
+            for ($_i879 = 0; $_i879 < $_size875; ++$_i879)
             {
-              $elem838 = null;
-              $elem838 = new \metastore\PartitionSpec();
-              $xfer += $elem838->read($input);
-              $this->success []= $elem838;
+              $elem880 = null;
+              $elem880 = new \metastore\PartitionSpec();
+              $xfer += $elem880->read($input);
+              $this->success []= $elem880;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24925,9 +25643,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter839)
+          foreach ($this->success as $iter881)
           {
-            $xfer += $iter839->write($output);
+            $xfer += $iter881->write($output);
           }
         }
         $output->writeListEnd();
@@ -25134,14 +25852,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size840 = 0;
-            $_etype843 = 0;
-            $xfer += $input->readListBegin($_etype843, $_size840);
-            for ($_i844 = 0; $_i844 < $_size840; ++$_i844)
+            $_size882 = 0;
+            $_etype885 = 0;
+            $xfer += $input->readListBegin($_etype885, $_size882);
+            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
             {
-              $elem845 = null;
-              $xfer += $input->readString($elem845);
-              $this->success []= $elem845;
+              $elem887 = null;
+              $xfer += $input->readString($elem887);
+              $this->success []= $elem887;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25177,9 +25895,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter846)
+          foreach ($this->success as $iter888)
           {
-            $xfer += $output->writeString($iter846);
+            $xfer += $output->writeString($iter888);
           }
         }
         $output->writeListEnd();
@@ -25295,14 +26013,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size847 = 0;
-            $_etype850 = 0;
-            $xfer += $input->readListBegin($_etype850, $_size847);
-            for ($_i851 = 0; $_i851 < $_size847; ++$_i851)
+            $_size889 = 0;
+            $_etype892 = 0;
+            $xfer += $input->readListBegin($_etype892, $_size889);
+            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
             {
-              $elem852 = null;
-              $xfer += $input->readString($elem852);
-              $this->part_vals []= $elem852;
+              $elem894 = null;
+              $xfer += $input->readString($elem894);
+              $this->part_vals []= $elem894;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25347,9 +26065,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter853)
+          foreach ($this->part_vals as $iter895)
           {
-            $xfer += $output->writeString($iter853);
+            $xfer += $output->writeString($iter895);
           }
         }
         $output->writeListEnd();
@@ -25443,15 +26161,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size854 = 0;
-            $_etype857 = 0;
-            $xfer += $input->readListBegin($_etype857, $_size854);
-            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
+            $_size896 = 0;
+            $_etype899 = 0;
+            $xfer += $input->readListBegin($_etype899, $_size896);
+            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
             {
-              $elem859 = null;
-              $elem859 = new \metastore\Partition();
-              $xfer += $elem859->read($input);
-              $this->success []= $elem859;
+              $elem901 = null;
+              $elem901 = new \metastore\Partition();
+              $xfer += $elem901->read($input);
+              $this->success []= $elem901;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25495,9 +26213,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter860)
+          foreach ($this->success as $iter902)
           {
-            $xfer += $iter860->write($output);
+            $xfer += $iter902->write($output);
           }
         }
         $output->writeListEnd();
@@ -25644,14 +26362,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size861 = 0;
-            $_etype864 = 0;
-            $xfer += $input->readListBegin($_etype864, $_size861);
-            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
+            $_size903 = 0;
+            $_etype906 = 0;
+            $xfer += $input->readListBegin($_etype906, $_size903);
+            for ($_i907 = 0; $_i907 < $_size903; ++$_i907)
             {
-              $elem866 = null;
-              $xfer += $input->readString($elem866);
-              $this->part_vals []= $elem866;
+              $elem908 = null;
+              $xfer += $input->readString($elem908);
+              $this->part_vals []= 

<TRUNCATED>

[30/31] hive git commit: HIVE-15834: Add unit tests for org.json usage on master (Daniel Voros via Zoltan Haindrich)

Posted by we...@apache.org.
HIVE-15834: Add unit tests for org.json usage on master (Daniel Voros via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/hive-14535
Commit: 2fa4dc277e0cd28261602c392c2f55d040d22677
Parents: 811e599
Author: Daniel Voros <da...@gmail.com>
Authored: Thu May 25 21:16:44 2017 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Thu May 25 21:16:44 2017 +0200

----------------------------------------------------------------------
 .../hadoop/hive/common/jsonexplain/Op.java      |   4 +-
 .../hadoop/hive/common/jsonexplain/TestOp.java  |  81 +++++
 .../hive/common/jsonexplain/TestStage.java      | 194 ++++++++++++
 .../hive/common/jsonexplain/TestVertex.java     | 108 +++++++
 .../jsonexplain/tez/TestTezJsonParser.java      |  53 ++++
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |  19 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   4 +-
 .../hadoop/hive/ql/exec/TestExplainTask.java    | 293 ++++++++++++++++++-
 .../hadoop/hive/ql/hooks/TestATSHook.java       |  59 ++++
 9 files changed, 796 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/common/src/java/org/apache/hadoop/hive/common/jsonexplain/Op.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/Op.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/Op.java
index 39c44f1..e9eb5a7 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/Op.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/Op.java
@@ -29,6 +29,7 @@ import java.util.Set;
 
 import org.apache.hadoop.hive.common.jsonexplain.Vertex.VertexType;
 import org.json.JSONArray;
+import com.google.common.annotations.VisibleForTesting;
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -85,7 +86,8 @@ public final class Op {
     }
   }
 
-  private void inlineJoinOp() throws Exception {
+  @VisibleForTesting
+  void inlineJoinOp() throws Exception {
     // inline map join operator
     if (this.type == OpType.MAPJOIN) {
       // get the map for posToVertex

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestOp.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestOp.java b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestOp.java
new file mode 100644
index 0000000..eb5dca4
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestOp.java
@@ -0,0 +1,81 @@
+/**
+ * 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.common.jsonexplain;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.hive.common.jsonexplain.tez.TezJsonParser;
+import org.json.JSONObject;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestOp {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private TezJsonParser tezJsonParser;
+
+  @Before
+  public void setUp() throws Exception {
+    this.tezJsonParser = new TezJsonParser();
+  }
+
+
+  @Test
+  public void testInlineJoinOpJsonHandling() throws Exception {
+    String jsonString = "{" +
+            "\"input vertices:\":{\"a\":\"AVERTEX\"}," + "\"condition map:\": [" +
+            "{\"c1\": \"{\\\"type\\\": \\\"type\\\", \\\"left\\\": \\\"left\\\", " +
+            "\\\"right\\\": \\\"right\\\"}\"}]," +
+            "\"keys:\":{\"left\":\"AKEY\", \"right\":\"BKEY\"}}";
+    JSONObject mapJoin = new JSONObject(jsonString);
+
+    Vertex vertexB = new Vertex("vertex-b", null, null, tezJsonParser);
+    Op dummyOp = new Op("Dummy Op", "dummy-id", "output-vertex-name", null, Collections.EMPTY_LIST,
+            null, mapJoin, null, tezJsonParser);
+    vertexB.outputOps.add(dummyOp);
+
+    Vertex vertexC = new Vertex("vertex-c", null, null, tezJsonParser);
+    vertexC.outputOps.add(dummyOp);
+
+
+    Vertex vertexA = new Vertex("vertex-a", null, null, tezJsonParser);
+    vertexA.tagToInput = new HashMap<>();
+    vertexA.tagToInput.put("left", "vertex-b");
+    vertexA.tagToInput.put("right", "vertex-c");
+    vertexA.parentConnections.add(new Connection("left", vertexB));
+    vertexA.parentConnections.add(new Connection("right", vertexC));
+
+
+    Map<String, String> attrs = new HashMap<>();
+
+    Op uut = new Op("Map Join Operator", "op-id", "output-vertex-name", null, Collections.EMPTY_LIST,
+            attrs, mapJoin, vertexA, tezJsonParser);
+    uut.inlineJoinOp();
+
+    assertEquals(1, attrs.size());
+
+    String result = attrs.get("Conds:");
+    String expected = "dummy-id.AKEY=dummy-id.BKEY(type)";
+    assertEquals(expected, result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestStage.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestStage.java b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestStage.java
new file mode 100644
index 0000000..e344eb3
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestStage.java
@@ -0,0 +1,194 @@
+/**
+ * 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.common.jsonexplain;
+
+import org.apache.hadoop.hive.common.jsonexplain.tez.TezJsonParser;
+import org.json.JSONObject;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class TestStage {
+
+  private Stage uut;
+  private Stage stageA;
+  private Stage stageB;
+  private TezJsonParser tezJsonParser;
+
+  @Before
+  public void setUp() {
+    this.tezJsonParser = new TezJsonParser();
+    this.uut = new Stage("uut", tezJsonParser);
+    this.stageA = new Stage("stage-a", tezJsonParser);
+    this.stageB = new Stage("stage-b", tezJsonParser);
+  }
+
+  @Test
+  public void testAddDependencyNonRoot() throws Exception {
+    Map<String, Stage> children = new LinkedHashMap<>();
+    children.put("a", stageA);
+    children.put("b", stageB);
+
+
+    String jsonString = "{\"DEPENDENT STAGES\":\"a,b\"}";
+    JSONObject names = new JSONObject(jsonString);
+
+    uut.addDependency(names, children);
+
+    assertEquals(2, uut.parentStages.size());
+    assertEquals(stageA, uut.parentStages.get(0));
+    assertEquals(stageB, uut.parentStages.get(1));
+
+    assertEquals(1, stageA.childStages.size());
+    assertEquals(uut, stageA.childStages.get(0));
+
+    assertEquals(1, stageB.childStages.size());
+    assertEquals(uut, stageB.childStages.get(0));
+  }
+
+  @Test
+  public void testAddDependencyRoot() throws Exception {
+    Map<String, Stage> children = new LinkedHashMap<>();
+    children.put("a", stageA);
+    children.put("b", stageB);
+
+    String jsonString = "{\"ROOT STAGE\":\"X\",\"DEPENDENT STAGES\":\"a,b\"}";
+    JSONObject names = new JSONObject(jsonString);
+
+    uut.addDependency(names, children);
+
+    assertEquals(2, uut.parentStages.size());
+    assertEquals(1, stageA.childStages.size());
+    assertEquals(1, stageB.childStages.size());
+  }
+
+
+  @Test
+  public void testExtractVertexNonTez() throws Exception {
+    String jsonString = "{\"OperatorName\":{\"a\":\"A\",\"b\":\"B\"}," +
+            "\"attr1\":\"ATTR1\"}";
+    JSONObject object = new JSONObject(jsonString);
+
+    uut.extractVertex(object);
+
+    assertEquals("OperatorName", uut.op.name);
+    assertEquals(1, uut.attrs.size());
+    assertEquals("ATTR1", uut.attrs.get("attr1"));
+  }
+
+  @Test
+  public void testExtractVertexTezNoEdges() throws Exception {
+    String jsonString = "{\"Tez\":{\"a\":\"A\",\"Vertices:\":{\"v1\":{}}}}";
+    JSONObject object = new JSONObject(jsonString);
+    uut.extractVertex(object);
+
+    assertEquals(1, uut.vertexs.size());
+    assertTrue(uut.vertexs.containsKey("v1"));
+  }
+
+  @Test
+  public void testExtractVertexTezWithOneEdge() throws Exception {
+    String jsonString = "{\"Tez\":{\"a\":\"A\"," +
+            "\"Vertices:\":{\"v1\":{},\"v2\":{}}," +
+            "\"Edges:\":{\"v2\":{\"parent\":\"v1\",\"type\":\"TYPE\"}}}}";
+    JSONObject object = new JSONObject(jsonString);
+    uut.extractVertex(object);
+
+    assertEquals(2, uut.vertexs.size());
+    assertTrue(uut.vertexs.containsKey("v1"));
+    assertTrue(uut.vertexs.containsKey("v2"));
+
+    assertEquals(0, uut.vertexs.get("v1").parentConnections.size());
+    assertEquals(1, uut.vertexs.get("v2").parentConnections.size());
+    assertEquals("v1", uut.vertexs.get("v2").parentConnections.get(0).from.name);
+    assertEquals("TYPE", uut.vertexs.get("v2").parentConnections.get(0).type);
+
+  }
+
+
+  @Test
+  public void testExtractVertexTezWithOneToManyEdge() throws Exception {
+    String jsonString = "{\"Tez\":{\"a\":\"A\"," +
+            "\"Vertices:\":{\"v1\":{},\"v2\":{},\"v3\":{}}," +
+            "\"Edges:\":{\"v1\":[{\"parent\":\"v2\",\"type\":\"TYPE1\"}," +
+            "{\"parent\":\"v3\",\"type\":\"TYPE2\"}]}}}";
+    JSONObject object = new JSONObject(jsonString);
+
+    uut.extractVertex(object);
+
+    assertEquals(3, uut.vertexs.size());
+    assertTrue(uut.vertexs.containsKey("v1"));
+    assertTrue(uut.vertexs.containsKey("v2"));
+    assertTrue(uut.vertexs.containsKey("v3"));
+
+    assertEquals(2, uut.vertexs.get("v1").parentConnections.size());
+    assertEquals(1, uut.vertexs.get("v2").children.size());
+    assertEquals(1, uut.vertexs.get("v3").children.size());
+    assertEquals("v1", uut.vertexs.get("v2").children.get(0).name);
+    assertEquals("v1", uut.vertexs.get("v3").children.get(0).name);
+    assertEquals("TYPE1", uut.vertexs.get("v1").parentConnections.get(0).type);
+    assertEquals("TYPE2", uut.vertexs.get("v1").parentConnections.get(1).type);
+
+  }
+
+  @Test
+  public void testExtractOpEmptyObject() throws Exception {
+    JSONObject object = new JSONObject();
+    Op result = uut.extractOp("op-name", object);
+
+    assertEquals("op-name", result.name);
+    assertEquals(0, result.attrs.size());
+    assertNull(result.vertex);
+  }
+
+  @Test
+  public void testExtractOpSimple() throws Exception {
+    String jsonString = "{\"a\":\"A\",\"b\":\"B\"}";
+    JSONObject object = new JSONObject(jsonString);
+
+    Op result = uut.extractOp("op-name", object);
+
+    assertEquals("op-name", result.name);
+    assertEquals(2, result.attrs.size());
+    assertNull(result.vertex);
+  }
+
+  @Test
+  public void testExtract() throws Exception {
+    String jsonString = "{\"b\":{\"b2\":\"B2\",\"b1\":\"B1\"}," +
+            "\"Processor Tree:\":{\"a1\":{\"t1\":\"T1\"}}}";
+    JSONObject object = new JSONObject(jsonString);
+
+    Op result = uut.extractOp("op-name", object);
+    assertEquals("op-name", result.name);
+    assertEquals(2, result.attrs.size());
+
+    List<String> attrs = new ArrayList<>();
+    assertEquals("B1", result.attrs.get("b1"));
+    assertEquals("B2", result.attrs.get("b2"));
+    assertNotNull(result.vertex);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestVertex.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestVertex.java b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestVertex.java
new file mode 100644
index 0000000..4303be7
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/TestVertex.java
@@ -0,0 +1,108 @@
+/**
+ * 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.common.jsonexplain;
+
+import org.apache.hadoop.hive.common.jsonexplain.tez.TezJsonParser;
+import org.json.JSONObject;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestVertex {
+
+  private TezJsonParser tezJsonParser;
+
+  @Before
+  public void setUp() throws Exception {
+    this.tezJsonParser = new TezJsonParser();
+  }
+
+
+  @Test
+  public void testExtractOpTree() throws Exception {
+    JSONObject object = new JSONObject("{\"Join:\":[{},{}]}");
+
+    Vertex uut = new Vertex("name", object, null, tezJsonParser);
+    uut.extractOpTree();
+
+    assertEquals(2, uut.mergeJoinDummyVertexs.size());
+  }
+
+  @Test
+  public void testExtractOpNonJsonChildrenShouldThrow() throws Exception {
+    String jsonString = "{\"opName\":{\"children\":\"not-json\"}}";
+    JSONObject operator = new JSONObject(jsonString);
+
+    Vertex uut = new Vertex("name", null, null, tezJsonParser);
+
+    try {
+      uut.extractOp(operator, null);
+    } catch (Exception e) {
+      assertEquals("Unsupported operator name's children operator is neither a jsonobject nor a jsonarray", e.getMessage());
+    }
+  }
+
+  @Test
+  public void testExtractOpNoChildrenOperatorId() throws Exception {
+    String jsonString = "{\"opName\":{\"OperatorId:\":\"operator-id\"}}";
+    JSONObject operator = new JSONObject(jsonString);
+
+    Vertex uut = new Vertex("name", null, null, tezJsonParser);
+
+    Op result = uut.extractOp(operator, null);
+    assertEquals("opName", result.name);
+    assertEquals("operator-id", result.operatorId);
+    assertEquals(0, result.children.size());
+    assertEquals(0, result.attrs.size());
+  }
+
+  @Test
+  public void testExtractOpOneChild() throws Exception {
+    String jsonString = "{\"opName\":{\"children\":{\"childName\":" +
+            "{\"OperatorId:\":\"child-operator-id\"}}}}";
+    JSONObject operator = new JSONObject(jsonString);
+
+    Vertex uut = new Vertex("name", null, null, tezJsonParser);
+
+    Op result = uut.extractOp(operator, null);
+    assertEquals("opName", result.name);
+    assertEquals(1, result.children.size());
+    assertEquals("childName", result.children.get(0).name);
+    assertEquals("child-operator-id", result.children.get(0).operatorId);
+  }
+
+  @Test
+  public void testExtractOpMultipleChildren() throws Exception {
+    String jsonString = "{\"opName\":{\"children\":[" +
+            "{\"childName1\":{\"OperatorId:\":\"child-operator-id1\"}}," +
+            "{\"childName2\":{\"OperatorId:\":\"child-operator-id2\"}}]}}";
+    JSONObject operator = new JSONObject(jsonString);
+
+    Vertex uut = new Vertex("name", null, null, tezJsonParser);
+
+    Op result = uut.extractOp(operator, null);
+    assertEquals("opName", result.name);
+    assertEquals(2, result.children.size());
+    assertEquals("childName1", result.children.get(0).name);
+    assertEquals("child-operator-id1", result.children.get(0).operatorId);
+    assertEquals("childName2", result.children.get(1).name);
+    assertEquals("child-operator-id2", result.children.get(1).operatorId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/common/src/test/org/apache/hadoop/hive/common/jsonexplain/tez/TestTezJsonParser.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/jsonexplain/tez/TestTezJsonParser.java b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/tez/TestTezJsonParser.java
new file mode 100644
index 0000000..cf34ab8
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/jsonexplain/tez/TestTezJsonParser.java
@@ -0,0 +1,53 @@
+/**
+ * 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.common.jsonexplain.tez;
+
+import org.json.JSONObject;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTezJsonParser {
+
+  private TezJsonParser uut;
+
+  @Before
+  public void setUp() throws Exception {
+    this.uut = new TezJsonParser();
+  }
+
+  @Test
+  public void testExtractStagesAndPlans() throws Exception {
+    String jsonString = "{\"STAGE DEPENDENCIES\":{\"s1\":{\"ROOT STAGE\":\"\"}," +
+            "\"s2\":{\"DEPENDENT STAGES\":\"s1\"}},\"STAGE PLANS\":{}}";
+    JSONObject input = new JSONObject(jsonString);
+
+    uut.extractStagesAndPlans(input);
+
+    assertEquals(2, uut.stages.size());
+    assertEquals(1, uut.stages.get("s1").childStages.size());
+    assertEquals("s2", uut.stages.get("s1").childStages.get(0).internalName);
+    assertEquals(0, uut.stages.get("s2").childStages.size());
+    assertEquals(0, uut.stages.get("s1").parentStages.size());
+    assertEquals(1, uut.stages.get("s2").parentStages.size());
+    assertEquals("s1", uut.stages.get("s2").parentStages.get(0).internalName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
index 8ddb8d6..902664d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
@@ -42,6 +42,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.jsonexplain.JsonParser;
@@ -132,7 +133,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
    * {"input_tables":[{"tablename": "default@test_sambavi_v1", "tabletype": "TABLE"}],
    *  "input partitions":["default@srcpart@ds=2008-04-08/hr=11"]}
    */
-  private static JSONObject getJSONDependencies(ExplainWork work)
+  @VisibleForTesting
+  static JSONObject getJSONDependencies(ExplainWork work)
       throws Exception {
     assert(work.getDependency());
 
@@ -202,7 +204,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
   private static String falseCondNameVectorizationEnabled =
       HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED.varname + " IS false";
 
-  private ImmutablePair<Boolean, JSONObject> outputPlanVectorization(PrintStream out, boolean jsonOutput)
+  @VisibleForTesting
+  ImmutablePair<Boolean, JSONObject> outputPlanVectorization(PrintStream out, boolean jsonOutput)
       throws Exception {
 
     if (out != null) {
@@ -422,7 +425,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     }
   }
 
-  private JSONObject collectAuthRelatedEntities(PrintStream out, ExplainWork work)
+  @VisibleForTesting
+  JSONObject collectAuthRelatedEntities(PrintStream out, ExplainWork work)
       throws Exception {
 
     BaseSemanticAnalyzer analyzer = work.getAnalyzer();
@@ -486,7 +490,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     return sb.toString();
   }
 
-  private JSONObject outputMap(Map<?, ?> mp, boolean hasHeader, PrintStream out,
+  @VisibleForTesting
+  JSONObject outputMap(Map<?, ?> mp, boolean hasHeader, PrintStream out,
       boolean extended, boolean jsonOutput, int indent) throws Exception {
 
     TreeMap<Object, Object> tree = getBasictypeKeyedMap(mp);
@@ -696,7 +701,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     return outputPlan(work, out, extended, jsonOutput, indent, "");
   }
 
-  private JSONObject outputPlan(Object work, PrintStream out,
+  @VisibleForTesting
+  JSONObject outputPlan(Object work, PrintStream out,
       boolean extended, boolean jsonOutput, int indent, String appendToHeader) throws Exception {
     // Check if work has an explain annotation
     Annotation note = AnnotationUtils.getAnnotation(work.getClass(), Explain.class);
@@ -1072,7 +1078,8 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     return null;
   }
 
-  private JSONObject outputDependencies(Task<?> task,
+  @VisibleForTesting
+  JSONObject outputDependencies(Task<?> task,
       PrintStream out, JSONObject parentJson, boolean jsonOutput, boolean taskType, int indent)
       throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index f44661e..84f992a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
@@ -78,7 +79,8 @@ public class ATSHook implements ExecuteWithHookContext {
   private static boolean defaultATSDomainCreated = false;
   private static final String DEFAULT_ATS_DOMAIN = "hive_default_ats";
 
-  private enum OtherInfoTypes {
+  @VisibleForTesting
+  enum OtherInfoTypes {
     QUERY, STATUS, TEZ, MAPRED, INVOKER_INFO, SESSION_ID, THREAD_NAME, VERSION,
     CLIENT_IP_ADDRESS, HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID
   };

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java
index 805bc5b..cac1fad 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java
@@ -18,26 +18,52 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.PrintStream;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.hooks.Entity;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ExplainConfiguration;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.plan.Explain;
+import org.apache.hadoop.hive.ql.plan.*;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
-import org.apache.hadoop.hive.ql.plan.ExplainWork;
-import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-import org.junit.Ignore;
+import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.io.PrintStream;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestExplainTask {
 
+  private static final String BACKUP_ID = "backup-id-mock";
+  private static final String AST = "ast-mock";
+
+  private PrintStream out;
+  private ExplainTask uut;
+  private ObjectMapper objectMapper = new ObjectMapper();
+
+  @Before
+  public void setUp() {
+    uut = new ExplainTask();
+    uut.conf = mock(HiveConf.class);
+    out = mock(PrintStream.class);
+  }
+
   public static class DummyExplainDesc<K, V> extends TableScanDesc {
     private static final long serialVersionUID = 1L;
     private Map<K, V> explainResult;
@@ -139,4 +165,249 @@ public class TestExplainTask {
     return baos.toString();
   }
 
+  @Test
+  public void testGetJSONDependenciesJsonShhouldMatch() throws Exception {
+    ExplainWork work = mockExplainWork();
+
+    when(work.getDependency()).thenReturn(true);
+
+    // Mock inputs
+    HashSet<ReadEntity> inputs = new HashSet<>();
+
+    // One input table
+    Table table = mock(Table.class);
+    when(table.getCompleteName()).thenReturn("table-name-mock");
+    when(table.getTableType()).thenReturn(TableType.EXTERNAL_TABLE);
+    ReadEntity input1 = mock(ReadEntity.class);
+    when(input1.getType()).thenReturn(Entity.Type.TABLE);
+    when(input1.getTable()).thenReturn(table);
+    inputs.add(input1);
+
+    // And one partition
+    Partition partition = mock(Partition.class);
+    when(partition.getCompleteName()).thenReturn("partition-name-mock");
+    ReadEntity input2 = mock(ReadEntity.class);
+    when(input2.getType()).thenReturn(Entity.Type.PARTITION);
+    when(input2.getPartition()).thenReturn(partition);
+    inputs.add(input2);
+
+    when(work.getInputs()).thenReturn(inputs);
+
+    JsonNode result = objectMapper.readTree(ExplainTask.getJSONDependencies(work).toString());
+    JsonNode expected = objectMapper.readTree("{\"input_partitions\":[{\"partitionName\":" +
+            "\"partition-name-mock\"}],\"input_tables\":[{\"tablename\":\"table-name-mock\"," +
+            "\"tabletype\":\"EXTERNAL_TABLE\"}]}");
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testGetJSONPlan() throws Exception {
+    when(uut.conf.getVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE)).thenReturn("EXECUTION");
+    Task mockTask = mockTask();
+    when(mockTask.getId()).thenReturn("mockTaskId");
+    ExplainWork explainWorkMock = mockExplainWork();
+    when(mockTask.getWork()).thenReturn(explainWorkMock);
+    List<Task<?>> tasks = Arrays.<Task<?>>asList(mockTask);
+
+
+    JsonNode result = objectMapper.readTree(uut.getJSONPlan(null, tasks, null, true,
+            false, false).toString());
+    JsonNode expected = objectMapper.readTree("{\"STAGE DEPENDENCIES\":{\"mockTaskId\":" +
+            "{\"ROOT STAGE\":\"TRUE\",\"BACKUP STAGE\":\"backup-id-mock\"}},\"STAGE PLANS\":" +
+            "{\"mockTaskId\":{}}}");
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testOutputDependenciesJsonShouldMatch() throws Exception {
+    Task<? extends ExplainTask> task = mockTask();
+
+    JsonNode result = objectMapper.readTree(
+            uut.outputDependencies(task, out, null, true, true, 0).toString());
+    JsonNode expected = objectMapper.readTree("{\"ROOT STAGE\":\"TRUE\",\"BACKUP STAGE\":" +
+            "\""+BACKUP_ID+"\",\"TASK TYPE\":\"EXPLAIN\"}");
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testGetJSONLogicalPlanJsonShouldMatch() throws Exception {
+    JsonNode result = objectMapper.readTree(
+            uut.getJSONLogicalPlan(null, mockExplainWork()).toString());
+    JsonNode expected = objectMapper.readTree("{}");
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testOutputMapJsonShouldMatch() throws Exception {
+    Map<Object, Object> map = new LinkedHashMap<>();
+
+    // String
+    map.put("key-1", "value-1");
+
+    // SparkWork
+    map.put("spark-work", new SparkWork("spark-work"));
+
+    // Empty list
+    List<Object> emptList = Collections.emptyList();
+    map.put("empty-list", emptList);
+
+    // List of TezWork.Dependency
+    List<Object> tezList1 = new ArrayList<>(Arrays.asList(new Object[] {mockTezWorkDependency()}));
+    map.put("tez-list-1", tezList1);
+    List<Object> tezList2 = new ArrayList<>(
+            Arrays.asList(new Object[] {mockTezWorkDependency(), mockTezWorkDependency()}));
+    map.put("tez-list-2", tezList2);
+
+    // List of SparkWork.Dependency
+    List<Object> sparkList1 = new ArrayList<>(
+            Arrays.asList(new Object[]{mockSparkWorkDependency()}));
+    map.put("spark-list-1", sparkList1);
+    List<Object> sparkList2 = new ArrayList<>(
+            Arrays.asList(new Object[]{mockSparkWorkDependency(), mockSparkWorkDependency()}));
+    map.put("spark-list-2", sparkList2);
+
+    // inner Map
+    Map<Object, Object> innerMap = new LinkedHashMap<>();
+    innerMap.put("inner-key-1", "inner-value-1");
+    innerMap.put("inner-key-2", tezList1);
+    map.put("map-1", innerMap);
+
+    JsonNode result = objectMapper.readTree(
+            uut.outputMap(map, false, null, false, true, 0).toString());
+    JsonNode expected = objectMapper.readTree("{\"key-1\":\"value-1\",\"tez-list-2\":" +
+            "[{\"parent\":\"name\"}," + "{\"parent\":\"name\"}],\"tez-list-1\":" +
+            "{\"parent\":\"name\"},\"empty-list\":\"[]\",\"spark-list-2\":" +
+            "[{\"parent\":\"mock-name\"},{\"parent\":\"mock-name\"}]," +
+            "\"spark-list-1\":{\"parent\":" +
+            "\"mock-name\"}, \"map-1\":\"{inner-key-1=inner-value-1, " +
+            "inner-key-2=[mock-tez-dependency]}\",\"spark-work\":" +
+            "{\"Spark\":{\"DagName:\":\"spark-work:2\"}}}");
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testOutputPlanJsonShouldMatch() throws Exception {
+    // SparkWork
+    SparkWork work = new SparkWork("spark-work");
+
+    JsonNode result = objectMapper.readTree(
+            uut.outputPlan(work, null, false, true, 0, null).toString());
+    JsonNode expected = objectMapper.readTree("{\"Spark\":{\"DagName:\":\"spark-work:1\"}}");
+    assertEquals(expected, result);
+
+    // Operator with single child
+    CollectOperator parentCollectOperator1 = new CollectOperator();
+    CollectOperator child1 = new CollectOperator();
+    parentCollectOperator1.setChildOperators(new ArrayList<Operator<? extends OperatorDesc>>(
+            Arrays.asList(new CollectOperator[] {child1})));
+    parentCollectOperator1.setConf(new CollectDesc());
+
+    result = objectMapper.readTree(
+            uut.outputPlan(parentCollectOperator1, null, false, true, 0, null).toString());
+    expected = objectMapper.readTree("{\"Collect\":{\"children\":{}}}");
+    assertEquals(expected, result);
+
+    // Operator with 2 children
+    CollectOperator parentCollectOperator2 = new CollectOperator();
+    CollectOperator child2 = new CollectOperator();
+    parentCollectOperator2.setChildOperators(new ArrayList<Operator<? extends OperatorDesc>>(
+            Arrays.asList(new CollectOperator[] {child1, child2})));
+    parentCollectOperator2.setConf(new CollectDesc());
+    result = objectMapper.readTree(
+            uut.outputPlan(parentCollectOperator2, null, false, true, 0, null).toString());
+    expected = objectMapper.readTree("{\"Collect\":{\"children\":[{},{}]}}");
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCollectAuthRelatedEntitiesJsonShouldMatch() throws Exception {
+    QueryState qs = mock(QueryState.class);
+    when(qs.getHiveOperation()).thenReturn(HiveOperation.EXPLAIN);
+    uut.queryState = qs;
+
+    SessionState.start(new HiveConf(ExplainTask.class));
+    // SessionState.get().setCommandType(HiveOperation.EXPLAIN);
+    HiveAuthenticationProvider authenticationProviderMock = mock(HiveAuthenticationProvider.class);
+    when(authenticationProviderMock.getUserName()).thenReturn("test-user");
+    SessionState.get().setAuthenticator(authenticationProviderMock);
+    SessionState.get().setAuthorizer(mock(HiveAuthorizationProvider.class));
+    ExplainWork work = mockExplainWork();
+
+    JsonNode result = objectMapper.readTree(uut.collectAuthRelatedEntities(null, work).toString());
+    JsonNode expected = objectMapper.readTree("{\"CURRENT_USER\":\"test-user\"," +
+            "\"OPERATION\":\"EXPLAIN\",\"INPUTS\":[],\"OUTPUTS\":[]}");
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testOutputPlanVectorizationJsonShouldMatch() throws Exception {
+    QueryState qs = mock(QueryState.class);
+    when(qs.getHiveOperation()).thenReturn(HiveOperation.EXPLAIN);
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, true);
+    when(qs.getConf()).thenReturn(hiveConf);
+    uut.queryState = qs;
+
+    JsonNode result = objectMapper.readTree(uut.outputPlanVectorization(null, true).getRight().toString());
+    JsonNode expected = objectMapper.readTree("{\"enabled\":true,\"enabledConditionsMet\":[\"hive.vectorized.execution.enabled IS true\"]}");
+    assertEquals(expected, result);
+
+
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false);
+    result = objectMapper.readTree(uut.outputPlanVectorization(null, true).getRight().toString());
+    expected = objectMapper.readTree("{\"enabled\":false,\"enabledConditionsNotMet\":[\"hive.vectorized.execution.enabled IS false\"]}");
+    assertEquals(expected, result);
+
+  }
+
+  private TezWork.Dependency mockTezWorkDependency() {
+    TezWork.Dependency dep = mock(TezWork.Dependency.class);
+    when(dep.getName()).thenReturn("name");
+    when(dep.toString()).thenReturn("mock-tez-dependency");
+    return dep;
+  }
+
+  private SparkWork.Dependency mockSparkWorkDependency() {
+    SparkWork.Dependency dep = mock(SparkWork.Dependency.class);
+    when(dep.getName()).thenReturn("mock-name");
+    when(dep.toString()).thenReturn("mock-spark-dependency");
+    return dep;
+  }
+
+  private ExplainWork mockExplainWork() {
+    ExplainWork explainWork = mock(ExplainWork.class);
+
+    // Should produce JSON
+    when(explainWork.isFormatted()).thenReturn(true);
+
+    // Should have some AST
+    // when(explainWork.getAstStringTree()).thenReturn(AST);
+
+    when(explainWork.getAnalyzer()).thenReturn(mock(BaseSemanticAnalyzer.class));
+
+    return explainWork;
+  }
+
+  private Task<ExplainTask> mockTask() {
+    Task<ExplainTask> task = mock(Task.class);
+
+    // Explain type
+    when(task.getType()).thenReturn(StageType.EXPLAIN);
+
+    // This is a root task
+    when(task.isRootTask()).thenReturn(true);
+
+    // Set up backup task
+    Task backupTask = mock(Task.class);
+    when(backupTask.getId()).thenReturn(BACKUP_ID);
+    when(task.getBackupTask()).thenReturn(backupTask);
+
+    return task;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2fa4dc27/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestATSHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestATSHook.java
new file mode 100644
index 0000000..c484062
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestATSHook.java
@@ -0,0 +1,59 @@
+/**
+ * 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.hooks;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestATSHook {
+
+  private ObjectMapper objectMapper = new ObjectMapper();
+  private ATSHook uut;
+
+  @Before
+  public void setUp() {
+    uut = new ATSHook();
+  }
+
+  @Test
+  public void testCreatePreHookEventJsonShhouldMatch() throws Exception {
+    TimelineEntity timelineEntity =  uut.createPreHookEvent(
+            "test-query-id", "test-query", new org.json.JSONObject(), 0L,
+            "test-user", "test-request-user", 0, 0, "test-opid",
+            "client-ip-address", "hive-instance-address", "hive-instance-type", "session-id", "log-id",
+            "thread-id", "execution-mode", Collections.<String>emptyList(), Collections.<String>emptyList(),
+            new HiveConf(), null, "domain-id");
+    String resultStr = (String) timelineEntity.getOtherInfo()
+            .get(ATSHook.OtherInfoTypes.QUERY.name());
+
+    JsonNode result = objectMapper.readTree(resultStr);
+    JsonNode expected = objectMapper.readTree("{\"queryText\":\"test-query\"," +
+            "\"queryPlan\":{}}");
+
+    assertEquals(expected, result);
+  }
+}


[19/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
new file mode 100644
index 0000000..01dbc68
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
@@ -0,0 +1,1005 @@
+/**
+ * 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)")
+public class SQLNotNullConstraint implements org.apache.thrift.TBase<SQLNotNullConstraint, SQLNotNullConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLNotNullConstraint> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLNotNullConstraint");
+
+  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 NN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("nn_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  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)5);
+  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)6);
+  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)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SQLNotNullConstraintStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SQLNotNullConstraintTupleSchemeFactory());
+  }
+
+  private String table_db; // required
+  private String table_name; // required
+  private String column_name; // required
+  private String nn_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"),
+    NN_NAME((short)4, "nn_name"),
+    ENABLE_CSTR((short)5, "enable_cstr"),
+    VALIDATE_CSTR((short)6, "validate_cstr"),
+    RELY_CSTR((short)7, "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: // NN_NAME
+          return NN_NAME;
+        case 5: // ENABLE_CSTR
+          return ENABLE_CSTR;
+        case 6: // VALIDATE_CSTR
+          return VALIDATE_CSTR;
+        case 7: // 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.NN_NAME, new org.apache.thrift.meta_data.FieldMetaData("nn_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(SQLNotNullConstraint.class, metaDataMap);
+  }
+
+  public SQLNotNullConstraint() {
+  }
+
+  public SQLNotNullConstraint(
+    String table_db,
+    String table_name,
+    String column_name,
+    String nn_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.nn_name = nn_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 SQLNotNullConstraint(SQLNotNullConstraint 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.isSetNn_name()) {
+      this.nn_name = other.nn_name;
+    }
+    this.enable_cstr = other.enable_cstr;
+    this.validate_cstr = other.validate_cstr;
+    this.rely_cstr = other.rely_cstr;
+  }
+
+  public SQLNotNullConstraint deepCopy() {
+    return new SQLNotNullConstraint(this);
+  }
+
+  @Override
+  public void clear() {
+    this.table_db = null;
+    this.table_name = null;
+    this.column_name = null;
+    this.nn_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 getNn_name() {
+    return this.nn_name;
+  }
+
+  public void setNn_name(String nn_name) {
+    this.nn_name = nn_name;
+  }
+
+  public void unsetNn_name() {
+    this.nn_name = null;
+  }
+
+  /** Returns true if field nn_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetNn_name() {
+    return this.nn_name != null;
+  }
+
+  public void setNn_nameIsSet(boolean value) {
+    if (!value) {
+      this.nn_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 NN_NAME:
+      if (value == null) {
+        unsetNn_name();
+      } else {
+        setNn_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 NN_NAME:
+      return getNn_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 NN_NAME:
+      return isSetNn_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 SQLNotNullConstraint)
+      return this.equals((SQLNotNullConstraint)that);
+    return false;
+  }
+
+  public boolean equals(SQLNotNullConstraint 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_nn_name = true && this.isSetNn_name();
+    boolean that_present_nn_name = true && that.isSetNn_name();
+    if (this_present_nn_name || that_present_nn_name) {
+      if (!(this_present_nn_name && that_present_nn_name))
+        return false;
+      if (!this.nn_name.equals(that.nn_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_nn_name = true && (isSetNn_name());
+    list.add(present_nn_name);
+    if (present_nn_name)
+      list.add(nn_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(SQLNotNullConstraint 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(isSetNn_name()).compareTo(other.isSetNn_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNn_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nn_name, other.nn_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("SQLNotNullConstraint(");
+    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("nn_name:");
+    if (this.nn_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.nn_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 SQLNotNullConstraintStandardSchemeFactory implements SchemeFactory {
+    public SQLNotNullConstraintStandardScheme getScheme() {
+      return new SQLNotNullConstraintStandardScheme();
+    }
+  }
+
+  private static class SQLNotNullConstraintStandardScheme extends StandardScheme<SQLNotNullConstraint> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SQLNotNullConstraint 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: // NN_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.nn_name = iprot.readString();
+              struct.setNn_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // 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 6: // 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 7: // 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, SQLNotNullConstraint 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.nn_name != null) {
+        oprot.writeFieldBegin(NN_NAME_FIELD_DESC);
+        oprot.writeString(struct.nn_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 SQLNotNullConstraintTupleSchemeFactory implements SchemeFactory {
+    public SQLNotNullConstraintTupleScheme getScheme() {
+      return new SQLNotNullConstraintTupleScheme();
+    }
+  }
+
+  private static class SQLNotNullConstraintTupleScheme extends TupleScheme<SQLNotNullConstraint> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SQLNotNullConstraint 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.isSetNn_name()) {
+        optionals.set(3);
+      }
+      if (struct.isSetEnable_cstr()) {
+        optionals.set(4);
+      }
+      if (struct.isSetValidate_cstr()) {
+        optionals.set(5);
+      }
+      if (struct.isSetRely_cstr()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
+      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.isSetNn_name()) {
+        oprot.writeString(struct.nn_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, SQLNotNullConstraint struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(7);
+      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.nn_name = iprot.readString();
+        struct.setNn_nameIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.enable_cstr = iprot.readBool();
+        struct.setEnable_cstrIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.validate_cstr = iprot.readBool();
+        struct.setValidate_cstrIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.rely_cstr = iprot.readBool();
+        struct.setRely_cstrIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
new file mode 100644
index 0000000..5b78613
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
@@ -0,0 +1,1103 @@
+/**
+ * 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)")
+public class SQLUniqueConstraint implements org.apache.thrift.TBase<SQLUniqueConstraint, SQLUniqueConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLUniqueConstraint> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLUniqueConstraint");
+
+  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 KEY_SEQ_FIELD_DESC = new org.apache.thrift.protocol.TField("key_seq", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField UK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("uk_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 SQLUniqueConstraintStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SQLUniqueConstraintTupleSchemeFactory());
+  }
+
+  private String table_db; // required
+  private String table_name; // required
+  private String column_name; // required
+  private int key_seq; // required
+  private String uk_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"),
+    KEY_SEQ((short)4, "key_seq"),
+    UK_NAME((short)5, "uk_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: // KEY_SEQ
+          return KEY_SEQ;
+        case 5: // UK_NAME
+          return UK_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 __KEY_SEQ_ISSET_ID = 0;
+  private static final int __ENABLE_CSTR_ISSET_ID = 1;
+  private static final int __VALIDATE_CSTR_ISSET_ID = 2;
+  private static final int __RELY_CSTR_ISSET_ID = 3;
+  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.KEY_SEQ, new org.apache.thrift.meta_data.FieldMetaData("key_seq", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UK_NAME, new org.apache.thrift.meta_data.FieldMetaData("uk_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(SQLUniqueConstraint.class, metaDataMap);
+  }
+
+  public SQLUniqueConstraint() {
+  }
+
+  public SQLUniqueConstraint(
+    String table_db,
+    String table_name,
+    String column_name,
+    int key_seq,
+    String uk_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.key_seq = key_seq;
+    setKey_seqIsSet(true);
+    this.uk_name = uk_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 SQLUniqueConstraint(SQLUniqueConstraint 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;
+    }
+    this.key_seq = other.key_seq;
+    if (other.isSetUk_name()) {
+      this.uk_name = other.uk_name;
+    }
+    this.enable_cstr = other.enable_cstr;
+    this.validate_cstr = other.validate_cstr;
+    this.rely_cstr = other.rely_cstr;
+  }
+
+  public SQLUniqueConstraint deepCopy() {
+    return new SQLUniqueConstraint(this);
+  }
+
+  @Override
+  public void clear() {
+    this.table_db = null;
+    this.table_name = null;
+    this.column_name = null;
+    setKey_seqIsSet(false);
+    this.key_seq = 0;
+    this.uk_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 int getKey_seq() {
+    return this.key_seq;
+  }
+
+  public void setKey_seq(int key_seq) {
+    this.key_seq = key_seq;
+    setKey_seqIsSet(true);
+  }
+
+  public void unsetKey_seq() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  /** Returns true if field key_seq is set (has been assigned a value) and false otherwise */
+  public boolean isSetKey_seq() {
+    return EncodingUtils.testBit(__isset_bitfield, __KEY_SEQ_ISSET_ID);
+  }
+
+  public void setKey_seqIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEY_SEQ_ISSET_ID, value);
+  }
+
+  public String getUk_name() {
+    return this.uk_name;
+  }
+
+  public void setUk_name(String uk_name) {
+    this.uk_name = uk_name;
+  }
+
+  public void unsetUk_name() {
+    this.uk_name = null;
+  }
+
+  /** Returns true if field uk_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetUk_name() {
+    return this.uk_name != null;
+  }
+
+  public void setUk_nameIsSet(boolean value) {
+    if (!value) {
+      this.uk_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 KEY_SEQ:
+      if (value == null) {
+        unsetKey_seq();
+      } else {
+        setKey_seq((Integer)value);
+      }
+      break;
+
+    case UK_NAME:
+      if (value == null) {
+        unsetUk_name();
+      } else {
+        setUk_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 KEY_SEQ:
+      return getKey_seq();
+
+    case UK_NAME:
+      return getUk_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 KEY_SEQ:
+      return isSetKey_seq();
+    case UK_NAME:
+      return isSetUk_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 SQLUniqueConstraint)
+      return this.equals((SQLUniqueConstraint)that);
+    return false;
+  }
+
+  public boolean equals(SQLUniqueConstraint 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_key_seq = true;
+    boolean that_present_key_seq = true;
+    if (this_present_key_seq || that_present_key_seq) {
+      if (!(this_present_key_seq && that_present_key_seq))
+        return false;
+      if (this.key_seq != that.key_seq)
+        return false;
+    }
+
+    boolean this_present_uk_name = true && this.isSetUk_name();
+    boolean that_present_uk_name = true && that.isSetUk_name();
+    if (this_present_uk_name || that_present_uk_name) {
+      if (!(this_present_uk_name && that_present_uk_name))
+        return false;
+      if (!this.uk_name.equals(that.uk_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_key_seq = true;
+    list.add(present_key_seq);
+    if (present_key_seq)
+      list.add(key_seq);
+
+    boolean present_uk_name = true && (isSetUk_name());
+    list.add(present_uk_name);
+    if (present_uk_name)
+      list.add(uk_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(SQLUniqueConstraint 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(isSetKey_seq()).compareTo(other.isSetKey_seq());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetKey_seq()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key_seq, other.key_seq);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUk_name()).compareTo(other.isSetUk_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUk_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uk_name, other.uk_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("SQLUniqueConstraint(");
+    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("key_seq:");
+    sb.append(this.key_seq);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uk_name:");
+    if (this.uk_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.uk_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 SQLUniqueConstraintStandardSchemeFactory implements SchemeFactory {
+    public SQLUniqueConstraintStandardScheme getScheme() {
+      return new SQLUniqueConstraintStandardScheme();
+    }
+  }
+
+  private static class SQLUniqueConstraintStandardScheme extends StandardScheme<SQLUniqueConstraint> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SQLUniqueConstraint 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: // KEY_SEQ
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.key_seq = iprot.readI32();
+              struct.setKey_seqIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // UK_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.uk_name = iprot.readString();
+              struct.setUk_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, SQLUniqueConstraint 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();
+      }
+      oprot.writeFieldBegin(KEY_SEQ_FIELD_DESC);
+      oprot.writeI32(struct.key_seq);
+      oprot.writeFieldEnd();
+      if (struct.uk_name != null) {
+        oprot.writeFieldBegin(UK_NAME_FIELD_DESC);
+        oprot.writeString(struct.uk_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 SQLUniqueConstraintTupleSchemeFactory implements SchemeFactory {
+    public SQLUniqueConstraintTupleScheme getScheme() {
+      return new SQLUniqueConstraintTupleScheme();
+    }
+  }
+
+  private static class SQLUniqueConstraintTupleScheme extends TupleScheme<SQLUniqueConstraint> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SQLUniqueConstraint 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.isSetKey_seq()) {
+        optionals.set(3);
+      }
+      if (struct.isSetUk_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.isSetKey_seq()) {
+        oprot.writeI32(struct.key_seq);
+      }
+      if (struct.isSetUk_name()) {
+        oprot.writeString(struct.uk_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, SQLUniqueConstraint 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.key_seq = iprot.readI32();
+        struct.setKey_seqIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.uk_name = iprot.readString();
+        struct.setUk_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/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index ed86165..1dd13cc 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list534 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list534.size);
-                ShowCompactResponseElement _elem535;
-                for (int _i536 = 0; _i536 < _list534.size; ++_i536)
+                org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list566.size);
+                ShowCompactResponseElement _elem567;
+                for (int _i568 = 0; _i568 < _list566.size; ++_i568)
                 {
-                  _elem535 = new ShowCompactResponseElement();
-                  _elem535.read(iprot);
-                  struct.compacts.add(_elem535);
+                  _elem567 = new ShowCompactResponseElement();
+                  _elem567.read(iprot);
+                  struct.compacts.add(_elem567);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter537 : struct.compacts)
+          for (ShowCompactResponseElement _iter569 : struct.compacts)
           {
-            _iter537.write(oprot);
+            _iter569.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter538 : struct.compacts)
+        for (ShowCompactResponseElement _iter570 : struct.compacts)
         {
-          _iter538.write(oprot);
+          _iter570.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
     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 _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list539.size);
-        ShowCompactResponseElement _elem540;
-        for (int _i541 = 0; _i541 < _list539.size; ++_i541)
+        org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list571.size);
+        ShowCompactResponseElement _elem572;
+        for (int _i573 = 0; _i573 < _list571.size; ++_i573)
         {
-          _elem540 = new ShowCompactResponseElement();
-          _elem540.read(iprot);
-          struct.compacts.add(_elem540);
+          _elem572 = new ShowCompactResponseElement();
+          _elem572.read(iprot);
+          struct.compacts.add(_elem572);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index da181e6..11ef050 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list500.size);
-                ShowLocksResponseElement _elem501;
-                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
+                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list532.size);
+                ShowLocksResponseElement _elem533;
+                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                 {
-                  _elem501 = new ShowLocksResponseElement();
-                  _elem501.read(iprot);
-                  struct.locks.add(_elem501);
+                  _elem533 = new ShowLocksResponseElement();
+                  _elem533.read(iprot);
+                  struct.locks.add(_elem533);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter503 : struct.locks)
+          for (ShowLocksResponseElement _iter535 : struct.locks)
           {
-            _iter503.write(oprot);
+            _iter535.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter504 : struct.locks)
+          for (ShowLocksResponseElement _iter536 : struct.locks)
           {
-            _iter504.write(oprot);
+            _iter536.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list505.size);
-          ShowLocksResponseElement _elem506;
-          for (int _i507 = 0; _i507 < _list505.size; ++_i507)
+          org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list537.size);
+          ShowLocksResponseElement _elem538;
+          for (int _i539 = 0; _i539 < _list537.size; ++_i539)
           {
-            _elem506 = new ShowLocksResponseElement();
-            _elem506.read(iprot);
-            struct.locks.add(_elem506);
+            _elem538 = new ShowLocksResponseElement();
+            _elem538.read(iprot);
+            struct.locks.add(_elem538);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index c8c762a..b4a0490 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@ -537,13 +537,13 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list388 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list388.size);
-                String _elem389;
-                for (int _i390 = 0; _i390 < _list388.size; ++_i390)
+                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)
                 {
-                  _elem389 = iprot.readString();
-                  struct.colNames.add(_elem389);
+                  _elem421 = iprot.readString();
+                  struct.colNames.add(_elem421);
                 }
                 iprot.readListEnd();
               }
@@ -579,9 +579,9 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
         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 _iter391 : struct.colNames)
+          for (String _iter423 : struct.colNames)
           {
-            oprot.writeString(_iter391);
+            oprot.writeString(_iter423);
           }
           oprot.writeListEnd();
         }
@@ -608,9 +608,9 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter392 : struct.colNames)
+        for (String _iter424 : struct.colNames)
         {
-          oprot.writeString(_iter392);
+          oprot.writeString(_iter424);
         }
       }
     }
@@ -623,13 +623,13 @@ public class TableStatsRequest implements org.apache.thrift.TBase<TableStatsRequ
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list393.size);
-        String _elem394;
-        for (int _i395 = 0; _i395 < _list393.size; ++_i395)
+        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)
         {
-          _elem394 = iprot.readString();
-          struct.colNames.add(_elem394);
+          _elem426 = iprot.readString();
+          struct.colNames.add(_elem426);
         }
       }
       struct.setColNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index 49da294..3a2c0fb 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@ -354,14 +354,14 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
           case 1: // TABLE_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
-                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list362.size);
-                ColumnStatisticsObj _elem363;
-                for (int _i364 = 0; _i364 < _list362.size; ++_i364)
+                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)
                 {
-                  _elem363 = new ColumnStatisticsObj();
-                  _elem363.read(iprot);
-                  struct.tableStats.add(_elem363);
+                  _elem395 = new ColumnStatisticsObj();
+                  _elem395.read(iprot);
+                  struct.tableStats.add(_elem395);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
         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 _iter365 : struct.tableStats)
+          for (ColumnStatisticsObj _iter397 : struct.tableStats)
           {
-            _iter365.write(oprot);
+            _iter397.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tableStats.size());
-        for (ColumnStatisticsObj _iter366 : struct.tableStats)
+        for (ColumnStatisticsObj _iter398 : struct.tableStats)
         {
-          _iter366.write(oprot);
+          _iter398.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class TableStatsResult implements org.apache.thrift.TBase<TableStatsResul
     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 _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list367.size);
-        ColumnStatisticsObj _elem368;
-        for (int _i369 = 0; _i369 < _list367.size; ++_i369)
+        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)
         {
-          _elem368 = new ColumnStatisticsObj();
-          _elem368.read(iprot);
-          struct.tableStats.add(_elem368);
+          _elem400 = new ColumnStatisticsObj();
+          _elem400.read(iprot);
+          struct.tableStats.add(_elem400);
         }
       }
       struct.setTableStatsIsSet(true);


[28/31] hive git commit: HIVE-15300: Reuse table information in SemanticAnalyzer::getMetaData to reduce compilation time (Rajesh Balamohan, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
HIVE-15300: Reuse table information in SemanticAnalyzer::getMetaData to reduce compilation time (Rajesh Balamohan, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: b18577426d93d9b73ad8f50dfcf428681765b57e
Parents: 696be9f
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu May 25 21:30:43 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Thu May 25 21:30:43 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java   | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b1857742/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 f2b0bd0..7f5051c 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
@@ -1988,7 +1988,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       String tabName = qb.getTabNameForAlias(alias);
       String cteName = tabName.toLowerCase();
 
-      Table tab = db.getTable(tabName, false);
+      // Get table details from tabNameToTabObject cache
+      Table tab = getTableObjectByName(tabName, false);
+      if (tab != null) {
+        // do a deep copy, in case downstream changes it.
+        tab = new Table(tab.getTTable().deepCopy());
+      }
       if (tab == null ||
               tab.getDbName().equals(SessionState.get().getCurrentDatabase())) {
         Table materializedTab = ctx.getMaterializedTable(cteName);
@@ -10924,16 +10929,22 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private Table getTableObjectByName(String tableName) throws HiveException {
+  private Table getTableObjectByName(String tableName, boolean throwException) throws HiveException {
     if (!tabNameToTabObject.containsKey(tableName)) {
-      Table table = db.getTable(tableName);
-      tabNameToTabObject.put(tableName, table);
+      Table table = db.getTable(tableName, throwException);
+      if (table != null) {
+        tabNameToTabObject.put(tableName, table);
+      }
       return table;
     } else {
       return tabNameToTabObject.get(tableName);
     }
   }
 
+  private Table getTableObjectByName(String tableName) throws HiveException {
+    return getTableObjectByName(tableName, true);
+  }
+
   private void walkASTMarkTABREF(ASTNode ast, Set<String> cteAlias)
       throws SemanticException {
     Queue<Node> queue = new LinkedList<>();


[13/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index b6306f0..a4937e2 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -156,12 +156,14 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
     """
     Parameters:
      - tbl
      - primaryKeys
      - foreignKeys
+     - uniqueConstraints
+     - notNullConstraints
     """
     pass
 
@@ -186,6 +188,20 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def add_unique_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
+  def add_not_null_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
   def drop_table(self, dbname, name, deleteData):
     """
     Parameters:
@@ -771,6 +787,20 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_unique_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
+  def get_not_null_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -1897,22 +1927,26 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o4
     return
 
-  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+  def create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
     """
     Parameters:
      - tbl
      - primaryKeys
      - foreignKeys
+     - uniqueConstraints
+     - notNullConstraints
     """
-    self.send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+    self.send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
     self.recv_create_table_with_constraints()
 
-  def send_create_table_with_constraints(self, tbl, primaryKeys, foreignKeys):
+  def send_create_table_with_constraints(self, tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints):
     self._oprot.writeMessageBegin('create_table_with_constraints', TMessageType.CALL, self._seqid)
     args = create_table_with_constraints_args()
     args.tbl = tbl
     args.primaryKeys = primaryKeys
     args.foreignKeys = foreignKeys
+    args.uniqueConstraints = uniqueConstraints
+    args.notNullConstraints = notNullConstraints
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
@@ -2037,6 +2071,72 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     return
 
+  def add_unique_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_add_unique_constraint(req)
+    self.recv_add_unique_constraint()
+
+  def send_add_unique_constraint(self, req):
+    self._oprot.writeMessageBegin('add_unique_constraint', TMessageType.CALL, self._seqid)
+    args = add_unique_constraint_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_unique_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_unique_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 add_not_null_constraint(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_add_not_null_constraint(req)
+    self.recv_add_not_null_constraint()
+
+  def send_add_not_null_constraint(self, req):
+    self._oprot.writeMessageBegin('add_not_null_constraint', TMessageType.CALL, self._seqid)
+    args = add_not_null_constraint_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_not_null_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_not_null_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:
@@ -4568,6 +4668,76 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_foreign_keys failed: unknown result")
 
+  def get_unique_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_unique_constraints(request)
+    return self.recv_get_unique_constraints()
+
+  def send_get_unique_constraints(self, request):
+    self._oprot.writeMessageBegin('get_unique_constraints', TMessageType.CALL, self._seqid)
+    args = get_unique_constraints_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_unique_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_unique_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_unique_constraints failed: unknown result")
+
+  def get_not_null_constraints(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_not_null_constraints(request)
+    return self.recv_get_not_null_constraints()
+
+  def send_get_not_null_constraints(self, request):
+    self._oprot.writeMessageBegin('get_not_null_constraints', TMessageType.CALL, self._seqid)
+    args = get_not_null_constraints_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_not_null_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_not_null_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_not_null_constraints failed: unknown result")
+
   def update_table_column_statistics(self, stats_obj):
     """
     Parameters:
@@ -6861,6 +7031,8 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["drop_constraint"] = Processor.process_drop_constraint
     self._processMap["add_primary_key"] = Processor.process_add_primary_key
     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["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
@@ -6926,6 +7098,8 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_index_names"] = Processor.process_get_index_names
     self._processMap["get_primary_keys"] = Processor.process_get_primary_keys
     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["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
@@ -7484,7 +7658,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)
+      self._handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints)
       msg_type = TMessageType.REPLY
     except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
       raise
@@ -7584,6 +7758,56 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_add_unique_constraint(self, seqid, iprot, oprot):
+    args = add_unique_constraint_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_unique_constraint_result()
+    try:
+      self._handler.add_unique_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_unique_constraint", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_add_not_null_constraint(self, seqid, iprot, oprot):
+    args = add_not_null_constraint_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_not_null_constraint_result()
+    try:
+      self._handler.add_not_null_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_not_null_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)
@@ -9242,6 +9466,56 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_unique_constraints(self, seqid, iprot, oprot):
+    args = get_unique_constraints_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_unique_constraints_result()
+    try:
+      result.success = self._handler.get_unique_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_unique_constraints", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_not_null_constraints(self, seqid, iprot, oprot):
+    args = get_not_null_constraints_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_not_null_constraints_result()
+    try:
+      result.success = self._handler.get_not_null_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_not_null_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)
@@ -11672,10 +11946,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = iprot.readString()
-            self.success.append(_elem599)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = iprot.readString()
+            self.success.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11698,8 +11972,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 iter600 in self.success:
-        oprot.writeString(iter600)
+      for iter628 in self.success:
+        oprot.writeString(iter628)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11804,10 +12078,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype604, _size601) = iprot.readListBegin()
-          for _i605 in xrange(_size601):
-            _elem606 = iprot.readString()
-            self.success.append(_elem606)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = iprot.readString()
+            self.success.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11830,8 +12104,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 iter607 in self.success:
-        oprot.writeString(iter607)
+      for iter635 in self.success:
+        oprot.writeString(iter635)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12601,12 +12875,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype609, _vtype610, _size608 ) = iprot.readMapBegin()
-          for _i612 in xrange(_size608):
-            _key613 = iprot.readString()
-            _val614 = Type()
-            _val614.read(iprot)
-            self.success[_key613] = _val614
+          (_ktype637, _vtype638, _size636 ) = iprot.readMapBegin()
+          for _i640 in xrange(_size636):
+            _key641 = iprot.readString()
+            _val642 = Type()
+            _val642.read(iprot)
+            self.success[_key641] = _val642
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -12629,9 +12903,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 kiter615,viter616 in self.success.items():
-        oprot.writeString(kiter615)
-        viter616.write(oprot)
+      for kiter643,viter644 in self.success.items():
+        oprot.writeString(kiter643)
+        viter644.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -12774,11 +13048,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype620, _size617) = iprot.readListBegin()
-          for _i621 in xrange(_size617):
-            _elem622 = FieldSchema()
-            _elem622.read(iprot)
-            self.success.append(_elem622)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = FieldSchema()
+            _elem650.read(iprot)
+            self.success.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12813,8 +13087,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 iter623 in self.success:
-        iter623.write(oprot)
+      for iter651 in self.success:
+        iter651.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12981,11 +13255,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype627, _size624) = iprot.readListBegin()
-          for _i628 in xrange(_size624):
-            _elem629 = FieldSchema()
-            _elem629.read(iprot)
-            self.success.append(_elem629)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = FieldSchema()
+            _elem657.read(iprot)
+            self.success.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13020,8 +13294,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 iter630 in self.success:
-        iter630.write(oprot)
+      for iter658 in self.success:
+        iter658.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13174,11 +13448,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype634, _size631) = iprot.readListBegin()
-          for _i635 in xrange(_size631):
-            _elem636 = FieldSchema()
-            _elem636.read(iprot)
-            self.success.append(_elem636)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = FieldSchema()
+            _elem664.read(iprot)
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13213,8 +13487,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 iter637 in self.success:
-        iter637.write(oprot)
+      for iter665 in self.success:
+        iter665.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13381,11 +13655,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = FieldSchema()
-            _elem643.read(iprot)
-            self.success.append(_elem643)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = FieldSchema()
+            _elem671.read(iprot)
+            self.success.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13420,8 +13694,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 iter644 in self.success:
-        iter644.write(oprot)
+      for iter672 in self.success:
+        iter672.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13830,6 +14104,8 @@ class create_table_with_constraints_args:
    - tbl
    - primaryKeys
    - foreignKeys
+   - uniqueConstraints
+   - notNullConstraints
   """
 
   thrift_spec = (
@@ -13837,12 +14113,16 @@ class create_table_with_constraints_args:
     (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1
     (2, TType.LIST, 'primaryKeys', (TType.STRUCT,(SQLPrimaryKey, SQLPrimaryKey.thrift_spec)), None, ), # 2
     (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
   )
 
-  def __init__(self, tbl=None, primaryKeys=None, foreignKeys=None,):
+  def __init__(self, tbl=None, primaryKeys=None, foreignKeys=None, uniqueConstraints=None, notNullConstraints=None,):
     self.tbl = tbl
     self.primaryKeys = primaryKeys
     self.foreignKeys = foreignKeys
+    self.uniqueConstraints = uniqueConstraints
+    self.notNullConstraints = notNullConstraints
 
   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:
@@ -13862,22 +14142,44 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype648, _size645) = iprot.readListBegin()
-          for _i649 in xrange(_size645):
-            _elem650 = SQLPrimaryKey()
-            _elem650.read(iprot)
-            self.primaryKeys.append(_elem650)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = SQLPrimaryKey()
+            _elem678.read(iprot)
+            self.primaryKeys.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype654, _size651) = iprot.readListBegin()
-          for _i655 in xrange(_size651):
-            _elem656 = SQLForeignKey()
-            _elem656.read(iprot)
-            self.foreignKeys.append(_elem656)
+          (_etype682, _size679) = iprot.readListBegin()
+          for _i683 in xrange(_size679):
+            _elem684 = SQLForeignKey()
+            _elem684.read(iprot)
+            self.foreignKeys.append(_elem684)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.uniqueConstraints = []
+          (_etype688, _size685) = iprot.readListBegin()
+          for _i689 in xrange(_size685):
+            _elem690 = SQLUniqueConstraint()
+            _elem690.read(iprot)
+            self.uniqueConstraints.append(_elem690)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.LIST:
+          self.notNullConstraints = []
+          (_etype694, _size691) = iprot.readListBegin()
+          for _i695 in xrange(_size691):
+            _elem696 = SQLNotNullConstraint()
+            _elem696.read(iprot)
+            self.notNullConstraints.append(_elem696)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13898,15 +14200,29 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter657 in self.primaryKeys:
-        iter657.write(oprot)
+      for iter697 in self.primaryKeys:
+        iter697.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 iter658 in self.foreignKeys:
-        iter658.write(oprot)
+      for iter698 in self.foreignKeys:
+        iter698.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 iter699 in self.uniqueConstraints:
+        iter699.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 iter700 in self.notNullConstraints:
+        iter700.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13921,6 +14237,8 @@ class create_table_with_constraints_args:
     value = (value * 31) ^ hash(self.tbl)
     value = (value * 31) ^ hash(self.primaryKeys)
     value = (value * 31) ^ hash(self.foreignKeys)
+    value = (value * 31) ^ hash(self.uniqueConstraints)
+    value = (value * 31) ^ hash(self.notNullConstraints)
     return value
 
   def __repr__(self):
@@ -14480,6 +14798,298 @@ class add_foreign_key_result:
   def __ne__(self, other):
     return not (self == other)
 
+class add_unique_constraint_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (AddUniqueConstraintRequest, AddUniqueConstraintRequest.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 = AddUniqueConstraintRequest()
+          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_unique_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_unique_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_unique_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 add_not_null_constraint_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (AddNotNullConstraintRequest, AddNotNullConstraintRequest.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 = AddNotNullConstraintRequest()
+          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_not_null_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_not_null_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_not_null_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:
@@ -14878,10 +15488,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = iprot.readString()
-            self.partNames.append(_elem664)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = iprot.readString()
+            self.partNames.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14906,8 +15516,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 iter665 in self.partNames:
-        oprot.writeString(iter665)
+      for iter707 in self.partNames:
+        oprot.writeString(iter707)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15107,10 +15717,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype669, _size666) = iprot.readListBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = iprot.readString()
-            self.success.append(_elem671)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.success.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15133,8 +15743,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 iter672 in self.success:
-        oprot.writeString(iter672)
+      for iter714 in self.success:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15284,10 +15894,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = iprot.readString()
-            self.success.append(_elem678)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString()
+            self.success.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15310,8 +15920,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 iter679 in self.success:
-        oprot.writeString(iter679)
+      for iter721 in self.success:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15384,10 +15994,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype683, _size680) = iprot.readListBegin()
-          for _i684 in xrange(_size680):
-            _elem685 = iprot.readString()
-            self.tbl_types.append(_elem685)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString()
+            self.tbl_types.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15412,8 +16022,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 iter686 in self.tbl_types:
-        oprot.writeString(iter686)
+      for iter728 in self.tbl_types:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15469,11 +16079,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype690, _size687) = iprot.readListBegin()
-          for _i691 in xrange(_size687):
-            _elem692 = TableMeta()
-            _elem692.read(iprot)
-            self.success.append(_elem692)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = TableMeta()
+            _elem734.read(iprot)
+            self.success.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15496,8 +16106,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 iter693 in self.success:
-        iter693.write(oprot)
+      for iter735 in self.success:
+        iter735.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15621,10 +16231,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype697, _size694) = iprot.readListBegin()
-          for _i698 in xrange(_size694):
-            _elem699 = iprot.readString()
-            self.success.append(_elem699)
+          (_etype739, _size736) = iprot.readListBegin()
+          for _i740 in xrange(_size736):
+            _elem741 = iprot.readString()
+            self.success.append(_elem741)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15647,8 +16257,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 iter700 in self.success:
-        oprot.writeString(iter700)
+      for iter742 in self.success:
+        oprot.writeString(iter742)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15884,10 +16494,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = iprot.readString()
-            self.tbl_names.append(_elem706)
+          (_etype746, _size743) = iprot.readListBegin()
+          for _i747 in xrange(_size743):
+            _elem748 = iprot.readString()
+            self.tbl_names.append(_elem748)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15908,8 +16518,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 iter707 in self.tbl_names:
-        oprot.writeString(iter707)
+      for iter749 in self.tbl_names:
+        oprot.writeString(iter749)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15961,11 +16571,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = Table()
-            _elem713.read(iprot)
-            self.success.append(_elem713)
+          (_etype753, _size750) = iprot.readListBegin()
+          for _i754 in xrange(_size750):
+            _elem755 = Table()
+            _elem755.read(iprot)
+            self.success.append(_elem755)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15982,8 +16592,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 iter714 in self.success:
-        iter714.write(oprot)
+      for iter756 in self.success:
+        iter756.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16466,10 +17076,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = iprot.readString()
-            self.success.append(_elem720)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = iprot.readString()
+            self.success.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16504,8 +17114,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 iter721 in self.success:
-        oprot.writeString(iter721)
+      for iter763 in self.success:
+        oprot.writeString(iter763)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17475,11 +18085,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = Partition()
-            _elem727.read(iprot)
-            self.new_parts.append(_elem727)
+          (_etype767, _size764) = iprot.readListBegin()
+          for _i768 in xrange(_size764):
+            _elem769 = Partition()
+            _elem769.read(iprot)
+            self.new_parts.append(_elem769)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17496,8 +18106,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 iter728 in self.new_parts:
-        iter728.write(oprot)
+      for iter770 in self.new_parts:
+        iter770.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17655,11 +18265,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype732, _size729) = iprot.readListBegin()
-          for _i733 in xrange(_size729):
-            _elem734 = PartitionSpec()
-            _elem734.read(iprot)
-            self.new_parts.append(_elem734)
+          (_etype774, _size771) = iprot.readListBegin()
+          for _i775 in xrange(_size771):
+            _elem776 = PartitionSpec()
+            _elem776.read(iprot)
+            self.new_parts.append(_elem776)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17676,8 +18286,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 iter735 in self.new_parts:
-        iter735.write(oprot)
+      for iter777 in self.new_parts:
+        iter777.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17851,10 +18461,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype739, _size736) = iprot.readListBegin()
-          for _i740 in xrange(_size736):
-            _elem741 = iprot.readString()
-            self.part_vals.append(_elem741)
+          (_etype781, _size778) = iprot.readListBegin()
+          for _i782 in xrange(_size778):
+            _elem783 = iprot.readString()
+            self.part_vals.append(_elem783)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17879,8 +18489,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 iter742 in self.part_vals:
-        oprot.writeString(iter742)
+      for iter784 in self.part_vals:
+        oprot.writeString(iter784)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18233,10 +18843,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype746, _size743) = iprot.readListBegin()
-          for _i747 in xrange(_size743):
-            _elem748 = iprot.readString()
-            self.part_vals.append(_elem748)
+          (_etype788, _size785) = iprot.readListBegin()
+          for _i789 in xrange(_size785):
+            _elem790 = iprot.readString()
+            self.part_vals.append(_elem790)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18267,8 +18877,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 iter749 in self.part_vals:
-        oprot.writeString(iter749)
+      for iter791 in self.part_vals:
+        oprot.writeString(iter791)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -18863,10 +19473,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype753, _size750) = iprot.readListBegin()
-          for _i754 in xrange(_size750):
-            _elem755 = iprot.readString()
-            self.part_vals.append(_elem755)
+          (_etype795, _size792) = iprot.readListBegin()
+          for _i796 in xrange(_size792):
+            _elem797 = iprot.readString()
+            self.part_vals.append(_elem797)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18896,8 +19506,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 iter756 in self.part_vals:
-        oprot.writeString(iter756)
+      for iter798 in self.part_vals:
+        oprot.writeString(iter798)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -19070,10 +19680,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype760, _size757) = iprot.readListBegin()
-          for _i761 in xrange(_size757):
-            _elem762 = iprot.readString()
-            self.part_vals.append(_elem762)
+          (_etype802, _size799) = iprot.readListBegin()
+          for _i803 in xrange(_size799):
+            _elem804 = iprot.readString()
+            self.part_vals.append(_elem804)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19109,8 +19719,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 iter763 in self.part_vals:
-        oprot.writeString(iter763)
+      for iter805 in self.part_vals:
+        oprot.writeString(iter805)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -19847,10 +20457,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype767, _size764) = iprot.readListBegin()
-          for _i768 in xrange(_size764):
-            _elem769 = iprot.readString()
-            self.part_vals.append(_elem769)
+          (_etype809, _size806) = iprot.readListBegin()
+          for _i810 in xrange(_size806):
+            _elem811 = iprot.readString()
+            self.part_vals.append(_elem811)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19875,8 +20485,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 iter770 in self.part_vals:
-        oprot.writeString(iter770)
+      for iter812 in self.part_vals:
+        oprot.writeString(iter812)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20035,11 +20645,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype772, _vtype773, _size771 ) = iprot.readMapBegin()
-          for _i775 in xrange(_size771):
-            _key776 = iprot.readString()
-            _val777 = iprot.readString()
-            self.partitionSpecs[_key776] = _val777
+          (_ktype814, _vtype815, _size813 ) = iprot.readMapBegin()
+          for _i817 in xrange(_size813):
+            _key818 = iprot.readString()
+            _val819 = iprot.readString()
+            self.partitionSpecs[_key818] = _val819
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -20076,9 +20686,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 kiter778,viter779 in self.partitionSpecs.items():
-        oprot.writeString(kiter778)
-        oprot.writeString(viter779)
+      for kiter820,viter821 in self.partitionSpecs.items():
+        oprot.writeString(kiter820)
+        oprot.writeString(viter821)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -20283,11 +20893,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype781, _vtype782, _size780 ) = iprot.readMapBegin()
-          for _i784 in xrange(_size780):
-            _key785 = iprot.readString()
-            _val786 = iprot.readString()
-            self.partitionSpecs[_key785] = _val786
+          (_ktype823, _vtype824, _size822 ) = iprot.readMapBegin()
+          for _i826 in xrange(_size822):
+            _key827 = iprot.readString()
+            _val828 = iprot.readString()
+            self.partitionSpecs[_key827] = _val828
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -20324,9 +20934,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 kiter787,viter788 in self.partitionSpecs.items():
-        oprot.writeString(kiter787)
-        oprot.writeString(viter788)
+      for kiter829,viter830 in self.partitionSpecs.items():
+        oprot.writeString(kiter829)
+        oprot.writeString(viter830)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -20409,11 +21019,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype792, _size789) = iprot.readListBegin()
-          for _i793 in xrange(_size789):
-            _elem794 = Partition()
-            _elem794.read(iprot)
-            self.success.append(_elem794)
+          (_etype834, _size831) = iprot.readListBegin()
+          for _i835 in xrange(_size831):
+            _elem836 = Partition()
+            _elem836.read(iprot)
+            self.success.append(_elem836)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20454,8 +21064,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 iter795 in self.success:
-        iter795.write(oprot)
+      for iter837 in self.success:
+        iter837.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20549,10 +21159,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype799, _size796) = iprot.readListBegin()
-          for _i800 in xrange(_size796):
-            _elem801 = iprot.readString()
-            self.part_vals.append(_elem801)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = iprot.readString()
+            self.part_vals.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20564,10 +21174,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype805, _size802) = iprot.readListBegin()
-          for _i806 in xrange(_size802):
-            _elem807 = iprot.readString()
-            self.group_names.append(_elem807)
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = iprot.readString()
+            self.group_names.append(_elem849)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20592,8 +21202,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 iter808 in self.part_vals:
-        oprot.writeString(iter808)
+      for iter850 in self.part_vals:
+        oprot.writeString(iter850)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -20603,8 +21213,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 iter809 in self.group_names:
-        oprot.writeString(iter809)
+      for iter851 in self.group_names:
+        oprot.writeString(iter851)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21033,11 +21643,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Partition()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = Partition()
+            _elem857.read(iprot)
+            self.success.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21066,8 +21676,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 iter816 in self.success:
-        iter816.write(oprot)
+      for iter858 in self.success:
+        iter858.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21161,10 +21771,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = iprot.readString()
-            self.group_names.append(_elem822)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = iprot.readString()
+            self.group_names.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21197,8 +21807,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 iter823 in self.group_names:
-        oprot.writeString(iter823)
+      for iter865 in self.group_names:
+        oprot.writeString(iter865)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21259,11 +21869,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype827, _size824) = iprot.readListBegin()
-          for _i828 in xrange(_size824):
-            _elem829 = Partition()
-            _elem829.read(iprot)
-            self.success.append(_elem829)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = Partition()
+            _elem871.read(iprot)
+            self.success.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21292,8 +21902,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 iter830 in self.success:
-        iter830.write(oprot)
+      for iter872 in self.success:
+        iter872.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21451,11 +22061,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = PartitionSpec()
-            _elem836.read(iprot)
-            self.success.append(_elem836)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = PartitionSpec()
+            _elem878.read(iprot)
+            self.success.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21484,8 +22094,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 iter837 in self.success:
-        iter837.write(oprot)
+      for iter879 in self.success:
+        iter879.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21640,10 +22250,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = iprot.readString()
-            self.success.append(_elem843)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = iprot.readString()
+            self.success.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21666,8 +22276,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 iter844 in self.success:
-        oprot.writeString(iter844)
+      for iter886 in self.success:
+        oprot.writeString(iter886)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -21743,10 +22353,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.part_vals.append(_elem850)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = iprot.readString()
+            self.part_vals.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21776,8 +22386,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 iter851 in self.part_vals:
-        oprot.writeString(iter851)
+      for iter893 in self.part_vals:
+        oprot.writeString(iter893)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21841,11 +22451,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = Partition()
-            _elem857.read(iprot)
-            self.success.append(_elem857)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = Partition()
+            _elem899.read(iprot)
+            self.success.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21874,8 +22484,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 iter858 in self.success:
-        iter858.write(oprot)
+      for iter900 in self.success:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21962,10 +22572,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = iprot.readString()
-            self.part_vals.append(_elem864)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.part_vals.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21982,10 +22592,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype868, _size865) = iprot.readListBegin()
-          for _i869 in xrange(_size865):
-            _elem870 = iprot.readString()
-            self.group_names.append(_elem870)
+          (_etype910, _size907) = iprot.readListBegin()
+          for _i911 in xrange(_size907):
+            _elem912 = iprot.readString()
+            self.group_names.append(_elem912)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22010,8 +22620,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 iter871 in self.part_vals:
-        oprot.writeString(iter871)
+      for iter913 in self.part_vals:
+        oprot.writeString(iter913)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -22025,8 +22635,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 iter872 in self.group_names:
-        oprot.writeString(iter872)
+      for iter914 in self.group_names:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22088,11 +22698,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = Partition()
-            _elem878.read(iprot)
-            self.success.append(_elem878)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = Partition()
+            _elem920.read(iprot)
+            self.success.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22121,8 +22731,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 iter879 in self.success:
-        iter879.write(oprot)
+      for iter921 in self.success:
+        iter921.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22203,10 +22813,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = iprot.readString()
-            self.part_vals.append(_elem885)
+          (_etype925, _size922) = iprot.readListBegin()
+          for _i926 in xrange(_size922):
+            _elem927 = iprot.readString()
+            self.part_vals.append(_elem927)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22236,8 +22846,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 iter886 in self.part_vals:
-        oprot.writeString(iter886)
+      for iter928 in self.part_vals:
+        oprot.writeString(iter928)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -22301,10 +22911,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = iprot.readString()
-            self.success.append(_elem892)
+          (_etype932, _size929) = iprot.readListBegin()
+          for _i933 in xrange(_size929):
+            _elem934 = iprot.readString()
+            self.success.append(_elem934)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22333,8 +22943,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 iter893 in self.success:
-        oprot.writeString(iter893)
+      for iter935 in self.success:
+        oprot.writeString(iter935)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22505,11 +23115,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = Partition()
-            _elem899.read(iprot)
-            self.success.append(_elem899)
+          (_etype939, _size936) = iprot.readListBegin()
+          for _i940 in xrange(_size936):
+            _elem941 = Partition()
+            _elem941.read(iprot)
+            self.success.append(_elem941)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22538,8 +23148,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 iter900 in self.success:
-        iter900.write(oprot)
+      for iter942 in self.success:
+        iter942.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22710,11 +23320,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = PartitionSpec()
-            _elem906.read(iprot)
-            self.success.append(_elem906)
+          (_etype946, _size943) = iprot.readListBegin()
+          for _i947 in xrange(_size943):
+            _elem948 = PartitionSpec()
+            _elem948.read(iprot)
+            self.success.append(_elem948)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22743,8 +23353,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 iter907 in self.success:
-        iter907.write(oprot)
+      for iter949 in self.success:
+        iter949.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23164,10 +23774,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.names.append(_elem913)
+          (_etype953, _size950) = iprot.readListBegin()
+          for _i954 in xrange(_size950):
+            _elem955 = iprot.readString()
+            self.names.append(_elem955)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23192,8 +23802,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 iter914 in self.names:
-        oprot.writeString(iter914)
+      for iter956 in self.names:
+        oprot.writeString(iter956)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23252,11 +23862,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype918, _size915) = iprot.readListBegin()
-          for _i919 in xrange(_size915):
-            _elem920 = Partition()
-            _elem920.read(iprot)
-            self.success.append(_elem920)
+          (_etype960, _size957) = iprot.readListBegin()
+          for _i961 in xrange(_size957):
+            _elem962 = Partition()
+            _elem962.read(iprot)
+            self.success.append(_elem962)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23285,8 +23895,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 iter921 in self.success:
-        iter921.write(oprot)
+      for iter963 in self.success:
+        iter963.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23536,11 +24146,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype925, _size922) = iprot.readListBegin()
-          for _i926 in xrange(_size922):
-            _elem927 = Partition()
-            _elem927.read(iprot)
-            self.new_parts.append(_elem927)
+          (_etype967, _size964) = iprot.readListBegin()
+          for _i968 in xrange(_size964):
+            _elem969 = Partition()
+            _elem969.read(iprot)
+            self.new_parts.append(_elem969)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23565,8 +24175,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 iter928 in self.new_parts:
-        iter928.write(oprot)
+      for iter970 in self.new_parts:
+        iter970.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23719,11 +24329,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype932, _size929) = iprot.readListBegin()
-          for _i933 in xrange(_size929):
-            _elem934 = Partition()
-            _elem934.read(iprot)
-            self.new_parts.append(_elem934)
+          (_etype974, _size971) = iprot.readListBegin()
+          for _i975 in xrange(_size971):
+            _elem976 = Partition()
+            _elem976.read(iprot)
+            self.new_parts.append(_elem976)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23754,8 +24364,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 iter935 in self.new_parts:
-        iter935.write(oprot)
+      for iter977 in self.new_parts:
+        iter977.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -24099,10 +24709,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype939, _size936) = iprot.readListBegin()
-          for _i940 in xrange(_size936):
-            _elem941 = iprot.readString()
-            self.part_vals.append(_elem941)
+          (_etype981, _size978) = iprot.readListBegin()
+          for _i982 in xrange(_size978):
+            _elem983 = iprot.readString()
+            self.part_vals.append(_elem983)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24133,8 +24743,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 iter942 in self.part_vals:
-        oprot.writeString(iter942)
+      for iter984 in self.part_vals:
+        oprot.writeString(iter984)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -24276,10 +24886,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype946, _size943) = iprot.readListBegin()
-          for _i947 in xrange(_size943):
-            _elem948 = iprot.readString()
-            self.part_vals.append(_elem948)
+          (_etype988, _size985) = iprot.readListBegin()
+          for _i989 in xrange(_size985):
+            _elem990 = iprot.readString()
+            self.part_vals.append(_elem990)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24301,8 +24911,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 iter949 in self.part_vals:
-        oprot.writeString(iter949)
+      for iter991 in self.part_vals:
+        oprot.writeString(iter991)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -24660,10 +25270,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype953, _size950) = iprot.readListBegin()
-          for _i954 in xrange(_size950):
-            _elem955 = iprot.readString()
-            self.success.append(_elem955)
+          (_etype995, _size992) = iprot.readListBegin()
+          for _i996 in xrange(_size992):
+            _elem997 = iprot.readString()
+            self.success.append(_elem997)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24686,8 +25296,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 iter956 in self.success:
-        oprot.writeString(iter956)
+      for iter998 in self.success:
+        oprot.writeString(iter998)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24811,11 +25421,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype958, _vtype959, _size957 ) = iprot.readMapBegin()
-          for _i961 in xrange(_size957):
-            _key962 = iprot.readString()
-            _val963 = iprot.readString()
-            self.success[_key962] = _val963
+          (_ktype1000, _vtype1001, _size999 ) = iprot.readMapBegin()
+          for _i1003 in xrange(_size999):
+            _key1004 = iprot.readString()
+            _val1005 = iprot.readString()
+            self.success[_key1004] = _val1005
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24838,9 +25448,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 kiter964,viter965 in self.success.items():
-        oprot.writeString(kiter964)
-        oprot.writeString(viter965)
+      for kiter1006,viter1007 in self.success.items():
+        oprot.writeString(kiter1006)
+        oprot.writeString(viter1007)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24916,11 +25526,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype967, _vtype968, _size966 ) = iprot.readMapBegin()
-          for _i970 in xrange(_size966):
-            _key971 = iprot.readString()
-            _val972 = iprot.readString()
-            self.part_vals[_key971] = _val972
+          (_ktype1009, _vtype1010, _size1008 ) = iprot.readMapBegin()
+          for _i1012 in xrange(_size1008):
+            _key1013 = iprot.readString()
+            _val1014 = iprot.readString()
+            self.part_vals[_key1013] = _val1014
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24950,9 +25560,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 kiter973,viter974 in self.part_vals.items():
-        oprot.writeString(kiter973)
-        oprot.writeString(viter974)
+      for kiter1015,viter1016 in self.part_vals.items():
+        oprot.writeString(kiter1015)
+        oprot.writeString(viter1016)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25166,11 +25776,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype976, _vtype977, _size975 ) = iprot.readMapBegin()
-          for _i979 in xrange(_size975):
-            _key980 = iprot.readString()
-            _val981 = iprot.readString()
-            self.part_vals[_key980] = _val981
+          (_ktype1018, _vtype1019, _size1017 ) = iprot.readMapBegin()
+          for _i1021 in xrange(_size1017):
+            _key1022 = iprot.readString()
+            _val1023 = iprot.readString()
+            self.part_vals[_key1022] = _val1023
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25200,9 +25810,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 kiter982,viter983 in self.part_vals.items():
-        oprot.writeString(kiter982)
-        oprot.writeString(viter983)
+      for kiter1024,viter1025 in self.part_vals.items():
+        oprot.writeString(kiter1024)
+        oprot.writeString(viter1025)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25991,8 +26601,384 @@ class get_index_by_name_args:
         else:
           iprot.skip(ftype)
       elif fid == 3:
-        if ftype == TType.STRING:
-          self.index_name = iprot.readString()
+        if ftype == TType.STRING:
+          self.index_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('get_index_by_name_args')
+    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()
+    if self.index_name is not None:
+      oprot.writeFieldBegin('index_name', TType.STRING, 3)
+      oprot.writeString(self.index_name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
+    value = (value * 31) ^ hash(self.index_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 get_index_by_name_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (Index, Index.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 = Index()
+          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_index_by_name_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 

<TRUNCATED>

[08/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientpositive/create_with_constraints.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_with_constraints.q.out b/ql/src/test/results/clientpositive/create_with_constraints.q.out
index 64c3ec6..59b86b3 100644
--- a/ql/src/test/results/clientpositive/create_with_constraints.q.out
+++ b/ql/src/test/results/clientpositive/create_with_constraints.q.out
@@ -1,97 +1,153 @@
-PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, PRIMARY KEY (a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table1
-POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table1 (a STRING, b STRING, PRIMARY KEY (a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
-PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+PREHOOK: query: CREATE TABLE table2 (a STRING, b STRING, CONSTRAINT pk1 PRIMARY KEY (a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table2
-POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, constraint pk1 primary key (a) disable novalidate)
+POSTHOOK: query: CREATE TABLE table2 (a STRING, b STRING, CONSTRAINT pk1 PRIMARY KEY (a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table2
-PREHOOK: query: CREATE TABLE table3 (x string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE)
+PREHOOK: query: CREATE TABLE table3 (x string NOT NULL DISABLE, PRIMARY KEY (x) DISABLE, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table3
-POSTHOOK: query: CREATE TABLE table3 (x string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE)
+POSTHOOK: query: CREATE TABLE table3 (x string NOT NULL DISABLE, PRIMARY KEY (x) DISABLE, CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table3
-PREHOOK: query: CREATE TABLE table4 (x string, y string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE, 
-CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a)  DISABLE NOVALIDATE)
+PREHOOK: query: CREATE TABLE table4 (x string CONSTRAINT nn4_1 NOT NULL DISABLE, y string CONSTRAINT nn4_2 NOT NULL DISABLE, UNIQUE (x) DISABLE, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a) DISABLE, 
+CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table4
-POSTHOOK: query: CREATE TABLE table4 (x string, y string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE, 
-CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a)  DISABLE NOVALIDATE)
+POSTHOOK: query: CREATE TABLE table4 (x string CONSTRAINT nn4_1 NOT NULL DISABLE, y string CONSTRAINT nn4_2 NOT NULL DISABLE, UNIQUE (x) DISABLE, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(a) DISABLE, 
+CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table4
-PREHOOK: query: CREATE TABLE table5 (x string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE)
+PREHOOK: query: CREATE TABLE table5 (x string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table5
-POSTHOOK: query: CREATE TABLE table5 (x string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE)
+POSTHOOK: query: CREATE TABLE table5 (x string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table5
-PREHOOK: query: CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE,
-CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE)
+PREHOOK: query: CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE,
+CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table6
-POSTHOOK: query: CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE,
-CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE)
+POSTHOOK: query: CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) DISABLE, FOREIGN KEY (x) REFERENCES table2(a) DISABLE,
+CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table6
-PREHOOK: query: CREATE TABLE table7 (a STRING, b STRING, primary key (a) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table7 (a STRING, b STRING, PRIMARY KEY (a) DISABLE RELY)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table7
-POSTHOOK: query: CREATE TABLE table7 (a STRING, b STRING, primary key (a) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table7 (a STRING, b STRING, PRIMARY KEY (a) DISABLE RELY)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table7
-PREHOOK: query: CREATE TABLE table8 (a STRING, b STRING, constraint pk8 primary key (a) disable novalidate norely)
+PREHOOK: query: CREATE TABLE table8 (a STRING, b STRING, CONSTRAINT pk8 PRIMARY KEY (a) DISABLE NORELY)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table8
-POSTHOOK: query: CREATE TABLE table8 (a STRING, b STRING, constraint pk8 primary key (a) disable novalidate norely)
+POSTHOOK: query: CREATE TABLE table8 (a STRING, b STRING, CONSTRAINT pk8 PRIMARY KEY (a) DISABLE NORELY)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table8
-PREHOOK: query: CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely)
+PREHOOK: query: CREATE TABLE table9 (a STRING, b STRING, PRIMARY KEY (a, b) DISABLE RELY)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table9
-POSTHOOK: query: CREATE TABLE table9 (a STRING, b STRING, primary key (a, b) disable novalidate rely)
+POSTHOOK: query: CREATE TABLE table9 (a STRING, b STRING, PRIMARY KEY (a, b) DISABLE RELY)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table9
-PREHOOK: query: CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate)
+PREHOOK: query: CREATE TABLE table10 (a STRING, b STRING, CONSTRAINT pk10 PRIMARY KEY (a) DISABLE NORELY, FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table10
-POSTHOOK: query: CREATE TABLE table10 (a STRING, b STRING, constraint pk10 primary key (a) disable novalidate norely, foreign key (a, b) references table9(a, b) disable novalidate)
+POSTHOOK: query: CREATE TABLE table10 (a STRING, b STRING, CONSTRAINT pk10 PRIMARY KEY (a) DISABLE NORELY, FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table10
-PREHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, constraint fk11_1 foreign key (a, b) references table9(a, b) disable novalidate,
-constraint fk11_2 foreign key (c) references table4(x) disable novalidate)
+PREHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, CONSTRAINT pk11 PRIMARY KEY (a) DISABLE RELY, CONSTRAINT fk11_1 FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE,
+CONSTRAINT fk11_2 FOREIGN KEY (c) REFERENCES table4(x) DISABLE)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@table11
-POSTHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, constraint pk11 primary key (a) disable novalidate rely, constraint fk11_1 foreign key (a, b) references table9(a, b) disable novalidate,
-constraint fk11_2 foreign key (c) references table4(x) disable novalidate)
+POSTHOOK: query: CREATE TABLE table11 (a STRING, b STRING, c STRING, CONSTRAINT pk11 PRIMARY KEY (a) DISABLE RELY, CONSTRAINT fk11_1 FOREIGN KEY (a, b) REFERENCES table9(a, b) DISABLE,
+CONSTRAINT fk11_2 FOREIGN KEY (c) REFERENCES table4(x) DISABLE)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table11
+PREHOOK: query: CREATE TABLE table12 (a STRING CONSTRAINT nn12_1 NOT NULL DISABLE NORELY, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table12
+POSTHOOK: query: CREATE TABLE table12 (a STRING CONSTRAINT nn12_1 NOT NULL DISABLE NORELY, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table12
+PREHOOK: query: CREATE TABLE table13 (a STRING NOT NULL DISABLE RELY, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table13
+POSTHOOK: query: CREATE TABLE table13 (a STRING NOT NULL DISABLE RELY, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table13
+PREHOOK: query: CREATE TABLE table14 (a STRING CONSTRAINT nn14_1 NOT NULL DISABLE RELY, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table14
+POSTHOOK: query: CREATE TABLE table14 (a STRING CONSTRAINT nn14_1 NOT NULL DISABLE RELY, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table14
+PREHOOK: query: CREATE TABLE table15 (a STRING REFERENCES table4(x) DISABLE, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table15
+POSTHOOK: query: CREATE TABLE table15 (a STRING REFERENCES table4(x) DISABLE, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table15
+PREHOOK: query: CREATE TABLE table16 (a STRING CONSTRAINT nn16_1 REFERENCES table4(x) DISABLE RELY, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table16
+POSTHOOK: query: CREATE TABLE table16 (a STRING CONSTRAINT nn16_1 REFERENCES table4(x) DISABLE RELY, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table16
+PREHOOK: query: CREATE TABLE table17 (a STRING CONSTRAINT uk17_1 UNIQUE DISABLE RELY, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table17
+POSTHOOK: query: CREATE TABLE table17 (a STRING CONSTRAINT uk17_1 UNIQUE DISABLE RELY, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table17
+PREHOOK: query: CREATE TABLE table18 (a STRING, b STRING, CONSTRAINT uk18_1 UNIQUE (b) DISABLE RELY)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table18
+POSTHOOK: query: CREATE TABLE table18 (a STRING, b STRING, CONSTRAINT uk18_1 UNIQUE (b) DISABLE RELY)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table18
 PREHOOK: query: DESCRIBE EXTENDED table1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table1
@@ -125,6 +181,7 @@ x                   	string
 #### A masked pattern was here ####
 Constraints	Primary Key for default.table3:[x], Constraint Name: #### A masked pattern was here ####	 
 Foreign Keys for default.table3:[ {Constraint Name: fk1, (Parent Column Name: default.table2.a, Column Name: x, Key Sequence: 1)}]	 	 
+Not Null Constraints for default.table3:[ {Constraint Name: #### A masked pattern was here ####, Column Name: x}]	 	 
 PREHOOK: query: DESCRIBE EXTENDED table4
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table4
@@ -135,8 +192,9 @@ x                   	string
 y                   	string              	                    
 	 	 
 #### A masked pattern was here ####
-Constraints	Primary Key for default.table4:[x], Constraint Name: #### A masked pattern was here ####	 
-Foreign Keys for default.table4:[ {Constraint Name: fk2, (Parent Column Name: default.table2.a, Column Name: x, Key Sequence: 1)}, {Constraint Name: fk3, (Parent Column Name: default.table2.a, Column Name: y, Key Sequence: 1)}]	 	 
+Constraints	Foreign Keys for default.table4:[ {Constraint Name: fk2, (Parent Column Name: default.table2.a, Column Name: x, Key Sequence: 1)}, {Constraint Name: fk3, (Parent Column Name: default.table2.a, Column Name: y, Key Sequence: 1)}]	 
+Unique Constraints for default.table4:[ {Constraint Name: #### A masked pattern was here ####, (Column Name: x, Key Sequence: 1)}]	 	 
+Not Null Constraints for default.table4:[ {Constraint Name: nn4_1, Column Name: x}, {Constraint Name: nn4_2, Column Name: y}]	 	 
 PREHOOK: query: DESCRIBE EXTENDED table5
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table5
@@ -217,7 +275,82 @@ c                   	string
 	 	 
 #### A masked pattern was here ####
 Constraints	Primary Key for default.table11:[a], Constraint Name: pk11	 
-Foreign Keys for default.table11:[ {Constraint Name: fk11_1, (Parent Column Name: default.table9.a, Column Name: a, Key Sequence: 1), (Parent Column Name: default.table9.b, Column Name: b, Key Sequence: 2)}, {Constraint Name: fk11_2, (Parent Column Name: default.table4.x, Column Name: c, Key Sequence: 1)}]	 	 
+Foreign Keys for default.table11:[ {Constraint Name: fk11_1, (Parent Column Name: default.table9.a, Column Name: a, Key Sequence: 1), (Parent Column Name: default.table9.b, Column Name: b, Key Sequence: 2)}]	 	 
+PREHOOK: query: DESCRIBE EXTENDED table12
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table12
+POSTHOOK: query: DESCRIBE EXTENDED table12
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table12
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Not Null Constraints for default.table12:[ {Constraint Name: nn12_1, Column Name: a}]	 
+PREHOOK: query: DESCRIBE EXTENDED table13
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table13
+POSTHOOK: query: DESCRIBE EXTENDED table13
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table13
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Not Null Constraints for default.table13:[ {Constraint Name: #### A masked pattern was here ####, Column Name: a}]	 
+PREHOOK: query: DESCRIBE EXTENDED table14
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table14
+POSTHOOK: query: DESCRIBE EXTENDED table14
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table14
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Not Null Constraints for default.table14:[ {Constraint Name: nn14_1, Column Name: a}]	 
+PREHOOK: query: DESCRIBE EXTENDED table15
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table15
+POSTHOOK: query: DESCRIBE EXTENDED table15
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table15
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: DESCRIBE EXTENDED table16
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table16
+POSTHOOK: query: DESCRIBE EXTENDED table16
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table16
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: DESCRIBE EXTENDED table17
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table17
+POSTHOOK: query: DESCRIBE EXTENDED table17
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table17
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Unique Constraints for default.table17:[ {Constraint Name: uk17_1, (Column Name: a, Key Sequence: 1)}]	 
+PREHOOK: query: DESCRIBE EXTENDED table18
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table18
+POSTHOOK: query: DESCRIBE EXTENDED table18
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table18
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Unique Constraints for default.table18:[ {Constraint Name: uk18_1, (Column Name: b, Key Sequence: 1)}]	 
 PREHOOK: query: DESCRIBE FORMATTED table1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table1
@@ -349,6 +482,12 @@ Table:              	default.table3
 Constraint Name:    	fk1                 	 
 Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1      
 	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table3      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	x                   	 
+	 	 
 PREHOOK: query: DESCRIBE FORMATTED table4
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table4
@@ -387,11 +526,6 @@ Storage Desc Params:
 	 	 
 # Constraints	 	 
 	 	 
-# Primary Key	 	 
-Table:              	default.table4      	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	 
-	 	 
 # Foreign Keys	 	 
 Table:              	default.table4      	 
 Constraint Name:    	fk2                 	 
@@ -400,6 +534,21 @@ Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1
 Constraint Name:    	fk3                 	 
 Parent Column Name:default.table2.a	Column Name:y       	Key Sequence:1      
 	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.table4      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:x       	Key Sequence:1      	 
+	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table4      	 
+Constraint Name:    	nn4_1               	 
+Column Name:        	x                   	 
+	 	 
+Constraint Name:    	nn4_2               	 
+Column Name:        	y                   	 
+	 	 
 PREHOOK: query: DESCRIBE FORMATTED table5
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table5
@@ -723,58 +872,55 @@ Constraint Name:    	fk11_1
 Parent Column Name:default.table9.a	Column Name:a       	Key Sequence:1      
 Parent Column Name:default.table9.b	Column Name:b       	Key Sequence:2      
 	 	 
-Constraint Name:    	fk11_2              	 
-Parent Column Name:default.table4.x	Column Name:c       	Key Sequence:1      
-	 	 
-PREHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
-PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
-POSTHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
-POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
-PREHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
-PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
-POSTHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
-POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
-PREHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
-PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
-POSTHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
-POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
-PREHOOK: query: DESCRIBE EXTENDED table2
+PREHOOK: query: DESCRIBE FORMATTED table12
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table2
-POSTHOOK: query: DESCRIBE EXTENDED table2
+PREHOOK: Input: default@table12
+POSTHOOK: query: DESCRIBE FORMATTED table12
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table2
+POSTHOOK: Input: default@table12
+# col_name            	data_type           	comment             
+	 	 
 a                   	string              	                    
 b                   	string              	                    
 	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
 #### A masked pattern was here ####
-PREHOOK: query: DESCRIBE EXTENDED table3
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table3
-POSTHOOK: query: DESCRIBE EXTENDED table3
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table3
-x                   	string              	                    
-	 	 
+Retention:          	0                   	 
 #### A masked pattern was here ####
-Constraints	Primary Key for default.table3:[x], Constraint Name: #### A masked pattern was here ####	 
-PREHOOK: query: DESCRIBE EXTENDED table6
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table6
-POSTHOOK: query: DESCRIBE EXTENDED table6
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table6
-x                   	string              	                    
-y                   	string              	                    
-	 	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
 #### A masked pattern was here ####
-Constraints	Primary Key for default.table6:[x], Constraint Name: #### A masked pattern was here ####	 
-PREHOOK: query: DESCRIBE FORMATTED table2
+	 	 
+# 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	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table12     	 
+Constraint Name:    	nn12_1              	 
+Column Name:        	a                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table13
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table2
-POSTHOOK: query: DESCRIBE FORMATTED table2
+PREHOOK: Input: default@table13
+POSTHOOK: query: DESCRIBE FORMATTED table13
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table2
+POSTHOOK: Input: default@table13
 # col_name            	data_type           	comment             
 	 	 
 a                   	string              	                    
@@ -804,15 +950,24 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: DESCRIBE FORMATTED table3
+	 	 
+# Constraints	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table13     	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	a                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table14
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table3
-POSTHOOK: query: DESCRIBE FORMATTED table3
+PREHOOK: Input: default@table14
+POSTHOOK: query: DESCRIBE FORMATTED table14
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table3
+POSTHOOK: Input: default@table14
 # col_name            	data_type           	comment             
 	 	 
-x                   	string              	                    
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
@@ -841,20 +996,21 @@ Storage Desc Params:
 	 	 
 # Constraints	 	 
 	 	 
-# Primary Key	 	 
-Table:              	default.table3      	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	 
-PREHOOK: query: DESCRIBE FORMATTED table6
+# Not Null Constraints	 	 
+Table:              	default.table14     	 
+Constraint Name:    	nn14_1              	 
+Column Name:        	a                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table15
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table6
-POSTHOOK: query: DESCRIBE FORMATTED table6
+PREHOOK: Input: default@table15
+POSTHOOK: query: DESCRIBE FORMATTED table15
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table6
+POSTHOOK: Input: default@table15
 # col_name            	data_type           	comment             
 	 	 
-x                   	string              	                    
-y                   	string              	                    
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
@@ -880,31 +1036,12 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-	 	 
-# Constraints	 	 
-	 	 
-# Primary Key	 	 
-Table:              	default.table6      	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	 
-PREHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 primary key (a) disable novalidate
-PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-POSTHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 primary key (a) disable novalidate
-POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
-PREHOOK: query: ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE RELY
-PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-POSTHOOK: query: ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a)  DISABLE NOVALIDATE RELY
-POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
-PREHOOK: query: ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE
-PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-POSTHOOK: query: ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE
-POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
-PREHOOK: query: DESCRIBE FORMATTED table2
+PREHOOK: query: DESCRIBE FORMATTED table16
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table2
-POSTHOOK: query: DESCRIBE FORMATTED table2
+PREHOOK: Input: default@table16
+POSTHOOK: query: DESCRIBE FORMATTED table16
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table2
+POSTHOOK: Input: default@table16
 # col_name            	data_type           	comment             
 	 	 
 a                   	string              	                    
@@ -934,22 +1071,16 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-	 	 
-# Constraints	 	 
-	 	 
-# Primary Key	 	 
-Table:              	default.table2      	 
-Constraint Name:    	pkt2                	 
-Column Names:       	a                   	 
-PREHOOK: query: DESCRIBE FORMATTED table3
+PREHOOK: query: DESCRIBE FORMATTED table17
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table3
-POSTHOOK: query: DESCRIBE FORMATTED table3
+PREHOOK: Input: default@table17
+POSTHOOK: query: DESCRIBE FORMATTED table17
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table3
+POSTHOOK: Input: default@table17
 # col_name            	data_type           	comment             
 	 	 
-x                   	string              	                    
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
@@ -978,26 +1109,21 @@ Storage Desc Params:
 	 	 
 # Constraints	 	 
 	 	 
-# Primary Key	 	 
-Table:              	default.table3      	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	 
-	 	 
-# Foreign Keys	 	 
-Table:              	default.table3      	 
-Constraint Name:    	fk1                 	 
-Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1      
+# Unique Constraints	 	 
+Table:              	default.table17     	 
+Constraint Name:    	uk17_1              	 
+Column Name:a       	Key Sequence:1      	 
 	 	 
-PREHOOK: query: DESCRIBE FORMATTED table6
+PREHOOK: query: DESCRIBE FORMATTED table18
 PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@table6
-POSTHOOK: query: DESCRIBE FORMATTED table6
+PREHOOK: Input: default@table18
+POSTHOOK: query: DESCRIBE FORMATTED table18
 POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@table6
+POSTHOOK: Input: default@table18
 # col_name            	data_type           	comment             
 	 	 
-x                   	string              	                    
-y                   	string              	                    
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
@@ -1026,38 +1152,769 @@ Storage Desc Params:
 	 	 
 # Constraints	 	 
 	 	 
-# Primary Key	 	 
-Table:              	default.table6      	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	 
-	 	 
-# Foreign Keys	 	 
-Table:              	default.table6      	 
-Constraint Name:    	fk4                 	 
-Parent Column Name:default.table1.a	Column Name:y       	Key Sequence:1      
+# Unique Constraints	 	 
+Table:              	default.table18     	 
+Constraint Name:    	uk18_1              	 
+Column Name:b       	Key Sequence:1      	 
 	 	 
-Constraint Name:    	#### A masked pattern was here ####	 
-Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1      
-	 	 
-PREHOOK: query: CREATE DATABASE DbConstraint
-PREHOOK: type: CREATEDATABASE
-PREHOOK: Output: database:DbConstraint
-POSTHOOK: query: CREATE DATABASE DbConstraint
-POSTHOOK: type: CREATEDATABASE
-POSTHOOK: Output: database:DbConstraint
-PREHOOK: query: USE DbConstraint
-PREHOOK: type: SWITCHDATABASE
-PREHOOK: Input: database:dbconstraint
-POSTHOOK: query: USE DbConstraint
-POSTHOOK: type: SWITCHDATABASE
-POSTHOOK: Input: database:dbconstraint
-PREHOOK: query: CREATE TABLE Table2 (a STRING, b STRING, constraint Pk1 primary key (a) disable novalidate)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: DbConstraint@Table2
-PREHOOK: Output: database:dbconstraint
-POSTHOOK: query: CREATE TABLE Table2 (a STRING, b STRING, constraint Pk1 primary key (a) disable novalidate)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: DbConstraint@Table2
+PREHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table2 DROP CONSTRAINT pk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table3 DROP CONSTRAINT fk1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table4 DROP CONSTRAINT nn4_1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table4 DROP CONSTRAINT nn4_1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table6 DROP CONSTRAINT fk4
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table16 DROP CONSTRAINT nn16_1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table16 DROP CONSTRAINT nn16_1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: ALTER TABLE table18 DROP CONSTRAINT uk18_1
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table18 DROP CONSTRAINT uk18_1
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: DESCRIBE EXTENDED table2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table2
+POSTHOOK: query: DESCRIBE EXTENDED table2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table2
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: DESCRIBE EXTENDED table3
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table3
+POSTHOOK: query: DESCRIBE EXTENDED table3
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table3
+x                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Primary Key for default.table3:[x], Constraint Name: #### A masked pattern was here ####	 
+Not Null Constraints for default.table3:[ {Constraint Name: #### A masked pattern was here ####, Column Name: x}]	 	 
+PREHOOK: query: DESCRIBE EXTENDED table4
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table4
+POSTHOOK: query: DESCRIBE EXTENDED table4
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table4
+x                   	string              	                    
+y                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Unique Constraints for default.table4:[ {Constraint Name: #### A masked pattern was here ####, (Column Name: x, Key Sequence: 1)}]	 
+Not Null Constraints for default.table4:[ {Constraint Name: nn4_2, Column Name: y}]	 	 
+PREHOOK: query: DESCRIBE EXTENDED table6
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table6
+POSTHOOK: query: DESCRIBE EXTENDED table6
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table6
+x                   	string              	                    
+y                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+Constraints	Primary Key for default.table6:[x], Constraint Name: #### A masked pattern was here ####	 
+PREHOOK: query: DESCRIBE EXTENDED table16
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table16
+POSTHOOK: query: DESCRIBE EXTENDED table16
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table16
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: DESCRIBE EXTENDED table18
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table18
+POSTHOOK: query: DESCRIBE EXTENDED table18
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table18
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: DESCRIBE FORMATTED table2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table2
+POSTHOOK: query: DESCRIBE FORMATTED table2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table2
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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                   
+PREHOOK: query: DESCRIBE FORMATTED table3
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table3
+POSTHOOK: query: DESCRIBE FORMATTED table3
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table3
+# col_name            	data_type           	comment             
+	 	 
+x                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Primary Key	 	 
+Table:              	default.table3      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Names:       	x                   	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table3      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	x                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table4
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table4
+POSTHOOK: query: DESCRIBE FORMATTED table4
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table4
+# col_name            	data_type           	comment             
+	 	 
+x                   	string              	                    
+y                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.table4      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:x       	Key Sequence:1      	 
+	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table4      	 
+Constraint Name:    	nn4_2               	 
+Column Name:        	y                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table6
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table6
+POSTHOOK: query: DESCRIBE FORMATTED table6
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table6
+# col_name            	data_type           	comment             
+	 	 
+x                   	string              	                    
+y                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Primary Key	 	 
+Table:              	default.table6      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Names:       	x                   	 
+PREHOOK: query: DESCRIBE FORMATTED table16
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table16
+POSTHOOK: query: DESCRIBE FORMATTED table16
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table16
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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                   
+PREHOOK: query: DESCRIBE FORMATTED table18
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table18
+POSTHOOK: query: DESCRIBE FORMATTED table18
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table18
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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                   
+PREHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 PRIMARY KEY (a) DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
+POSTHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 PRIMARY KEY (a) DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
+PREHOOK: query: ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE NOVALIDATE RELY
+PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
+POSTHOOK: query: ALTER TABLE table3 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES table2(a) DISABLE NOVALIDATE RELY
+POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
+PREHOOK: query: ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
+POSTHOOK: query: ALTER TABLE table6 ADD CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a) DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
+PREHOOK: query: ALTER TABLE table16 CHANGE a a STRING REFERENCES table4(x) DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@table16
+PREHOOK: Output: default@table16
+POSTHOOK: query: ALTER TABLE table16 CHANGE a a STRING REFERENCES table4(x) DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@table16
+POSTHOOK: Output: default@table16
+PREHOOK: query: ALTER TABLE table18 ADD CONSTRAINT uk18_2 UNIQUE (a, b) DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
+POSTHOOK: query: ALTER TABLE table18 ADD CONSTRAINT uk18_2 UNIQUE (a, b) DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
+PREHOOK: query: DESCRIBE FORMATTED table2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table2
+POSTHOOK: query: DESCRIBE FORMATTED table2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table2
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Primary Key	 	 
+Table:              	default.table2      	 
+Constraint Name:    	pkt2                	 
+Column Names:       	a                   	 
+PREHOOK: query: DESCRIBE FORMATTED table3
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table3
+POSTHOOK: query: DESCRIBE FORMATTED table3
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table3
+# col_name            	data_type           	comment             
+	 	 
+x                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Primary Key	 	 
+Table:              	default.table3      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Names:       	x                   	 
+	 	 
+# Foreign Keys	 	 
+Table:              	default.table3      	 
+Constraint Name:    	fk1                 	 
+Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1      
+	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table3      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	x                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table6
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table6
+POSTHOOK: query: DESCRIBE FORMATTED table6
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table6
+# col_name            	data_type           	comment             
+	 	 
+x                   	string              	                    
+y                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Primary Key	 	 
+Table:              	default.table6      	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Names:       	x                   	 
+	 	 
+# Foreign Keys	 	 
+Table:              	default.table6      	 
+Constraint Name:    	fk4                 	 
+Parent Column Name:default.table1.a	Column Name:y       	Key Sequence:1      
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Parent Column Name:default.table2.a	Column Name:x       	Key Sequence:1      
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table16
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table16
+POSTHOOK: query: DESCRIBE FORMATTED table16
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table16
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+#### A masked pattern was here ####
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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                   
+PREHOOK: query: DESCRIBE FORMATTED table18
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table18
+POSTHOOK: query: DESCRIBE FORMATTED table18
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table18
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Unique Constraints	 	 
+Table:              	default.table18     	 
+Constraint Name:    	uk18_2              	 
+Column Name:a       	Key Sequence:1      	 
+Column Name:b       	Key Sequence:2      	 
+	 	 
+PREHOOK: query: ALTER TABLE table12 CHANGE COLUMN b b STRING CONSTRAINT nn12_2 NOT NULL DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@table12
+PREHOOK: Output: default@table12
+POSTHOOK: query: ALTER TABLE table12 CHANGE COLUMN b b STRING CONSTRAINT nn12_2 NOT NULL DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@table12
+POSTHOOK: Output: default@table12
+PREHOOK: query: ALTER TABLE table13 CHANGE b b STRING NOT NULL DISABLE NOVALIDATE
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@table13
+PREHOOK: Output: default@table13
+POSTHOOK: query: ALTER TABLE table13 CHANGE b b STRING NOT NULL DISABLE NOVALIDATE
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@table13
+POSTHOOK: Output: default@table13
+PREHOOK: query: DESCRIBE FORMATTED table12
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table12
+POSTHOOK: query: DESCRIBE FORMATTED table12
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table12
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+#### A masked pattern was here ####
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table12     	 
+Constraint Name:    	nn12_1              	 
+Column Name:        	a                   	 
+	 	 
+Constraint Name:    	nn12_2              	 
+Column Name:        	b                   	 
+	 	 
+PREHOOK: query: DESCRIBE FORMATTED table13
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table13
+POSTHOOK: query: DESCRIBE FORMATTED table13
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table13
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+#### A masked pattern was here ####
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table13     	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	a                   	 
+	 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	b                   	 
+	 	 
+PREHOOK: query: ALTER TABLE table12 DROP CONSTRAINT nn12_2
+PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
+POSTHOOK: query: ALTER TABLE table12 DROP CONSTRAINT nn12_2
+POSTHOOK: type: ALTERTABLE_DROPCONSTRAINT
+PREHOOK: query: DESCRIBE FORMATTED table12
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@table12
+POSTHOOK: query: DESCRIBE FORMATTED table12
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@table12
+# col_name            	data_type           	comment             
+	 	 
+a                   	string              	                    
+b                   	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+#### A masked pattern was here ####
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### 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	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	default.table12     	 
+Constraint Name:    	nn12_1              	 
+Column Name:        	a                   	 
+	 	 
+PREHOOK: query: CREATE DATABASE DbConstraint
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:DbConstraint
+POSTHOOK: query: CREATE DATABASE DbConstraint
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:DbConstraint
+PREHOOK: query: USE DbConstraint
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:dbconstraint
+POSTHOOK: query: USE DbConstraint
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:dbconstraint
+PREHOOK: query: CREATE TABLE Table2 (a STRING, b STRING NOT NULL DISABLE, CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: DbConstraint@Table2
+PREHOOK: Output: database:dbconstraint
+POSTHOOK: query: CREATE TABLE Table2 (a STRING, b STRING NOT NULL DISABLE, CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: DbConstraint@Table2
 POSTHOOK: Output: database:dbconstraint
 PREHOOK: query: USE default
 PREHOOK: type: SWITCHDATABASE
@@ -1076,6 +1933,7 @@ b                   	string
 	 	 
 #### A masked pattern was here ####
 Constraints	Primary Key for dbconstraint.table2:[a], Constraint Name: pk1	 
+Not Null Constraints for dbconstraint.table2:[ {Constraint Name: #### A masked pattern was here ####, Column Name: b}]	 	 
 PREHOOK: query: DESCRIBE FORMATTED DbConstraint.Table2
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: dbconstraint@table2
@@ -1118,6 +1976,12 @@ Storage Desc Params:
 Table:              	dbconstraint.table2 	 
 Constraint Name:    	pk1                 	 
 Column Names:       	a                   	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	dbconstraint.table2 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	b                   	 
+	 	 
 PREHOOK: query: ALTER TABLE DbConstraint.Table2 DROP CONSTRAINT Pk1
 PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
 POSTHOOK: query: ALTER TABLE DbConstraint.Table2 DROP CONSTRAINT Pk1
@@ -1132,6 +1996,7 @@ a                   	string
 b                   	string              	                    
 	 	 
 #### A masked pattern was here ####
+Constraints	Not Null Constraints for dbconstraint.table2:[ {Constraint Name: #### A masked pattern was here ####, Column Name: b}]	 
 PREHOOK: query: DESCRIBE FORMATTED DbConstraint.Table2
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: dbconstraint@table2
@@ -1167,9 +2032,17 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
-PREHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 primary key (a) disable novalidate
+	 	 
+# Constraints	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	dbconstraint.table2 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	b                   	 
+	 	 
+PREHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE NOVALIDATE
 PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-POSTHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 primary key (a) disable novalidate
+POSTHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT Pk1 PRIMARY KEY (a) DISABLE NOVALIDATE
 POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
 PREHOOK: query: DESCRIBE FORMATTED DbConstraint.Table2
 PREHOOK: type: DESCTABLE
@@ -1213,9 +2086,15 @@ Storage Desc Params:
 Table:              	dbconstraint.table2 	 
 Constraint Name:    	pk1                 	 
 Column Names:       	a                   	 
-PREHOOK: query: ALTER TABLE DbConstraint.Table2  ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a)  DISABLE NOVALIDATE
+	 	 
+# Not Null Constraints	 	 
+Table:              	dbconstraint.table2 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	b                   	 
+	 	 
+PREHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a) DISABLE NOVALIDATE
 PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-POSTHOOK: query: ALTER TABLE DbConstraint.Table2  ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a)  DISABLE NOVALIDATE
+POSTHOOK: query: ALTER TABLE DbConstraint.Table2 ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a) DISABLE NOVALIDATE
 POSTHOOK: type: ALTERTABLE_ADDCONSTRAINT
 PREHOOK: query: DESCRIBE FORMATTED DbConstraint.Table2
 PREHOOK: type: DESCTABLE
@@ -1265,3 +2144,9 @@ Table:              	dbconstraint.table2
 Constraint Name:    	fkx                 	 
 Parent Column Name:default.table1.a	Column Name:b       	Key Sequence:1      
 	 	 
+	 	 
+# Not Null Constraints	 	 
+Table:              	dbconstraint.table2 	 
+Constraint Name:    	#### A masked pattern was here ####	 
+Column Name:        	b                   	 
+	 	 


[17/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
new file mode 100644
index 0000000..ccbecfc
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.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)")
+public class UniqueConstraintsRequest implements org.apache.thrift.TBase<UniqueConstraintsRequest, UniqueConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<UniqueConstraintsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UniqueConstraintsRequest");
+
+  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 UniqueConstraintsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new UniqueConstraintsRequestTupleSchemeFactory());
+  }
+
+  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(UniqueConstraintsRequest.class, metaDataMap);
+  }
+
+  public UniqueConstraintsRequest() {
+  }
+
+  public UniqueConstraintsRequest(
+    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 UniqueConstraintsRequest(UniqueConstraintsRequest other) {
+    if (other.isSetDb_name()) {
+      this.db_name = other.db_name;
+    }
+    if (other.isSetTbl_name()) {
+      this.tbl_name = other.tbl_name;
+    }
+  }
+
+  public UniqueConstraintsRequest deepCopy() {
+    return new UniqueConstraintsRequest(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 UniqueConstraintsRequest)
+      return this.equals((UniqueConstraintsRequest)that);
+    return false;
+  }
+
+  public boolean equals(UniqueConstraintsRequest 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(UniqueConstraintsRequest 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("UniqueConstraintsRequest(");
+    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 UniqueConstraintsRequestStandardSchemeFactory implements SchemeFactory {
+    public UniqueConstraintsRequestStandardScheme getScheme() {
+      return new UniqueConstraintsRequestStandardScheme();
+    }
+  }
+
+  private static class UniqueConstraintsRequestStandardScheme extends StandardScheme<UniqueConstraintsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, UniqueConstraintsRequest 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, UniqueConstraintsRequest 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 UniqueConstraintsRequestTupleSchemeFactory implements SchemeFactory {
+    public UniqueConstraintsRequestTupleScheme getScheme() {
+      return new UniqueConstraintsRequestTupleScheme();
+    }
+  }
+
+  private static class UniqueConstraintsRequestTupleScheme extends TupleScheme<UniqueConstraintsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsRequest 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, UniqueConstraintsRequest 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/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
new file mode 100644
index 0000000..7de84ea
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.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)")
+public class UniqueConstraintsResponse implements org.apache.thrift.TBase<UniqueConstraintsResponse, UniqueConstraintsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<UniqueConstraintsResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UniqueConstraintsResponse");
+
+  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)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new UniqueConstraintsResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new UniqueConstraintsResponseTupleSchemeFactory());
+  }
+
+  private List<SQLUniqueConstraint> uniqueConstraints; // 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 {
+    UNIQUE_CONSTRAINTS((short)1, "uniqueConstraints");
+
+    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: // UNIQUE_CONSTRAINTS
+          return UNIQUE_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.UNIQUE_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("uniqueConstraints", 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, SQLUniqueConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(UniqueConstraintsResponse.class, metaDataMap);
+  }
+
+  public UniqueConstraintsResponse() {
+  }
+
+  public UniqueConstraintsResponse(
+    List<SQLUniqueConstraint> uniqueConstraints)
+  {
+    this();
+    this.uniqueConstraints = uniqueConstraints;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public UniqueConstraintsResponse(UniqueConstraintsResponse other) {
+    if (other.isSetUniqueConstraints()) {
+      List<SQLUniqueConstraint> __this__uniqueConstraints = new ArrayList<SQLUniqueConstraint>(other.uniqueConstraints.size());
+      for (SQLUniqueConstraint other_element : other.uniqueConstraints) {
+        __this__uniqueConstraints.add(new SQLUniqueConstraint(other_element));
+      }
+      this.uniqueConstraints = __this__uniqueConstraints;
+    }
+  }
+
+  public UniqueConstraintsResponse deepCopy() {
+    return new UniqueConstraintsResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.uniqueConstraints = null;
+  }
+
+  public int getUniqueConstraintsSize() {
+    return (this.uniqueConstraints == null) ? 0 : this.uniqueConstraints.size();
+  }
+
+  public java.util.Iterator<SQLUniqueConstraint> getUniqueConstraintsIterator() {
+    return (this.uniqueConstraints == null) ? null : this.uniqueConstraints.iterator();
+  }
+
+  public void addToUniqueConstraints(SQLUniqueConstraint elem) {
+    if (this.uniqueConstraints == null) {
+      this.uniqueConstraints = new ArrayList<SQLUniqueConstraint>();
+    }
+    this.uniqueConstraints.add(elem);
+  }
+
+  public List<SQLUniqueConstraint> getUniqueConstraints() {
+    return this.uniqueConstraints;
+  }
+
+  public void setUniqueConstraints(List<SQLUniqueConstraint> uniqueConstraints) {
+    this.uniqueConstraints = uniqueConstraints;
+  }
+
+  public void unsetUniqueConstraints() {
+    this.uniqueConstraints = null;
+  }
+
+  /** Returns true if field uniqueConstraints is set (has been assigned a value) and false otherwise */
+  public boolean isSetUniqueConstraints() {
+    return this.uniqueConstraints != null;
+  }
+
+  public void setUniqueConstraintsIsSet(boolean value) {
+    if (!value) {
+      this.uniqueConstraints = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case UNIQUE_CONSTRAINTS:
+      if (value == null) {
+        unsetUniqueConstraints();
+      } else {
+        setUniqueConstraints((List<SQLUniqueConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case UNIQUE_CONSTRAINTS:
+      return getUniqueConstraints();
+
+    }
+    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 UNIQUE_CONSTRAINTS:
+      return isSetUniqueConstraints();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof UniqueConstraintsResponse)
+      return this.equals((UniqueConstraintsResponse)that);
+    return false;
+  }
+
+  public boolean equals(UniqueConstraintsResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_uniqueConstraints = true && this.isSetUniqueConstraints();
+    boolean that_present_uniqueConstraints = true && that.isSetUniqueConstraints();
+    if (this_present_uniqueConstraints || that_present_uniqueConstraints) {
+      if (!(this_present_uniqueConstraints && that_present_uniqueConstraints))
+        return false;
+      if (!this.uniqueConstraints.equals(that.uniqueConstraints))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_uniqueConstraints = true && (isSetUniqueConstraints());
+    list.add(present_uniqueConstraints);
+    if (present_uniqueConstraints)
+      list.add(uniqueConstraints);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(UniqueConstraintsResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetUniqueConstraints()).compareTo(other.isSetUniqueConstraints());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUniqueConstraints()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uniqueConstraints, other.uniqueConstraints);
+      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("UniqueConstraintsResponse(");
+    boolean first = true;
+
+    sb.append("uniqueConstraints:");
+    if (this.uniqueConstraints == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.uniqueConstraints);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetUniqueConstraints()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uniqueConstraints' 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 UniqueConstraintsResponseStandardSchemeFactory implements SchemeFactory {
+    public UniqueConstraintsResponseStandardScheme getScheme() {
+      return new UniqueConstraintsResponseStandardScheme();
+    }
+  }
+
+  private static class UniqueConstraintsResponseStandardScheme extends StandardScheme<UniqueConstraintsResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, UniqueConstraintsResponse 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: // UNIQUE_CONSTRAINTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
+                struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list338.size);
+                SQLUniqueConstraint _elem339;
+                for (int _i340 = 0; _i340 < _list338.size; ++_i340)
+                {
+                  _elem339 = new SQLUniqueConstraint();
+                  _elem339.read(iprot);
+                  struct.uniqueConstraints.add(_elem339);
+                }
+                iprot.readListEnd();
+              }
+              struct.setUniqueConstraintsIsSet(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, UniqueConstraintsResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.uniqueConstraints != null) {
+        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 _iter341 : struct.uniqueConstraints)
+          {
+            _iter341.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class UniqueConstraintsResponseTupleSchemeFactory implements SchemeFactory {
+    public UniqueConstraintsResponseTupleScheme getScheme() {
+      return new UniqueConstraintsResponseTupleScheme();
+    }
+  }
+
+  private static class UniqueConstraintsResponseTupleScheme extends TupleScheme<UniqueConstraintsResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.uniqueConstraints.size());
+        for (SQLUniqueConstraint _iter342 : struct.uniqueConstraints)
+        {
+          _iter342.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list343.size);
+        SQLUniqueConstraint _elem344;
+        for (int _i345 = 0; _i345 < _list343.size; ++_i345)
+        {
+          _elem344 = new SQLUniqueConstraint();
+          _elem344.read(iprot);
+          struct.uniqueConstraints.add(_elem344);
+        }
+      }
+      struct.setUniqueConstraintsIsSet(true);
+    }
+  }
+
+}
+


[18/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 19c88cb..89c59e1 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -78,7 +78,7 @@ public class ThriftHiveMetastore {
 
     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) 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 drop_constraint(DropConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
@@ -86,6 +86,10 @@ public class ThriftHiveMetastore {
 
     public void add_foreign_key(AddForeignKeyRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
 
+    public void add_unique_constraint(AddUniqueConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public void add_not_null_constraint(AddNotNullConstraintRequest 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;
@@ -216,6 +220,10 @@ public class ThriftHiveMetastore {
 
     public ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
+    public UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
+    public NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest 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;
@@ -392,7 +400,7 @@ public class ThriftHiveMetastore {
 
     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, 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 drop_constraint(DropConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -400,6 +408,10 @@ public class ThriftHiveMetastore {
 
     public void add_foreign_key(AddForeignKeyRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void add_unique_constraint(AddUniqueConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void add_not_null_constraint(AddNotNullConstraintRequest 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;
@@ -530,6 +542,10 @@ public class ThriftHiveMetastore {
 
     public void get_foreign_keys(ForeignKeysRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_unique_constraints(UniqueConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_not_null_constraints(NotNullConstraintsRequest 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;
@@ -1220,18 +1236,20 @@ public class ThriftHiveMetastore {
       return;
     }
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) 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
     {
-      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
       recv_create_table_with_constraints();
     }
 
-    public void send_create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) 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) throws org.apache.thrift.TException
     {
       create_table_with_constraints_args args = new create_table_with_constraints_args();
       args.setTbl(tbl);
       args.setPrimaryKeys(primaryKeys);
       args.setForeignKeys(foreignKeys);
+      args.setUniqueConstraints(uniqueConstraints);
+      args.setNotNullConstraints(notNullConstraints);
       sendBase("create_table_with_constraints", args);
     }
 
@@ -1332,6 +1350,58 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public void add_unique_constraint(AddUniqueConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_add_unique_constraint(req);
+      recv_add_unique_constraint();
+    }
+
+    public void send_add_unique_constraint(AddUniqueConstraintRequest req) throws org.apache.thrift.TException
+    {
+      add_unique_constraint_args args = new add_unique_constraint_args();
+      args.setReq(req);
+      sendBase("add_unique_constraint", args);
+    }
+
+    public void recv_add_unique_constraint() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      add_unique_constraint_result result = new add_unique_constraint_result();
+      receiveBase(result, "add_unique_constraint");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
+    public void add_not_null_constraint(AddNotNullConstraintRequest req) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_add_not_null_constraint(req);
+      recv_add_not_null_constraint();
+    }
+
+    public void send_add_not_null_constraint(AddNotNullConstraintRequest req) throws org.apache.thrift.TException
+    {
+      add_not_null_constraint_args args = new add_not_null_constraint_args();
+      args.setReq(req);
+      sendBase("add_not_null_constraint", args);
+    }
+
+    public void recv_add_not_null_constraint() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      add_not_null_constraint_result result = new add_not_null_constraint_result();
+      receiveBase(result, "add_not_null_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);
@@ -3341,6 +3411,64 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_foreign_keys failed: unknown result");
     }
 
+    public UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_unique_constraints(request);
+      return recv_get_unique_constraints();
+    }
+
+    public void send_get_unique_constraints(UniqueConstraintsRequest request) throws org.apache.thrift.TException
+    {
+      get_unique_constraints_args args = new get_unique_constraints_args();
+      args.setRequest(request);
+      sendBase("get_unique_constraints", args);
+    }
+
+    public UniqueConstraintsResponse recv_get_unique_constraints() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_unique_constraints_result result = new get_unique_constraints_result();
+      receiveBase(result, "get_unique_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_unique_constraints failed: unknown result");
+    }
+
+    public NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest request) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_not_null_constraints(request);
+      return recv_get_not_null_constraints();
+    }
+
+    public void send_get_not_null_constraints(NotNullConstraintsRequest request) throws org.apache.thrift.TException
+    {
+      get_not_null_constraints_args args = new get_not_null_constraints_args();
+      args.setRequest(request);
+      sendBase("get_not_null_constraints", args);
+    }
+
+    public NotNullConstraintsResponse recv_get_not_null_constraints() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_not_null_constraints_result result = new get_not_null_constraints_result();
+      receiveBase(result, "get_not_null_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_not_null_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);
@@ -5764,9 +5892,9 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public void create_table_with_constraints(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, 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 {
       checkReady();
-      create_table_with_constraints_call method_call = new create_table_with_constraints_call(tbl, primaryKeys, foreignKeys, resultHandler, this, ___protocolFactory, ___transport);
+      create_table_with_constraints_call method_call = new create_table_with_constraints_call(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -5775,11 +5903,15 @@ public class ThriftHiveMetastore {
       private Table tbl;
       private List<SQLPrimaryKey> primaryKeys;
       private List<SQLForeignKey> foreignKeys;
-      public create_table_with_constraints_call(Table tbl, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys, 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<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 {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tbl = tbl;
         this.primaryKeys = primaryKeys;
         this.foreignKeys = foreignKeys;
+        this.uniqueConstraints = uniqueConstraints;
+        this.notNullConstraints = notNullConstraints;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -5788,6 +5920,8 @@ public class ThriftHiveMetastore {
         args.setTbl(tbl);
         args.setPrimaryKeys(primaryKeys);
         args.setForeignKeys(foreignKeys);
+        args.setUniqueConstraints(uniqueConstraints);
+        args.setNotNullConstraints(notNullConstraints);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -5898,6 +6032,70 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void add_unique_constraint(AddUniqueConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_unique_constraint_call method_call = new add_unique_constraint_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class add_unique_constraint_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private AddUniqueConstraintRequest req;
+      public add_unique_constraint_call(AddUniqueConstraintRequest 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_unique_constraint", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_unique_constraint_args args = new add_unique_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_unique_constraint();
+      }
+    }
+
+    public void add_not_null_constraint(AddNotNullConstraintRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_not_null_constraint_call method_call = new add_not_null_constraint_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class add_not_null_constraint_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private AddNotNullConstraintRequest req;
+      public add_not_null_constraint_call(AddNotNullConstraintRequest 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_not_null_constraint", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_not_null_constraint_args args = new add_not_null_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_not_null_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);
@@ -8368,6 +8566,70 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_unique_constraints(UniqueConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_unique_constraints_call method_call = new get_unique_constraints_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_unique_constraints_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private UniqueConstraintsRequest request;
+      public get_unique_constraints_call(UniqueConstraintsRequest 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_unique_constraints", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_unique_constraints_args args = new get_unique_constraints_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public UniqueConstraintsResponse 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_unique_constraints();
+      }
+    }
+
+    public void get_not_null_constraints(NotNullConstraintsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_not_null_constraints_call method_call = new get_not_null_constraints_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_not_null_constraints_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private NotNullConstraintsRequest request;
+      public get_not_null_constraints_call(NotNullConstraintsRequest 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_not_null_constraints", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_not_null_constraints_args args = new get_not_null_constraints_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public NotNullConstraintsResponse 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_not_null_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);
@@ -10645,6 +10907,8 @@ public class ThriftHiveMetastore {
       processMap.put("drop_constraint", new drop_constraint());
       processMap.put("add_primary_key", new add_primary_key());
       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("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());
@@ -10710,6 +10974,8 @@ public class ThriftHiveMetastore {
       processMap.put("get_index_names", new get_index_names());
       processMap.put("get_primary_keys", new get_primary_keys());
       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("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());
@@ -11279,7 +11545,7 @@ public class ThriftHiveMetastore {
       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);
+          iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints);
         } catch (AlreadyExistsException o1) {
           result.o1 = o1;
         } catch (InvalidObjectException o2) {
@@ -11371,6 +11637,58 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class add_unique_constraint<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_unique_constraint_args> {
+      public add_unique_constraint() {
+        super("add_unique_constraint");
+      }
+
+      public add_unique_constraint_args getEmptyArgsInstance() {
+        return new add_unique_constraint_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_unique_constraint_result getResult(I iface, add_unique_constraint_args args) throws org.apache.thrift.TException {
+        add_unique_constraint_result result = new add_unique_constraint_result();
+        try {
+          iface.add_unique_constraint(args.req);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    public static class add_not_null_constraint<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_not_null_constraint_args> {
+      public add_not_null_constraint() {
+        super("add_not_null_constraint");
+      }
+
+      public add_not_null_constraint_args getEmptyArgsInstance() {
+        return new add_not_null_constraint_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_not_null_constraint_result getResult(I iface, add_not_null_constraint_args args) throws org.apache.thrift.TException {
+        add_not_null_constraint_result result = new add_not_null_constraint_result();
+        try {
+          iface.add_not_null_constraint(args.req);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     public static class drop_table<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_table_args> {
       public drop_table() {
         super("drop_table");
@@ -13091,6 +13409,58 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_unique_constraints<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_unique_constraints_args> {
+      public get_unique_constraints() {
+        super("get_unique_constraints");
+      }
+
+      public get_unique_constraints_args getEmptyArgsInstance() {
+        return new get_unique_constraints_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_unique_constraints_result getResult(I iface, get_unique_constraints_args args) throws org.apache.thrift.TException {
+        get_unique_constraints_result result = new get_unique_constraints_result();
+        try {
+          result.success = iface.get_unique_constraints(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    public static class get_not_null_constraints<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_not_null_constraints_args> {
+      public get_not_null_constraints() {
+        super("get_not_null_constraints");
+      }
+
+      public get_not_null_constraints_args getEmptyArgsInstance() {
+        return new get_not_null_constraints_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_not_null_constraints_result getResult(I iface, get_not_null_constraints_args args) throws org.apache.thrift.TException {
+        get_not_null_constraints_result result = new get_not_null_constraints_result();
+        try {
+          result.success = iface.get_not_null_constraints(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     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");
@@ -14760,6 +15130,8 @@ public class ThriftHiveMetastore {
       processMap.put("drop_constraint", new drop_constraint());
       processMap.put("add_primary_key", new add_primary_key());
       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("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());
@@ -14825,6 +15197,8 @@ public class ThriftHiveMetastore {
       processMap.put("get_index_names", new get_index_names());
       processMap.put("get_primary_keys", new get_primary_keys());
       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("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());
@@ -16105,7 +16479,7 @@ public class ThriftHiveMetastore {
       }
 
       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,resultHandler);
+        iface.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints,resultHandler);
       }
     }
 
@@ -16292,20 +16666,20 @@ public class ThriftHiveMetastore {
       }
     }
 
-    public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
-      public drop_table() {
-        super("drop_table");
+    public static class add_unique_constraint<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_unique_constraint_args, Void> {
+      public add_unique_constraint() {
+        super("add_unique_constraint");
       }
 
-      public drop_table_args getEmptyArgsInstance() {
-        return new drop_table_args();
+      public add_unique_constraint_args getEmptyArgsInstance() {
+        return new add_unique_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) {
-            drop_table_result result = new drop_table_result();
+            add_unique_constraint_result result = new add_unique_constraint_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -16317,15 +16691,137 @@ public class ThriftHiveMetastore {
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            drop_table_result result = new drop_table_result();
+            add_unique_constraint_result result = new add_unique_constraint_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
             else             if (e instanceof MetaException) {
-                        result.o3 = (MetaException) e;
-                        result.setO3IsSet(true);
+                        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_unique_constraint_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.add_unique_constraint(args.req,resultHandler);
+      }
+    }
+
+    public static class add_not_null_constraint<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_not_null_constraint_args, Void> {
+      public add_not_null_constraint() {
+        super("add_not_null_constraint");
+      }
+
+      public add_not_null_constraint_args getEmptyArgsInstance() {
+        return new add_not_null_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_not_null_constraint_result result = new add_not_null_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_not_null_constraint_result result = new add_not_null_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_not_null_constraint_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.add_not_null_constraint(args.req,resultHandler);
+      }
+    }
+
+    public static class drop_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_table_args, Void> {
+      public drop_table() {
+        super("drop_table");
+      }
+
+      public drop_table_args getEmptyArgsInstance() {
+        return new drop_table_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) {
+            drop_table_result result = new drop_table_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;
+            drop_table_result result = new drop_table_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
                         msg = result;
             }
              else 
@@ -20373,6 +20869,130 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_unique_constraints<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_unique_constraints_args, UniqueConstraintsResponse> {
+      public get_unique_constraints() {
+        super("get_unique_constraints");
+      }
+
+      public get_unique_constraints_args getEmptyArgsInstance() {
+        return new get_unique_constraints_args();
+      }
+
+      public AsyncMethodCallback<UniqueConstraintsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<UniqueConstraintsResponse>() { 
+          public void onComplete(UniqueConstraintsResponse o) {
+            get_unique_constraints_result result = new get_unique_constraints_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_unique_constraints_result result = new get_unique_constraints_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) 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, get_unique_constraints_args args, org.apache.thrift.async.AsyncMethodCallback<UniqueConstraintsResponse> resultHandler) throws TException {
+        iface.get_unique_constraints(args.request,resultHandler);
+      }
+    }
+
+    public static class get_not_null_constraints<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_not_null_constraints_args, NotNullConstraintsResponse> {
+      public get_not_null_constraints() {
+        super("get_not_null_constraints");
+      }
+
+      public get_not_null_constraints_args getEmptyArgsInstance() {
+        return new get_not_null_constraints_args();
+      }
+
+      public AsyncMethodCallback<NotNullConstraintsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<NotNullConstraintsResponse>() { 
+          public void onComplete(NotNullConstraintsResponse o) {
+            get_not_null_constraints_result result = new get_not_null_constraints_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_not_null_constraints_result result = new get_not_null_constraints_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) 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, get_not_null_constraints_args args, org.apache.thrift.async.AsyncMethodCallback<NotNullConstraintsResponse> resultHandler) throws TException {
+        iface.get_not_null_constraints(args.request,resultHandler);
+      }
+    }
+
     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");
@@ -29704,13 +30324,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list674.size);
-                  String _elem675;
-                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                  org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list706.size);
+                  String _elem707;
+                  for (int _i708 = 0; _i708 < _list706.size; ++_i708)
                   {
-                    _elem675 = iprot.readString();
-                    struct.success.add(_elem675);
+                    _elem707 = iprot.readString();
+                    struct.success.add(_elem707);
                   }
                   iprot.readListEnd();
                 }
@@ -29745,9 +30365,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter677 : struct.success)
+            for (String _iter709 : struct.success)
             {
-              oprot.writeString(_iter677);
+              oprot.writeString(_iter709);
             }
             oprot.writeListEnd();
           }
@@ -29786,9 +30406,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter678 : struct.success)
+            for (String _iter710 : struct.success)
             {
-              oprot.writeString(_iter678);
+              oprot.writeString(_iter710);
             }
           }
         }
@@ -29803,13 +30423,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list679.size);
-            String _elem680;
-            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+            org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list711.size);
+            String _elem712;
+            for (int _i713 = 0; _i713 < _list711.size; ++_i713)
             {
-              _elem680 = iprot.readString();
-              struct.success.add(_elem680);
+              _elem712 = iprot.readString();
+              struct.success.add(_elem712);
             }
           }
           struct.setSuccessIsSet(true);
@@ -30463,13 +31083,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list682.size);
-                  String _elem683;
-                  for (int _i684 = 0; _i684 < _list682.size; ++_i684)
+                  org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list714.size);
+                  String _elem715;
+                  for (int _i716 = 0; _i716 < _list714.size; ++_i716)
                   {
-                    _elem683 = iprot.readString();
-                    struct.success.add(_elem683);
+                    _elem715 = iprot.readString();
+                    struct.success.add(_elem715);
                   }
                   iprot.readListEnd();
                 }
@@ -30504,9 +31124,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter685 : struct.success)
+            for (String _iter717 : struct.success)
             {
-              oprot.writeString(_iter685);
+              oprot.writeString(_iter717);
             }
             oprot.writeListEnd();
           }
@@ -30545,9 +31165,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter686 : struct.success)
+            for (String _iter718 : struct.success)
             {
-              oprot.writeString(_iter686);
+              oprot.writeString(_iter718);
             }
           }
         }
@@ -30562,13 +31182,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list687.size);
-            String _elem688;
-            for (int _i689 = 0; _i689 < _list687.size; ++_i689)
+            org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list719.size);
+            String _elem720;
+            for (int _i721 = 0; _i721 < _list719.size; ++_i721)
             {
-              _elem688 = iprot.readString();
-              struct.success.add(_elem688);
+              _elem720 = iprot.readString();
+              struct.success.add(_elem720);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35175,16 +35795,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map690 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map690.size);
-                  String _key691;
-                  Type _val692;
-                  for (int _i693 = 0; _i693 < _map690.size; ++_i693)
+                  org.apache.thrift.protocol.TMap _map722 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map722.size);
+                  String _key723;
+                  Type _val724;
+                  for (int _i725 = 0; _i725 < _map722.size; ++_i725)
                   {
-                    _key691 = iprot.readString();
-                    _val692 = new Type();
-                    _val692.read(iprot);
-                    struct.success.put(_key691, _val692);
+                    _key723 = iprot.readString();
+                    _val724 = new Type();
+                    _val724.read(iprot);
+                    struct.success.put(_key723, _val724);
                   }
                   iprot.readMapEnd();
                 }
@@ -35219,10 +35839,10 @@ public class ThriftHiveMetastore {
           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> _iter694 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter726 : struct.success.entrySet())
             {
-              oprot.writeString(_iter694.getKey());
-              _iter694.getValue().write(oprot);
+              oprot.writeString(_iter726.getKey());
+              _iter726.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -35261,10 +35881,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter695 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter727 : struct.success.entrySet())
             {
-              oprot.writeString(_iter695.getKey());
-              _iter695.getValue().write(oprot);
+              oprot.writeString(_iter727.getKey());
+              _iter727.getValue().write(oprot);
             }
           }
         }
@@ -35279,16 +35899,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map696 = 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*_map696.size);
-            String _key697;
-            Type _val698;
-            for (int _i699 = 0; _i699 < _map696.size; ++_i699)
+            org.apache.thrift.protocol.TMap _map728 = 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*_map728.size);
+            String _key729;
+            Type _val730;
+            for (int _i731 = 0; _i731 < _map728.size; ++_i731)
             {
-              _key697 = iprot.readString();
-              _val698 = new Type();
-              _val698.read(iprot);
-              struct.success.put(_key697, _val698);
+              _key729 = iprot.readString();
+              _val730 = new Type();
+              _val730.read(iprot);
+              struct.success.put(_key729, _val730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36323,14 +36943,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list700.size);
-                  FieldSchema _elem701;
-                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list732.size);
+                  FieldSchema _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem701 = new FieldSchema();
-                    _elem701.read(iprot);
-                    struct.success.add(_elem701);
+                    _elem733 = new FieldSchema();
+                    _elem733.read(iprot);
+                    struct.success.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -36383,9 +37003,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter703 : struct.success)
+            for (FieldSchema _iter735 : struct.success)
             {
-              _iter703.write(oprot);
+              _iter735.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36440,9 +37060,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter704 : struct.success)
+            for (FieldSchema _iter736 : struct.success)
             {
-              _iter704.write(oprot);
+              _iter736.write(oprot);
             }
           }
         }
@@ -36463,14 +37083,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list705.size);
-            FieldSchema _elem706;
-            for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list737.size);
+            FieldSchema _elem738;
+            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
             {
-              _elem706 = new FieldSchema();
-              _elem706.read(iprot);
-              struct.success.add(_elem706);
+              _elem738 = new FieldSchema();
+              _elem738.read(iprot);
+              struct.success.add(_elem738);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37624,14 +38244,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list708.size);
-                  FieldSchema _elem709;
-                  for (int _i710 = 0; _i710 < _list708.size; ++_i710)
+                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list740.size);
+                  FieldSchema _elem741;
+                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                   {
-                    _elem709 = new FieldSchema();
-                    _elem709.read(iprot);
-                    struct.success.add(_elem709);
+                    _elem741 = new FieldSchema();
+                    _elem741.read(iprot);
+                    struct.success.add(_elem741);
                   }
                   iprot.readListEnd();
                 }
@@ -37684,9 +38304,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter711 : struct.success)
+            for (FieldSchema _iter743 : struct.success)
             {
-              _iter711.write(oprot);
+              _iter743.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -37741,9 +38361,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter712 : struct.success)
+            for (FieldSchema _iter744 : struct.success)
             {
-              _iter712.write(oprot);
+              _iter744.write(oprot);
             }
           }
         }
@@ -37764,14 +38384,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list713.size);
-            FieldSchema _elem714;
-            for (int _i715 = 0; _i715 < _list713.size; ++_i715)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list745.size);
+            FieldSchema _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem714 = new FieldSchema();
-              _elem714.read(iprot);
-              struct.success.add(_elem714);
+              _elem746 = new FieldSchema();
+              _elem746.read(iprot);
+              struct.success.add(_elem746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38816,14 +39436,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list716.size);
-                  FieldSchema _elem717;
-                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list748.size);
+                  FieldSchema _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem717 = new FieldSchema();
-                    _elem717.read(iprot);
-                    struct.success.add(_elem717);
+                    _elem749 = new FieldSchema();
+                    _elem749.read(iprot);
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -38876,9 +39496,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter719 : struct.success)
+            for (FieldSchema _iter751 : struct.success)
             {
-              _iter719.write(oprot);
+              _iter751.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -38933,9 +39553,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter720 : struct.success)
+            for (FieldSchema _iter752 : struct.success)
             {
-              _iter720.write(oprot);
+              _iter752.write(oprot);
             }
           }
         }
@@ -38956,14 +39576,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list721.size);
-            FieldSchema _elem722;
-            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list753.size);
+            FieldSchema _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem722 = new FieldSchema();
-              _elem722.read(iprot);
-              struct.success.add(_elem722);
+              _elem754 = new FieldSchema();
+              _elem754.read(iprot);
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40117,14 +40737,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list724.size);
-                  FieldSchema _elem725;
-                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list756.size);
+                  FieldSchema _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem725 = new FieldSchema();
-                    _elem725.read(iprot);
-                    struct.success.add(_elem725);
+                    _elem757 = new FieldSchema();
+                    _elem757.read(iprot);
+                    struct.success.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -40177,9 +40797,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter727 : struct.success)
+            for (FieldSchema _iter759 : struct.success)
             {
-              _iter727.write(oprot);
+              _iter759.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -40234,9 +40854,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter728 : struct.success)
+            for (FieldSchema _iter760 : struct.success)
             {
-              _iter728.write(oprot);
+              _iter760.write(oprot);
             }
           }
         }
@@ -40257,14 +40877,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list729.size);
-            FieldSchema _elem730;
-            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list761.size);
+            FieldSchema _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem730 = new FieldSchema();
-              _elem730.read(iprot);
-              struct.success.add(_elem730);
+              _elem762 = new FieldSchema();
+              _elem762.read(iprot);
+              struct.success.add(_elem762);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42486,6 +43106,8 @@ public class ThriftHiveMetastore {
     private static final org.apache.thrift.protocol.TField TBL_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField PRIMARY_KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("primaryKeys", org.apache.thrift.protocol.TType.LIST, (short)2);
     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 Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -42496,12 +43118,16 @@ public class ThriftHiveMetastore {
     private Table tbl; // required
     private List<SQLPrimaryKey> primaryKeys; // required
     private List<SQLForeignKey> foreignKeys; // required
+    private List<SQLUniqueConstraint> uniqueConstraints; // required
+    private List<SQLNotNullConstraint> notNullConstraints; // 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 {
       TBL((short)1, "tbl"),
       PRIMARY_KEYS((short)2, "primaryKeys"),
-      FOREIGN_KEYS((short)3, "foreignKeys");
+      FOREIGN_KEYS((short)3, "foreignKeys"),
+      UNIQUE_CONSTRAINTS((short)4, "uniqueConstraints"),
+      NOT_NULL_CONSTRAINTS((short)5, "notNullConstraints");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -42522,6 +43148,10 @@ public class ThriftHiveMetastore {
             return PRIMARY_KEYS;
           case 3: // FOREIGN_KEYS
             return FOREIGN_KEYS;
+          case 4: // UNIQUE_CONSTRAINTS
+            return UNIQUE_CONSTRAINTS;
+          case 5: // NOT_NULL_CONSTRAINTS
+            return NOT_NULL_CONSTRAINTS;
           default:
             return null;
         }
@@ -42573,6 +43203,12 @@ public class ThriftHiveMetastore {
       tmpMap.put(_Fields.FOREIGN_KEYS, new org.apache.thrift.meta_data.FieldMetaData("foreignKeys", 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, SQLForeignKey.class))));
+      tmpMap.put(_Fields.UNIQUE_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("uniqueConstraints", 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, SQLUniqueConstraint.class))));
+      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))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_table_with_constraints_args.class, metaDataMap);
     }
@@ -42583,12 +43219,16 @@ public class ThriftHiveMetastore {
     public create_table_with_constraints_args(
       Table tbl,
       List<SQLPrimaryKey> primaryKeys,
-      List<SQLForeignKey> foreignKeys)
+      List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints,
+      List<SQLNotNullConstraint> notNullConstraints)
     {
       this();
       this.tbl = tbl;
       this.primaryKeys = primaryKeys;
       this.foreignKeys = foreignKeys;
+      this.uniqueConstraints = uniqueConstraints;
+      this.notNullConstraints = notNullConstraints;
     }
 
     /**
@@ -42612,6 +43252,20 @@ public class ThriftHiveMetastore {
         }
         this.foreignKeys = __this__foreignKeys;
       }
+      if (other.isSetUniqueConstraints()) {
+        List<SQLUniqueConstraint> __this__uniqueConstraints = new ArrayList<SQLUniqueConstraint>(other.uniqueConstraints.size());
+        for (SQLUniqueConstraint other_element : other.uniqueConstraints) {
+          __this__uniqueConstraints.add(new SQLUniqueConstraint(other_element));
+        }
+        this.uniqueConstraints = __this__uniqueConstraints;
+      }
+      if (other.isSetNotNullConstraints()) {
+        List<SQLNotNullConstraint> __this__notNullConstraints = new ArrayList<SQLNotNullConstraint>(other.notNullConstraints.size());
+        for (SQLNotNullConstraint other_element : other.notNullConstraints) {
+          __this__notNullConstraints.add(new SQLNotNullConstraint(other_element));
+        }
+        this.notNullConstraints = __this__notNullConstraints;
+      }
     }
 
     public create_table_with_constraints_args deepCopy() {
@@ -42623,6 +43277,8 @@ public class ThriftHiveMetastore {
       this.tbl = null;
       this.primaryKeys = null;
       this.foreignKeys = null;
+      this.uniqueConstraints = null;
+      this.notNullConstraints = null;
     }
 
     public Table getTbl() {
@@ -42724,6 +43380,82 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public int getUniqueConstraintsSize() {
+      return (this.uniqueConstraints == null) ? 0 : this.uniqueConstraints.size();
+    }
+
+    public java.util.Iterator<SQLUniqueConstraint> getUniqueConstraintsIterator() {
+      return (this.uniqueConstraints == null) ? null : this.uniqueConstraints.iterator();
+    }
+
+    public void addToUniqueConstraints(SQLUniqueConstraint elem) {
+      if (this.uniqueConstraints == null) {
+        this.uniqueConstraints = new ArrayList<SQLUniqueConstraint>();
+      }
+      this.uniqueConstraints.add(elem);
+    }
+
+    public List<SQLUniqueConstraint> getUniqueConstraints() {
+      return this.uniqueConstraints;
+    }
+
+    public void setUniqueConstraints(List<SQLUniqueConstraint> uniqueConstraints) {
+      this.uniqueConstraints = uniqueConstraints;
+    }
+
+    public void unsetUniqueConstraints() {
+      this.uniqueConstraints = null;
+    }
+
+    /** Returns true if field uniqueConstraints is set (has been assigned a value) and false otherwise */
+    public boolean isSetUniqueConstraints() {
+      return this.uniqueConstraints != null;
+    }
+
+    public void setUniqueConstraintsIsSet(boolean value) {
+      if (!value) {
+        this.uniqueConstraints = null;
+      }
+    }
+
+    public int getNotNullConstraintsSize() {
+      return (this.notNullConstraints == null) ? 0 : this.notNullConstraints.size();
+    }
+
+    public java.util.Iterator<SQLNotNullConstraint> getNotNullConstraintsIterator() {
+      return (this.notNullConstraints == null) ? null : this.notNullConstraints.iterator();
+    }
+
+    public void addToNotNullConstraints(SQLNotNullConstraint elem) {
+      if (this.notNullConstraints == null) {
+        this.notNullConstraints = new ArrayList<SQLNotNullConstraint>();
+      }
+      this.notNullConstraints.add(elem);
+    }
+
+    public List<SQLNotNullConstraint> getNotNullConstraints() {
+      return this.notNullConstraints;
+    }
+
+    public void setNotNullConstraints(List<SQLNotNullConstraint> notNullConstraints) {
+      this.notNullConstraints = notNullConstraints;
+    }
+
+    public void unsetNotNullConstraints() {
+      this.notNullConstraints = null;
+    }
+
+    /** Returns true if field notNullConstraints is set (has been assigned a value) and false otherwise */
+    public boolean isSetNotNullConstraints() {
+      return this.notNullConstraints != null;
+    }
+
+    public void setNotNullConstraintsIsSet(boolean value) {
+      if (!value) {
+        this.notNullConstraints = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case TBL:
@@ -42750,6 +43482,22 @@ public class ThriftHiveMetastore {
         }
         break;
 
+      case UNIQUE_CONSTRAINTS:
+        if (value == null) {
+          unsetUniqueConstraints();
+        } else {
+          setUniqueConstraints((List<SQLUniqueConstraint>)value);
+        }
+        break;
+
+      case NOT_NULL_CONSTRAINTS:
+        if (value == null) {
+          unsetNotNullConstraints();
+        } else {
+          setNotNullConstraints((List<SQLNotNullConstraint>)value);
+        }
+        break;
+
       }
     }
 
@@ -42764,6 +43512,12 @@ public class ThriftHiveMetastore {
       case FOREIGN_KEYS:
         return getForeignKeys();
 
+      case UNIQUE_CONSTRAINTS:
+        return getUniqueConstraints();
+
+      case NOT_NULL_CONSTRAINTS:
+        return getNotNullConstraints();
+
       }
       throw new IllegalStateException();
     }
@@ -42781,6 +43535,10 @@ public class ThriftHiveMetastore {
         return isSetPrimaryKeys();
       case FOREIGN_KEYS:
         return isSetForeignKeys();
+      case UNIQUE_CONSTRAINTS:
+        return isSetUniqueConstraints();
+      case NOT_NULL_CONSTRAINTS:
+        return isSetNotNullConstraints();
       }
       throw new IllegalStateException();
     }
@@ -42825,6 +43583,24 @@ public class ThriftHiveMetastore {
           return false;
       }
 
+      boolean this_present_uniqueConstraints = true && this.isSetUniqueConstraints();
+      boolean that_present_uniqueConstraints = true && that.isSetUniqueConstraints();
+      if (this_present_uniqueConstraints || that_present_uniqueConstraints) {
+        if (!(this_present_uniqueConstraints && that_present_uniqueConstraints))
+          return false;
+        if (!this.uniqueConstraints.equals(that.uniqueConstraints))
+          return false;
+      }
+
+      boolean this_present_notNullConstraints = true && this.isSetNotNullConstraints();
+      boolean that_present_notNullConstraints = true && that.isSetNotNullConstraints();
+      if (this_present_notNullConstraints || that_present_notNullConstraints) {
+        if (!(this_present_notNullConstraints && that_present_notNullConstraints))
+          return false;
+        if (!this.notNullConstraints.equals(that.notNullConstraints))
+          return false;
+      }
+
       return true;
     }
 
@@ -42847,6 +43623,16 @@ public class ThriftHiveMetastore {
       if (present_foreignKeys)
         list.add(foreignKeys);
 
+      boolean present_uniqueConstraints = true && (isSetUniqueConstraints());
+      list.add(present_uniqueConstraints);
+      if (present_uniqueConstraints)
+        list.add(uniqueConstraints);
+
+      boolean present_notNullConstraints = true && (isSetNotNullConstraints());
+      list.add(present_notNullConstraints);
+      if (present_notNullConstraints)
+        list.add(notNullConstraints);
+
       return list.hashCode();
     }
 
@@ -42888,6 +43674,26 @@ public class ThriftHiveMetastore {
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetUniqueConstraints()).compareTo(other.isSetUniqueConstraints());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetUniqueConstraints()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uniqueConstraints, other.uniqueConstraints);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetNotNullConstraints()).compareTo(other.isSetNotNullConstraints());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNotNullConstraints()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.notNullConstraints, other.notNullConstraints);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -42931,6 +43737,22 @@ public class ThriftHiveMetastore {
         sb.append(this.foreignKeys);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("uniqueConstraints:");
+      if (this.uniqueConstraints == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.uniqueConstraints);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("notNullConstraints:");
+      if (this.notNullConstraints == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.notNullConstraints);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -42989,14 +43811,14 @@ public class ThriftHiveMetastore {
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list732.size);
-                  SQLPrimaryKey _elem733;
-                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list764.size);
+                  SQLPrimaryKey _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem733 = new SQLPrimaryKey();
-                    _elem733.read(iprot);
-                    struct.primaryKeys.add(_elem733);
+                    _elem765 = new SQLPrimaryKey();
+                    _elem765.read(iprot);
+                    struct.primaryKeys.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -43008,14 +43830,14 @@ public class ThriftHiveMetastore {
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list735 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list735.size);
-                  SQLForeignKey _elem736;
-                  for (int _i737 = 0; _i737 < _list735.size; ++_i737)
+                  org.apache.thrift.protocol.TList _list767 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list767.size);
+                  SQLForeignKey _elem768;
+                  for (int _i769 = 0; _i769 < _list767.size; ++_i769)
                   {
-                    _elem736 = new SQLForeignKey();
-                    _elem736.read(iprot);
-                    struct.foreignKeys.add(_elem736);
+                    _elem768 = new SQLForeignKey();
+                    _elem768.read(iprot);
+                    struct.foreignKeys.add(_elem768);
                   }
                   iprot.readListEnd();
                 }
@@ -43024,6 +43846,44 @@ public class ThriftHiveMetastore {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 4: // UNIQUE_CONSTRAINTS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list770 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list770.size);
+                  SQLUniqueConstraint _elem771;
+                  for (int _i772 = 0; _i772 < _list770.size; ++_i772)
+                  {
+                    _elem771 = new SQLUniqueConstraint();
+                    _elem771.read(iprot);
+                    struct.uniqueConstraints.add(_elem771);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setUniqueConstraintsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // NOT_NULL_CONSTRAINTS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list773 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list773.size);
+                  SQLNotNullConstraint _elem774;
+                  for (int _i775 = 0; _i775 < _list773.size; ++_i775)
+                  {
+                    _elem774 = new SQLNotNullConstraint();
+                    _elem774.read(iprot);
+                    struct.notNullConstraints.add(_elem774);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setNotNullConstraintsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -43046,9 +43906,9 @@ public class ThriftHiveMetastore {
           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 _iter738 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter776 : struct.primaryKeys)
             {
-              _iter738.write(oprot);
+              _iter776.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -43058,9 +43918,33 @@ public class ThriftHiveMetastore {
           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 _iter739 : struct.foreignKeys)
+            for (SQLForeignKey _iter777 : struct.foreignKeys)
+            {
+              _iter777.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.uniqueConstraints != null) {
+          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 _iter778 : struct.uniqueConstraints)
             {
-              _iter739.write(oprot);
+              _iter778.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.notNullConstraints != null) {
+          oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstrain

<TRUNCATED>

[24/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index ac08ce1..47d9f1d 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -40,10 +40,12 @@ 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) = 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 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 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;
@@ -109,6 +111,8 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_index_names(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0;
   virtual void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) = 0;
   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 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;
@@ -262,7 +266,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 */) {
+  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 */) {
     return;
   }
   void drop_constraint(const DropConstraintRequest& /* req */) {
@@ -274,6 +278,12 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void add_foreign_key(const AddForeignKeyRequest& /* req */) {
     return;
   }
+  void add_unique_constraint(const AddUniqueConstraintRequest& /* req */) {
+    return;
+  }
+  void add_not_null_constraint(const AddNotNullConstraintRequest& /* req */) {
+    return;
+  }
   void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) {
     return;
   }
@@ -479,6 +489,12 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_foreign_keys(ForeignKeysResponse& /* _return */, const ForeignKeysRequest& /* request */) {
     return;
   }
+  void get_unique_constraints(UniqueConstraintsResponse& /* _return */, const UniqueConstraintsRequest& /* request */) {
+    return;
+  }
+  void get_not_null_constraints(NotNullConstraintsResponse& /* _return */, const NotNullConstraintsRequest& /* request */) {
+    return;
+  }
   bool update_table_column_statistics(const ColumnStatistics& /* stats_obj */) {
     bool _return = false;
     return _return;
@@ -2927,10 +2943,12 @@ 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) {}
+  _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false), uniqueConstraints(false), notNullConstraints(false) {}
   bool tbl :1;
   bool primaryKeys :1;
   bool foreignKeys :1;
+  bool uniqueConstraints :1;
+  bool notNullConstraints :1;
 } _ThriftHiveMetastore_create_table_with_constraints_args__isset;
 
 class ThriftHiveMetastore_create_table_with_constraints_args {
@@ -2945,6 +2963,8 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
   Table tbl;
   std::vector<SQLPrimaryKey>  primaryKeys;
   std::vector<SQLForeignKey>  foreignKeys;
+  std::vector<SQLUniqueConstraint>  uniqueConstraints;
+  std::vector<SQLNotNullConstraint>  notNullConstraints;
 
   _ThriftHiveMetastore_create_table_with_constraints_args__isset __isset;
 
@@ -2954,6 +2974,10 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
 
   void __set_foreignKeys(const std::vector<SQLForeignKey> & val);
 
+  void __set_uniqueConstraints(const std::vector<SQLUniqueConstraint> & val);
+
+  void __set_notNullConstraints(const std::vector<SQLNotNullConstraint> & val);
+
   bool operator == (const ThriftHiveMetastore_create_table_with_constraints_args & rhs) const
   {
     if (!(tbl == rhs.tbl))
@@ -2962,6 +2986,10 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       return false;
     if (!(foreignKeys == rhs.foreignKeys))
       return false;
+    if (!(uniqueConstraints == rhs.uniqueConstraints))
+      return false;
+    if (!(notNullConstraints == rhs.notNullConstraints))
+      return false;
     return true;
   }
   bool operator != (const ThriftHiveMetastore_create_table_with_constraints_args &rhs) const {
@@ -2984,6 +3012,8 @@ class ThriftHiveMetastore_create_table_with_constraints_pargs {
   const Table* tbl;
   const std::vector<SQLPrimaryKey> * primaryKeys;
   const std::vector<SQLForeignKey> * foreignKeys;
+  const std::vector<SQLUniqueConstraint> * uniqueConstraints;
+  const std::vector<SQLNotNullConstraint> * notNullConstraints;
 
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 
@@ -3404,6 +3434,230 @@ class ThriftHiveMetastore_add_foreign_key_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_add_unique_constraint_args__isset {
+  _ThriftHiveMetastore_add_unique_constraint_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_add_unique_constraint_args__isset;
+
+class ThriftHiveMetastore_add_unique_constraint_args {
+ public:
+
+  ThriftHiveMetastore_add_unique_constraint_args(const ThriftHiveMetastore_add_unique_constraint_args&);
+  ThriftHiveMetastore_add_unique_constraint_args& operator=(const ThriftHiveMetastore_add_unique_constraint_args&);
+  ThriftHiveMetastore_add_unique_constraint_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_unique_constraint_args() throw();
+  AddUniqueConstraintRequest req;
+
+  _ThriftHiveMetastore_add_unique_constraint_args__isset __isset;
+
+  void __set_req(const AddUniqueConstraintRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_add_unique_constraint_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_unique_constraint_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_unique_constraint_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_unique_constraint_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_unique_constraint_pargs() throw();
+  const AddUniqueConstraintRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_unique_constraint_result__isset {
+  _ThriftHiveMetastore_add_unique_constraint_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_unique_constraint_result__isset;
+
+class ThriftHiveMetastore_add_unique_constraint_result {
+ public:
+
+  ThriftHiveMetastore_add_unique_constraint_result(const ThriftHiveMetastore_add_unique_constraint_result&);
+  ThriftHiveMetastore_add_unique_constraint_result& operator=(const ThriftHiveMetastore_add_unique_constraint_result&);
+  ThriftHiveMetastore_add_unique_constraint_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_unique_constraint_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_unique_constraint_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_unique_constraint_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_unique_constraint_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_unique_constraint_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_unique_constraint_presult__isset {
+  _ThriftHiveMetastore_add_unique_constraint_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_unique_constraint_presult__isset;
+
+class ThriftHiveMetastore_add_unique_constraint_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_unique_constraint_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_unique_constraint_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_add_not_null_constraint_args__isset {
+  _ThriftHiveMetastore_add_not_null_constraint_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_add_not_null_constraint_args__isset;
+
+class ThriftHiveMetastore_add_not_null_constraint_args {
+ public:
+
+  ThriftHiveMetastore_add_not_null_constraint_args(const ThriftHiveMetastore_add_not_null_constraint_args&);
+  ThriftHiveMetastore_add_not_null_constraint_args& operator=(const ThriftHiveMetastore_add_not_null_constraint_args&);
+  ThriftHiveMetastore_add_not_null_constraint_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_not_null_constraint_args() throw();
+  AddNotNullConstraintRequest req;
+
+  _ThriftHiveMetastore_add_not_null_constraint_args__isset __isset;
+
+  void __set_req(const AddNotNullConstraintRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_add_not_null_constraint_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_not_null_constraint_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_not_null_constraint_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_not_null_constraint_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_not_null_constraint_pargs() throw();
+  const AddNotNullConstraintRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_not_null_constraint_result__isset {
+  _ThriftHiveMetastore_add_not_null_constraint_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_not_null_constraint_result__isset;
+
+class ThriftHiveMetastore_add_not_null_constraint_result {
+ public:
+
+  ThriftHiveMetastore_add_not_null_constraint_result(const ThriftHiveMetastore_add_not_null_constraint_result&);
+  ThriftHiveMetastore_add_not_null_constraint_result& operator=(const ThriftHiveMetastore_add_not_null_constraint_result&);
+  ThriftHiveMetastore_add_not_null_constraint_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_not_null_constraint_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_not_null_constraint_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_not_null_constraint_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_not_null_constraint_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_not_null_constraint_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_not_null_constraint_presult__isset {
+  _ThriftHiveMetastore_add_not_null_constraint_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_not_null_constraint_presult__isset;
+
+class ThriftHiveMetastore_add_not_null_constraint_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_not_null_constraint_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_not_null_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;
@@ -12098,6 +12352,246 @@ class ThriftHiveMetastore_get_foreign_keys_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_get_unique_constraints_args__isset {
+  _ThriftHiveMetastore_get_unique_constraints_args__isset() : request(false) {}
+  bool request :1;
+} _ThriftHiveMetastore_get_unique_constraints_args__isset;
+
+class ThriftHiveMetastore_get_unique_constraints_args {
+ public:
+
+  ThriftHiveMetastore_get_unique_constraints_args(const ThriftHiveMetastore_get_unique_constraints_args&);
+  ThriftHiveMetastore_get_unique_constraints_args& operator=(const ThriftHiveMetastore_get_unique_constraints_args&);
+  ThriftHiveMetastore_get_unique_constraints_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_unique_constraints_args() throw();
+  UniqueConstraintsRequest request;
+
+  _ThriftHiveMetastore_get_unique_constraints_args__isset __isset;
+
+  void __set_request(const UniqueConstraintsRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_unique_constraints_args & rhs) const
+  {
+    if (!(request == rhs.request))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_unique_constraints_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_unique_constraints_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_unique_constraints_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_unique_constraints_pargs() throw();
+  const UniqueConstraintsRequest* request;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_unique_constraints_result__isset {
+  _ThriftHiveMetastore_get_unique_constraints_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_unique_constraints_result__isset;
+
+class ThriftHiveMetastore_get_unique_constraints_result {
+ public:
+
+  ThriftHiveMetastore_get_unique_constraints_result(const ThriftHiveMetastore_get_unique_constraints_result&);
+  ThriftHiveMetastore_get_unique_constraints_result& operator=(const ThriftHiveMetastore_get_unique_constraints_result&);
+  ThriftHiveMetastore_get_unique_constraints_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_unique_constraints_result() throw();
+  UniqueConstraintsResponse success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_unique_constraints_result__isset __isset;
+
+  void __set_success(const UniqueConstraintsResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_unique_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_unique_constraints_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_unique_constraints_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_unique_constraints_presult__isset {
+  _ThriftHiveMetastore_get_unique_constraints_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_unique_constraints_presult__isset;
+
+class ThriftHiveMetastore_get_unique_constraints_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_unique_constraints_presult() throw();
+  UniqueConstraintsResponse* success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_unique_constraints_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_not_null_constraints_args__isset {
+  _ThriftHiveMetastore_get_not_null_constraints_args__isset() : request(false) {}
+  bool request :1;
+} _ThriftHiveMetastore_get_not_null_constraints_args__isset;
+
+class ThriftHiveMetastore_get_not_null_constraints_args {
+ public:
+
+  ThriftHiveMetastore_get_not_null_constraints_args(const ThriftHiveMetastore_get_not_null_constraints_args&);
+  ThriftHiveMetastore_get_not_null_constraints_args& operator=(const ThriftHiveMetastore_get_not_null_constraints_args&);
+  ThriftHiveMetastore_get_not_null_constraints_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_not_null_constraints_args() throw();
+  NotNullConstraintsRequest request;
+
+  _ThriftHiveMetastore_get_not_null_constraints_args__isset __isset;
+
+  void __set_request(const NotNullConstraintsRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_not_null_constraints_args & rhs) const
+  {
+    if (!(request == rhs.request))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_not_null_constraints_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_not_null_constraints_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_not_null_constraints_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_not_null_constraints_pargs() throw();
+  const NotNullConstraintsRequest* request;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_not_null_constraints_result__isset {
+  _ThriftHiveMetastore_get_not_null_constraints_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_not_null_constraints_result__isset;
+
+class ThriftHiveMetastore_get_not_null_constraints_result {
+ public:
+
+  ThriftHiveMetastore_get_not_null_constraints_result(const ThriftHiveMetastore_get_not_null_constraints_result&);
+  ThriftHiveMetastore_get_not_null_constraints_result& operator=(const ThriftHiveMetastore_get_not_null_constraints_result&);
+  ThriftHiveMetastore_get_not_null_constraints_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_not_null_constraints_result() throw();
+  NotNullConstraintsResponse success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_not_null_constraints_result__isset __isset;
+
+  void __set_success(const NotNullConstraintsResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_not_null_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_not_null_constraints_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_not_null_constraints_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_not_null_constraints_presult__isset {
+  _ThriftHiveMetastore_get_not_null_constraints_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_not_null_constraints_presult__isset;
+
+class ThriftHiveMetastore_get_not_null_constraints_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_not_null_constraints_presult() throw();
+  NotNullConstraintsResponse* success;
+  MetaException o1;
+  NoSuchObjectException o2;
+
+  _ThriftHiveMetastore_get_not_null_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;
@@ -19892,8 +20386,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);
-  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  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 recv_create_table_with_constraints();
   void drop_constraint(const DropConstraintRequest& req);
   void send_drop_constraint(const DropConstraintRequest& req);
@@ -19904,6 +20398,12 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void add_foreign_key(const AddForeignKeyRequest& req);
   void send_add_foreign_key(const AddForeignKeyRequest& req);
   void recv_add_foreign_key();
+  void add_unique_constraint(const AddUniqueConstraintRequest& req);
+  void send_add_unique_constraint(const AddUniqueConstraintRequest& req);
+  void recv_add_unique_constraint();
+  void add_not_null_constraint(const AddNotNullConstraintRequest& req);
+  void send_add_not_null_constraint(const AddNotNullConstraintRequest& req);
+  void recv_add_not_null_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();
@@ -20099,6 +20599,12 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request);
   void send_get_foreign_keys(const ForeignKeysRequest& request);
   void recv_get_foreign_keys(ForeignKeysResponse& _return);
+  void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request);
+  void send_get_unique_constraints(const UniqueConstraintsRequest& request);
+  void recv_get_unique_constraints(UniqueConstraintsResponse& _return);
+  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);
   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();
@@ -20335,6 +20841,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_add_primary_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   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_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);
@@ -20400,6 +20908,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   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_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);
@@ -20494,6 +21004,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["drop_constraint"] = &ThriftHiveMetastoreProcessor::process_drop_constraint;
     processMap_["add_primary_key"] = &ThriftHiveMetastoreProcessor::process_add_primary_key;
     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_["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;
@@ -20559,6 +21071,8 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_index_names"] = &ThriftHiveMetastoreProcessor::process_get_index_names;
     processMap_["get_primary_keys"] = &ThriftHiveMetastoreProcessor::process_get_primary_keys;
     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_["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;
@@ -20832,13 +21346,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) {
+  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) {
     size_t sz = ifaces_.size();
     size_t i = 0;
     for (; i < (sz - 1); ++i) {
-      ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+      ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
     }
-    ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+    ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
   }
 
   void drop_constraint(const DropConstraintRequest& req) {
@@ -20868,6 +21382,24 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->add_foreign_key(req);
   }
 
+  void add_unique_constraint(const AddUniqueConstraintRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_unique_constraint(req);
+    }
+    ifaces_[i]->add_unique_constraint(req);
+  }
+
+  void add_not_null_constraint(const AddNotNullConstraintRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_not_null_constraint(req);
+    }
+    ifaces_[i]->add_not_null_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;
@@ -21495,6 +22027,26 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_unique_constraints(_return, request);
+    }
+    ifaces_[i]->get_unique_constraints(_return, request);
+    return;
+  }
+
+  void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_not_null_constraints(_return, request);
+    }
+    ifaces_[i]->get_not_null_constraints(_return, request);
+    return;
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -22216,8 +22768,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);
-  int32_t send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys);
+  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 recv_create_table_with_constraints(const int32_t seqid);
   void drop_constraint(const DropConstraintRequest& req);
   int32_t send_drop_constraint(const DropConstraintRequest& req);
@@ -22228,6 +22780,12 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void add_foreign_key(const AddForeignKeyRequest& req);
   int32_t send_add_foreign_key(const AddForeignKeyRequest& req);
   void recv_add_foreign_key(const int32_t seqid);
+  void add_unique_constraint(const AddUniqueConstraintRequest& req);
+  int32_t send_add_unique_constraint(const AddUniqueConstraintRequest& req);
+  void recv_add_unique_constraint(const int32_t seqid);
+  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 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);
@@ -22423,6 +22981,12 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request);
   int32_t send_get_foreign_keys(const ForeignKeysRequest& request);
   void recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid);
+  void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request);
+  int32_t send_get_unique_constraints(const UniqueConstraintsRequest& request);
+  void recv_get_unique_constraints(UniqueConstraintsResponse& _return, const int32_t seqid);
+  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);
   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/696be9f5/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 80786e1..28f9af9 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/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) {
+  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) {
     // Your implementation goes here
     printf("create_table_with_constraints\n");
   }
@@ -132,6 +132,16 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("add_foreign_key\n");
   }
 
+  void add_unique_constraint(const AddUniqueConstraintRequest& req) {
+    // Your implementation goes here
+    printf("add_unique_constraint\n");
+  }
+
+  void add_not_null_constraint(const AddNotNullConstraintRequest& req) {
+    // Your implementation goes here
+    printf("add_not_null_constraint\n");
+  }
+
   void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) {
     // Your implementation goes here
     printf("drop_table\n");
@@ -457,6 +467,16 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_foreign_keys\n");
   }
 
+  void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) {
+    // Your implementation goes here
+    printf("get_unique_constraints\n");
+  }
+
+  void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) {
+    // Your implementation goes here
+    printf("get_not_null_constraints\n");
+  }
+
   bool update_table_column_statistics(const ColumnStatistics& stats_obj) {
     // Your implementation goes here
     printf("update_table_column_statistics\n");


[11/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index f1aa9a6..2cf38b5 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -234,6 +234,64 @@ class SQLForeignKey
   ::Thrift::Struct.generate_accessors self
 end
 
+class SQLUniqueConstraint
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TABLE_DB = 1
+  TABLE_NAME = 2
+  COLUMN_NAME = 3
+  KEY_SEQ = 4
+  UK_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'},
+    KEY_SEQ => {:type => ::Thrift::Types::I32, :name => 'key_seq'},
+    UK_NAME => {:type => ::Thrift::Types::STRING, :name => 'uk_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 SQLNotNullConstraint
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TABLE_DB = 1
+  TABLE_NAME = 2
+  COLUMN_NAME = 3
+  NN_NAME = 4
+  ENABLE_CSTR = 5
+  VALIDATE_CSTR = 6
+  RELY_CSTR = 7
+
+  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'},
+    NN_NAME => {:type => ::Thrift::Types::STRING, :name => 'nn_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
@@ -1407,6 +1465,80 @@ class ForeignKeysResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class UniqueConstraintsRequest
+  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 UniqueConstraintsResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  UNIQUECONSTRAINTS = 1
+
+  FIELDS = {
+    UNIQUECONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'uniqueConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLUniqueConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field uniqueConstraints is unset!') unless @uniqueConstraints
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class NotNullConstraintsRequest
+  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 NotNullConstraintsResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NOTNULLCONSTRAINTS = 1
+
+  FIELDS = {
+    NOTNULLCONSTRAINTS => {:type => ::Thrift::Types::LIST, :name => 'notNullConstraints', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLNotNullConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field notNullConstraints is unset!') unless @notNullConstraints
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class DropConstraintRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   DBNAME = 1
@@ -1464,6 +1596,40 @@ class AddForeignKeyRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class AddUniqueConstraintRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  UNIQUECONSTRAINTCOLS = 1
+
+  FIELDS = {
+    UNIQUECONSTRAINTCOLS => {:type => ::Thrift::Types::LIST, :name => 'uniqueConstraintCols', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLUniqueConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field uniqueConstraintCols is unset!') unless @uniqueConstraintCols
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class AddNotNullConstraintRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NOTNULLCONSTRAINTCOLS = 1
+
+  FIELDS = {
+    NOTNULLCONSTRAINTCOLS => {:type => ::Thrift::Types::LIST, :name => 'notNullConstraintCols', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SQLNotNullConstraint}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field notNullConstraintCols is unset!') unless @notNullConstraintCols
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class PartitionsByExprResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   PARTITIONS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index b9d0fa2..2711381 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/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)
-      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
+    def create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
+      send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints)
       recv_create_table_with_constraints()
     end
 
-    def send_create_table_with_constraints(tbl, primaryKeys, foreignKeys)
-      send_message('create_table_with_constraints', Create_table_with_constraints_args, :tbl => tbl, :primaryKeys => primaryKeys, :foreignKeys => foreignKeys)
+    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)
     end
 
     def recv_create_table_with_constraints()
@@ -384,6 +384,38 @@ module ThriftHiveMetastore
       return
     end
 
+    def add_unique_constraint(req)
+      send_add_unique_constraint(req)
+      recv_add_unique_constraint()
+    end
+
+    def send_add_unique_constraint(req)
+      send_message('add_unique_constraint', Add_unique_constraint_args, :req => req)
+    end
+
+    def recv_add_unique_constraint()
+      result = receive_message(Add_unique_constraint_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
+    def add_not_null_constraint(req)
+      send_add_not_null_constraint(req)
+      recv_add_not_null_constraint()
+    end
+
+    def send_add_not_null_constraint(req)
+      send_message('add_not_null_constraint', Add_not_null_constraint_args, :req => req)
+    end
+
+    def recv_add_not_null_constraint()
+      result = receive_message(Add_not_null_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()
@@ -1487,6 +1519,40 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_foreign_keys failed: unknown result')
     end
 
+    def get_unique_constraints(request)
+      send_get_unique_constraints(request)
+      return recv_get_unique_constraints()
+    end
+
+    def send_get_unique_constraints(request)
+      send_message('get_unique_constraints', Get_unique_constraints_args, :request => request)
+    end
+
+    def recv_get_unique_constraints()
+      result = receive_message(Get_unique_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_unique_constraints failed: unknown result')
+    end
+
+    def get_not_null_constraints(request)
+      send_get_not_null_constraints(request)
+      return recv_get_not_null_constraints()
+    end
+
+    def send_get_not_null_constraints(request)
+      send_message('get_not_null_constraints', Get_not_null_constraints_args, :request => request)
+    end
+
+    def recv_get_not_null_constraints()
+      result = receive_message(Get_not_null_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_not_null_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()
@@ -2833,7 +2899,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)
+        @handler.create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints)
       rescue ::AlreadyExistsException => o1
         result.o1 = o1
       rescue ::InvalidObjectException => o2
@@ -2885,6 +2951,32 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'add_foreign_key', seqid)
     end
 
+    def process_add_unique_constraint(seqid, iprot, oprot)
+      args = read_args(iprot, Add_unique_constraint_args)
+      result = Add_unique_constraint_result.new()
+      begin
+        @handler.add_unique_constraint(args.req)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'add_unique_constraint', seqid)
+    end
+
+    def process_add_not_null_constraint(seqid, iprot, oprot)
+      args = read_args(iprot, Add_not_null_constraint_args)
+      result = Add_not_null_constraint_result.new()
+      begin
+        @handler.add_not_null_constraint(args.req)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'add_not_null_constraint', seqid)
+    end
+
     def process_drop_table(seqid, iprot, oprot)
       args = read_args(iprot, Drop_table_args)
       result = Drop_table_result.new()
@@ -3750,6 +3842,32 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_foreign_keys', seqid)
     end
 
+    def process_get_unique_constraints(seqid, iprot, oprot)
+      args = read_args(iprot, Get_unique_constraints_args)
+      result = Get_unique_constraints_result.new()
+      begin
+        result.success = @handler.get_unique_constraints(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_unique_constraints', seqid)
+    end
+
+    def process_get_not_null_constraints(seqid, iprot, oprot)
+      args = read_args(iprot, Get_not_null_constraints_args)
+      result = Get_not_null_constraints_result.new()
+      begin
+        result.success = @handler.get_not_null_constraints(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_not_null_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()
@@ -5162,11 +5280,15 @@ module ThriftHiveMetastore
     TBL = 1
     PRIMARYKEYS = 2
     FOREIGNKEYS = 3
+    UNIQUECONSTRAINTS = 4
+    NOTNULLCONSTRAINTS = 5
 
     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}}
+      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}}
     }
 
     def struct_fields; FIELDS; end
@@ -5301,6 +5423,74 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Add_unique_constraint_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::AddUniqueConstraintRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_unique_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 Add_not_null_constraint_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::AddNotNullConstraintRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_not_null_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
@@ -7903,6 +8093,78 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_unique_constraints_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::UniqueConstraintsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_unique_constraints_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::UniqueConstraintsResponse},
+      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 Get_not_null_constraints_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::NotNullConstraintsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_not_null_constraints_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::NotNullConstraintsResponse},
+      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/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index d9d50ab..52bfb26 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1367,8 +1367,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private void create_table_core(final RawStore ms, final Table tbl,
+        final EnvironmentContext envContext)
+            throws AlreadyExistsException, MetaException,
+            InvalidObjectException, NoSuchObjectException {
+      create_table_core(ms, tbl, envContext, 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<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
+        List<SQLNotNullConstraint> notNullConstraints)
         throws AlreadyExistsException, MetaException,
         InvalidObjectException, NoSuchObjectException {
       if (!MetaStoreUtils.validateName(tbl.getTableName(), hiveConf)) {
@@ -1453,10 +1461,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             tbl.getParameters().get(hive_metastoreConstants.DDL_TIME) == null) {
           tbl.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
         }
-        if (primaryKeys == null && foreignKeys == null) {
+        if (primaryKeys == null && foreignKeys == null
+                && uniqueConstraints == null && notNullConstraints == null) {
           ms.createTable(tbl);
         } else {
-          ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys);
+          ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys,
+              uniqueConstraints, notNullConstraints);
         }
 
         if (!transactionalListeners.isEmpty()) {
@@ -1500,7 +1510,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       boolean success = false;
       Exception ex = null;
       try {
-        create_table_core(getMS(), tbl, envContext, null, null);
+        create_table_core(getMS(), tbl, envContext);
         success = true;
       } catch (NoSuchObjectException e) {
         ex = e;
@@ -1523,13 +1533,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public void create_table_with_constraints(final Table tbl,
-        final List<SQLPrimaryKey> primaryKeys, final List<SQLForeignKey> foreignKeys)
+        final List<SQLPrimaryKey> primaryKeys, final List<SQLForeignKey> foreignKeys,
+        List<SQLUniqueConstraint> uniqueConstraints,
+        List<SQLNotNullConstraint> notNullConstraints)
         throws AlreadyExistsException, MetaException, InvalidObjectException {
       startFunction("create_table", ": " + tbl.toString());
       boolean success = false;
       Exception ex = null;
       try {
-        create_table_core(getMS(), tbl, null, primaryKeys, foreignKeys);
+        create_table_core(getMS(), tbl, null, primaryKeys, foreignKeys,
+            uniqueConstraints, notNullConstraints);
         success = true;
       } catch (NoSuchObjectException e) {
         ex = e;
@@ -1631,6 +1644,58 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
+    public void add_unique_constraint(AddUniqueConstraintRequest req)
+      throws MetaException, InvalidObjectException {
+      List<SQLUniqueConstraint> uniqueConstraintCols = req.getUniqueConstraintCols();
+      String constraintName = (uniqueConstraintCols != null && uniqueConstraintCols.size() > 0) ?
+              uniqueConstraintCols.get(0).getUk_name() : "null";
+      startFunction("add_unique_constraint", ": " + constraintName);
+      boolean success = false;
+      Exception ex = null;
+      try {
+        getMS().addUniqueConstraints(uniqueConstraintCols);
+        success = true;
+      } 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 {
+        endFunction("add_unique_constraint", success, ex, constraintName);
+      }
+    }
+
+    @Override
+    public void add_not_null_constraint(AddNotNullConstraintRequest req)
+      throws MetaException, InvalidObjectException {
+      List<SQLNotNullConstraint> notNullConstraintCols = req.getNotNullConstraintCols();
+      String constraintName = (notNullConstraintCols != null && notNullConstraintCols.size() > 0) ?
+              notNullConstraintCols.get(0).getNn_name() : "null";
+      startFunction("add_not_null_constraint", ": " + constraintName);
+      boolean success = false;
+      Exception ex = null;
+      try {
+        getMS().addNotNullConstraints(notNullConstraintCols);
+        success = true;
+      } 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 {
+        endFunction("add_not_null_constraint", success, ex, constraintName);
+      }
+    }
+
     private boolean is_table_exists(RawStore ms, String dbname, String name)
         throws MetaException {
       return (ms.getTable(dbname, name) != null);
@@ -6972,6 +7037,56 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
+    public UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request)
+        throws MetaException, NoSuchObjectException, TException {
+      String db_name = request.getDb_name();
+      String tbl_name = request.getTbl_name();
+      startTableFunction("get_unique_constraints", db_name, tbl_name);
+      List<SQLUniqueConstraint> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getUniqueConstraints(db_name, tbl_name);
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof NoSuchObjectException) {
+          throw (NoSuchObjectException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_unique_constraints", ret != null, ex, tbl_name);
+      }
+      return new UniqueConstraintsResponse(ret);
+    }
+
+    @Override
+    public NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest request)
+        throws MetaException, NoSuchObjectException, TException {
+      String db_name = request.getDb_name();
+      String tbl_name = request.getTbl_name();
+      startTableFunction("get_not_null_constraints", db_name, tbl_name);
+      List<SQLNotNullConstraint> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getNotNullConstraints(db_name, tbl_name);
+      } catch (Exception e) {
+        ex = e;
+        if (e instanceof MetaException) {
+          throw (MetaException) e;
+        } else if (e instanceof NoSuchObjectException) {
+          throw (NoSuchObjectException) e;
+        } else {
+          throw newMetaException(e);
+        }
+      } finally {
+        endFunction("get_not_null_constraints", ret != null, ex, tbl_name);
+      }
+      return new NotNullConstraintsResponse(ret);
+    }
+
+    @Override
     public String get_metastore_db_uuid() throws MetaException, TException {
       try {
         return getMS().getMetastoreDbUuid();
@@ -6982,7 +7097,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
   }
 
-
   public static IHMSHandler newRetryingHMSHandler(IHMSHandler baseHandler, HiveConf hiveConf)
       throws MetaException {
     return newRetryingHMSHandler(baseHandler, hiveConf, false);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 3e6add2..0ff4c11 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -752,9 +752,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
 
   @Override
   public void createTableWithConstraints(Table tbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
-    throws AlreadyExistsException, InvalidObjectException,
-    MetaException, NoSuchObjectException, TException {
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
+        throws AlreadyExistsException, InvalidObjectException,
+        MetaException, NoSuchObjectException, TException {
     HiveMetaHook hook = getHook(tbl);
     if (hook != null) {
       hook.preCreateTable(tbl);
@@ -762,7 +764,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     boolean success = false;
     try {
       // Subclasses can override this step (for example, for temporary tables)
-      client.create_table_with_constraints(tbl, primaryKeys, foreignKeys);
+      client.create_table_with_constraints(tbl, primaryKeys, foreignKeys,
+          uniqueConstraints, notNullConstraints);
       if (hook != null) {
         hook.commitCreateTable(tbl);
       }
@@ -792,7 +795,19 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     client.add_foreign_key(new AddForeignKeyRequest(foreignKeyCols));
   }
 
-/**
+  @Override
+  public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_unique_constraint(new AddUniqueConstraintRequest(uniqueConstraintCols));
+  }
+
+  @Override
+  public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_not_null_constraint(new AddNotNullConstraintRequest(notNullConstraintCols));
+  }
+
+  /**
    * @param type
    * @return true or false
    * @throws AlreadyExistsException
@@ -1632,6 +1647,18 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return client.get_foreign_keys(req).getForeignKeys();
   }
 
+  @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_unique_constraints(req).getUniqueConstraints();
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_not_null_constraints(req).getNotNullConstraints();
+  }
+
 
   /** {@inheritDoc} */
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 9c24c23..3663305 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
 import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
@@ -82,7 +83,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
@@ -91,6 +94,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
@@ -1651,9 +1655,17 @@ public interface IMetaStoreClient {
   List<SQLForeignKey> getForeignKeys(ForeignKeysRequest request) throws MetaException,
     NoSuchObjectException, TException;
 
+  List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest request) throws MetaException,
+    NoSuchObjectException, TException;
+
+  List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest request) throws MetaException,
+    NoSuchObjectException, TException;
+
   void createTableWithConstraints(
     org.apache.hadoop.hive.metastore.api.Table tTbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
     throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException;
 
   void dropConstraint(String dbName, String tableName, String constraintName) throws
@@ -1665,6 +1677,12 @@ public interface IMetaStoreClient {
   void addForeignKey(List<SQLForeignKey> foreignKeyCols) throws
   MetaException, NoSuchObjectException, TException;
 
+  void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws
+  MetaException, NoSuchObjectException, TException;
+
+  void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) 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
@@ -1672,4 +1690,5 @@ public interface IMetaStoreClient {
    * @throws TException in case of Thrift errors
    */
   String getMetastoreDbUuid() throws MetaException, TException;
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index df73693..500fba9 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -56,7 +56,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -1918,9 +1920,6 @@ class MetaStoreDirectSql {
       + (parent_db_name == null ? "" : " \"D2\".\"NAME\" = ?") ;
 
     queryText = queryText.trim();
-    if (queryText.endsWith("WHERE")) {
-      queryText = queryText.substring(0, queryText.length()-5);
-    }
     if (queryText.endsWith("AND")) {
       queryText = queryText.substring(0, queryText.length()-3);
     }
@@ -1986,9 +1985,6 @@ class MetaStoreDirectSql {
       + (tbl_name == null ? "" : " \"TBLS\".\"TBL_NAME\" = ? ") ;
 
     queryText = queryText.trim();
-    if (queryText.endsWith("WHERE")) {
-      queryText = queryText.substring(0, queryText.length()-5);
-    }
     if (queryText.endsWith("AND")) {
       queryText = queryText.substring(0, queryText.length()-3);
     }
@@ -2023,4 +2019,107 @@ class MetaStoreDirectSql {
     }
     return ret;
   }
+
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    List<SQLUniqueConstraint> ret = new ArrayList<SQLUniqueConstraint>();
+    String queryText =
+      "SELECT \"DBS\".\"NAME\", \"TBLS\".\"TBL_NAME\", \"COLUMNS_V2\".\"COLUMN_NAME\","
+      + "\"KEY_CONSTRAINTS\".\"POSITION\", "
+      + "\"KEY_CONSTRAINTS\".\"CONSTRAINT_NAME\", \"KEY_CONSTRAINTS\".\"ENABLE_VALIDATE_RELY\" "
+      + " 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\" "
+      + " INNER JOIN \"COLUMNS_V2\" ON \"COLUMNS_V2\".\"CD_ID\" = \"KEY_CONSTRAINTS\".\"PARENT_CD_ID\" AND "
+      + " \"COLUMNS_V2\".\"INTEGER_IDX\" = \"KEY_CONSTRAINTS\".\"PARENT_INTEGER_IDX\" "
+      + " WHERE \"KEY_CONSTRAINTS\".\"CONSTRAINT_TYPE\" = "+ MConstraint.UNIQUE_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);
+    }
+    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[5]);
+          boolean enable = (enableValidateRely & 4) != 0;
+          boolean validate = (enableValidateRely & 2) != 0;
+          boolean rely = (enableValidateRely & 1) != 0;
+        SQLUniqueConstraint currConstraint = new SQLUniqueConstraint(
+          extractSqlString(line[0]),
+          extractSqlString(line[1]),
+          extractSqlString(line[2]),
+          extractSqlInt(line[3]), extractSqlString(line[4]),
+          enable,
+          validate,
+          rely);
+          ret.add(currConstraint);
+      }
+    }
+    return ret;
+  }
+
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    List<SQLNotNullConstraint> ret = new ArrayList<SQLNotNullConstraint>();
+    String queryText =
+      "SELECT \"DBS\".\"NAME\", \"TBLS\".\"TBL_NAME\", \"COLUMNS_V2\".\"COLUMN_NAME\","
+      + "\"KEY_CONSTRAINTS\".\"CONSTRAINT_NAME\", \"KEY_CONSTRAINTS\".\"ENABLE_VALIDATE_RELY\" "
+      + " 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\" "
+      + " INNER JOIN \"COLUMNS_V2\" ON \"COLUMNS_V2\".\"CD_ID\" = \"KEY_CONSTRAINTS\".\"PARENT_CD_ID\" AND "
+      + " \"COLUMNS_V2\".\"INTEGER_IDX\" = \"KEY_CONSTRAINTS\".\"PARENT_INTEGER_IDX\" "
+      + " WHERE \"KEY_CONSTRAINTS\".\"CONSTRAINT_TYPE\" = "+ MConstraint.NOT_NULL_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);
+    }
+    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;
+        SQLNotNullConstraint currConstraint = new SQLNotNullConstraint(
+          extractSqlString(line[0]),
+          extractSqlString(line[1]),
+          extractSqlString(line[2]),
+          extractSqlString(line[3]),
+          enable,
+          validate,
+          rely);
+          ret.add(currConstraint);
+      }
+    }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 28b1e57..b16218d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -60,6 +60,7 @@ import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -108,7 +109,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -970,17 +973,21 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public void createTableWithConstraints(Table tbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
     throws InvalidObjectException, MetaException {
     boolean success = false;
     try {
       openTransaction();
       createTable(tbl);
-      // Add primary keys and foreign keys.
-      // We need not do a deep retrieval of the Table Column Descriptor while persisting the PK/FK
-      // since this transaction involving create table is not yet committed.
+      // Add constraints.
+      // We need not do a deep retrieval of the Table Column Descriptor while persisting the
+      // constraints since this transaction involving create table is not yet committed.
       addPrimaryKeys(primaryKeys, false);
       addForeignKeys(foreignKeys, false);
+      addUniqueConstraints(uniqueConstraints, false);
+      addNotNullConstraints(notNullConstraints, false);
       success = commitTransaction();
     } finally {
       if (!success) {
@@ -3588,32 +3595,39 @@ public class ObjectStore implements RawStore, Configurable {
     String currentConstraintName = null;
 
     for (int i = 0; i < fks.size(); i++) {
-      AttachedMTableInfo nParentTable = getMTable(fks.get(i).getPktable_db(), fks.get(i).getPktable_name(), retrieveCD);
+      final String pkTableDB = HiveStringUtils.normalizeIdentifier(fks.get(i).getPktable_db());
+      final String pkTableName = HiveStringUtils.normalizeIdentifier(fks.get(i).getPktable_name());
+      final String pkColumnName =HiveStringUtils.normalizeIdentifier(fks.get(i).getPkcolumn_name());
+      final String fkTableDB = HiveStringUtils.normalizeIdentifier(fks.get(i).getFktable_db());
+      final String fkTableName = HiveStringUtils.normalizeIdentifier(fks.get(i).getFktable_name());
+      final String fkColumnName = HiveStringUtils.normalizeIdentifier(fks.get(i).getFkcolumn_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(pkTableDB, pkTableName, retrieveCD);
       MTable parentTable = nParentTable.mtbl;
       if (parentTable == null) {
-        throw new InvalidObjectException("Parent table not found: " + fks.get(i).getPktable_name());
+        throw new InvalidObjectException("Parent table not found: " + pkTableName);
       }
 
-      AttachedMTableInfo nChildTable = getMTable(fks.get(i).getFktable_db(), fks.get(i).getFktable_name(), retrieveCD);
+      AttachedMTableInfo nChildTable = getMTable(fkTableDB, fkTableName, retrieveCD);
       MTable childTable = nChildTable.mtbl;
       if (childTable == null) {
-        throw new InvalidObjectException("Child table not found: " + fks.get(i).getFktable_name());
+        throw new InvalidObjectException("Child table not found: " + fkTableName);
       }
 
       MColumnDescriptor parentCD = retrieveCD ? nParentTable.mcd : parentTable.getSd().getCD();
       List<MFieldSchema> parentCols = parentCD == null ? null : parentCD.getCols();
-      int parentIntegerIndex =
-        getColumnIndexFromTableColumns(parentCols, fks.get(i).getPkcolumn_name());
+      int parentIntegerIndex = getColumnIndexFromTableColumns(parentCols, pkColumnName);
       if (parentIntegerIndex == -1) {
-        throw new InvalidObjectException("Parent column not found: " + fks.get(i).getPkcolumn_name());
+        throw new InvalidObjectException("Parent column not found: " + pkColumnName);
       }
 
       MColumnDescriptor childCD = retrieveCD ? nChildTable.mcd : childTable.getSd().getCD();
       List<MFieldSchema> childCols = childCD.getCols();
-      int childIntegerIndex =
-        getColumnIndexFromTableColumns(childCols, fks.get(i).getFkcolumn_name());
+      int childIntegerIndex = getColumnIndexFromTableColumns(childCols, fkColumnName);
       if (childIntegerIndex == -1) {
-        throw new InvalidObjectException("Child column not found: " + fks.get(i).getFkcolumn_name());
+        throw new InvalidObjectException("Child column not found: " + fkColumnName);
       }
 
       if (fks.get(i).getFk_name() == null) {
@@ -3625,12 +3639,11 @@ public class ObjectStore implements RawStore, Configurable {
         // However, this scenario can be ignored for practical purposes because of
         // the uniqueness of the generated constraint name.
         if (fks.get(i).getKey_seq() == 1) {
-          currentConstraintName = generateConstraintName(fks.get(i).getFktable_db(), fks.get(i).getFktable_name(),
-            fks.get(i).getPktable_db(), fks.get(i).getPktable_name(),
-            fks.get(i).getPkcolumn_name(), fks.get(i).getFkcolumn_name(), "fk");
+          currentConstraintName = generateConstraintName(
+            fkTableDB, fkTableName, pkTableDB, pkTableName, pkColumnName, fkColumnName, "fk");
         }
       } else {
-        currentConstraintName = fks.get(i).getFk_name();
+        currentConstraintName = HiveStringUtils.normalizeIdentifier(fks.get(i).getFk_name());
       }
       Integer updateRule = fks.get(i).getUpdate_rule();
       Integer deleteRule = fks.get(i).getDelete_rule();
@@ -3667,19 +3680,24 @@ public class ObjectStore implements RawStore, Configurable {
     String constraintName = null;
 
     for (int i = 0; i < pks.size(); i++) {
-      AttachedMTableInfo nParentTable =
-        getMTable(pks.get(i).getTable_db(), pks.get(i).getTable_name(), retrieveCD);
+      final String tableDB = HiveStringUtils.normalizeIdentifier(pks.get(i).getTable_db());
+      final String tableName = HiveStringUtils.normalizeIdentifier(pks.get(i).getTable_name());
+      final String columnName = HiveStringUtils.normalizeIdentifier(pks.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: " + pks.get(i).getTable_name());
+        throw new InvalidObjectException("Parent table not found: " + tableName);
       }
 
       MColumnDescriptor parentCD = retrieveCD ? nParentTable.mcd : parentTable.getSd().getCD();
       int parentIntegerIndex =
-        getColumnIndexFromTableColumns(parentCD == null ? null : parentCD.getCols(), pks.get(i).getColumn_name());
+        getColumnIndexFromTableColumns(parentCD == null ? null : parentCD.getCols(), columnName);
 
       if (parentIntegerIndex == -1) {
-        throw new InvalidObjectException("Parent column not found: " + pks.get(i).getColumn_name());
+        throw new InvalidObjectException("Parent column not found: " + columnName);
       }
       if (getPrimaryKeyConstraintName(
           parentTable.getDatabase().getName(), parentTable.getTableName()) != null) {
@@ -3688,11 +3706,10 @@ public class ObjectStore implements RawStore, Configurable {
       }
       if (pks.get(i).getPk_name() == null) {
         if (pks.get(i).getKey_seq() == 1) {
-          constraintName = generateConstraintName(pks.get(i).getTable_db(), pks.get(i).getTable_name(),
-            pks.get(i).getColumn_name(), "pk");
+          constraintName = generateConstraintName(tableDB, tableName, columnName, "pk");
         }
       } else {
-        constraintName = pks.get(i).getPk_name();
+        constraintName = HiveStringUtils.normalizeIdentifier(pks.get(i).getPk_name());
       }
 
       int enableValidateRely = (pks.get(i).isEnable_cstr() ? 4 : 0) +
@@ -3716,6 +3733,120 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks)
+          throws InvalidObjectException, MetaException {
+    addUniqueConstraints(uks, true);
+  }
+
+  private void addUniqueConstraints(List<SQLUniqueConstraint> uks, boolean retrieveCD)
+          throws InvalidObjectException, MetaException {
+    List<MConstraint> cstrs = new ArrayList<MConstraint>();
+    String constraintName = null;
+
+    for (int i = 0; i < uks.size(); i++) {
+      final String tableDB = HiveStringUtils.normalizeIdentifier(uks.get(i).getTable_db());
+      final String tableName = HiveStringUtils.normalizeIdentifier(uks.get(i).getTable_name());
+      final String columnName = HiveStringUtils.normalizeIdentifier(uks.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) {
+        throw new InvalidObjectException("Parent column not found: " + columnName);
+      }
+      if (uks.get(i).getUk_name() == null) {
+        if (uks.get(i).getKey_seq() == 1) {
+            constraintName = generateConstraintName(tableDB, tableName, columnName, "uk");
+        }
+      } else {
+        constraintName = HiveStringUtils.normalizeIdentifier(uks.get(i).getUk_name());
+      }
+
+      int enableValidateRely = (uks.get(i).isEnable_cstr() ? 4 : 0) +
+          (uks.get(i).isValidate_cstr() ? 2 : 0) + (uks.get(i).isRely_cstr() ? 1 : 0);
+      MConstraint muk = new MConstraint(
+        constraintName,
+        MConstraint.UNIQUE_CONSTRAINT,
+        uks.get(i).getKey_seq(),
+        null,
+        null,
+        enableValidateRely,
+        parentTable,
+        null,
+        parentCD,
+        null,
+        null,
+        parentIntegerIndex);
+      cstrs.add(muk);
+    }
+    pm.makePersistentAll(cstrs);
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns)
+          throws InvalidObjectException, MetaException {
+    addNotNullConstraints(nns, true);
+  }
+
+  private void addNotNullConstraints(List<SQLNotNullConstraint> nns, boolean retrieveCD)
+          throws InvalidObjectException, MetaException {
+    List<MConstraint> cstrs = new ArrayList<MConstraint>();
+    String constraintName = null;
+
+    for (int i = 0; i < nns.size(); i++) {
+      final String tableDB = HiveStringUtils.normalizeIdentifier(nns.get(i).getTable_db());
+      final String tableName = HiveStringUtils.normalizeIdentifier(nns.get(i).getTable_name());
+      final String columnName = HiveStringUtils.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) {
+        throw new InvalidObjectException("Parent column not found: " + columnName);
+      }
+      if (nns.get(i).getNn_name() == null) {
+        constraintName = generateConstraintName(tableDB, tableName, columnName, "nn");
+      } else {
+        constraintName = HiveStringUtils.normalizeIdentifier(nns.get(i).getNn_name());
+      }
+
+      int enableValidateRely = (nns.get(i).isEnable_cstr() ? 4 : 0) +
+          (nns.get(i).isValidate_cstr() ? 2 : 0) + (nns.get(i).isRely_cstr() ? 1 : 0);
+      MConstraint muk = new MConstraint(
+        constraintName,
+        MConstraint.NOT_NULL_CONSTRAINT,
+        1, // Not null constraint should reference a single column
+        null,
+        null,
+        enableValidateRely,
+        parentTable,
+        null,
+        parentCD,
+        null,
+        null,
+        parentIntegerIndex);
+      cstrs.add(muk);
+    }
+    pm.makePersistentAll(cstrs);
+  }
+
+  @Override
   public boolean addIndex(Index index) throws InvalidObjectException,
       MetaException {
     boolean commited = false;
@@ -8308,7 +8439,7 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       return getPrimaryKeysInternal(db_name, tbl_name, true, true);
     } catch (NoSuchObjectException e) {
-      throw new MetaException(e.getMessage());
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
     }
   }
 
@@ -8397,7 +8528,7 @@ public class ObjectStore implements RawStore, Configurable {
       return getForeignKeysInternal(parent_db_name,
         parent_tbl_name, foreign_db_name, foreign_tbl_name, true, true);
     } catch (NoSuchObjectException e) {
-      throw new MetaException(e.getMessage());
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
     }
   }
 
@@ -8514,6 +8645,143 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    try {
+      return getUniqueConstraintsInternal(db_name, tbl_name, true, true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
+    }
+  }
+
+  protected List<SQLUniqueConstraint> getUniqueConstraintsInternal(final String db_name_input,
+      final String tbl_name_input, boolean allowSql, boolean allowJdo)
+          throws MetaException, NoSuchObjectException {
+    final String db_name = HiveStringUtils.normalizeIdentifier(db_name_input);
+    final String tbl_name = HiveStringUtils.normalizeIdentifier(tbl_name_input);
+    return new GetListHelper<SQLUniqueConstraint>(db_name, tbl_name, allowSql, allowJdo) {
+
+      @Override
+      protected List<SQLUniqueConstraint> getSqlResult(GetHelper<List<SQLUniqueConstraint>> ctx)
+              throws MetaException {
+        return directSql.getUniqueConstraints(db_name, tbl_name);
+      }
+
+      @Override
+      protected List<SQLUniqueConstraint> getJdoResult(GetHelper<List<SQLUniqueConstraint>> ctx)
+              throws MetaException, NoSuchObjectException {
+        return getUniqueConstraintsViaJdo(db_name, tbl_name);
+      }
+    }.run(false);
+  }
+
+  private List<SQLUniqueConstraint> getUniqueConstraintsViaJdo(String db_name, String tbl_name)
+          throws MetaException {
+    boolean commited = false;
+    List<SQLUniqueConstraint> uniqueConstraints = null;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MConstraint.class,
+        "parentTable.tableName == tbl_name && parentTable.database.name == db_name &&"
+        + " constraintType == MConstraint.UNIQUE_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);
+      uniqueConstraints = new ArrayList<SQLUniqueConstraint>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currPK = (MConstraint) i.next();
+        int enableValidateRely = currPK.getEnableValidateRely();
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        uniqueConstraints.add(new SQLUniqueConstraint(db_name,
+         tbl_name,
+         currPK.getParentColumn().getCols().get(currPK.getParentIntegerIndex()).getName(),
+         currPK.getPosition(),
+         currPK.getConstraintName(), enable, validate, rely));
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return uniqueConstraints;
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    try {
+      return getNotNullConstraintsInternal(db_name, tbl_name, true, true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
+    }
+  }
+
+  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 = HiveStringUtils.normalizeIdentifier(db_name_input);
+    final String tbl_name = HiveStringUtils.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 {
+        return directSql.getNotNullConstraints(db_name, tbl_name);
+      }
+
+      @Override
+      protected List<SQLNotNullConstraint> getJdoResult(GetHelper<List<SQLNotNullConstraint>> ctx)
+              throws MetaException, NoSuchObjectException {
+        return getNotNullConstraintsViaJdo(db_name, tbl_name);
+      }
+    }.run(false);
+  }
+
+  private List<SQLNotNullConstraint> getNotNullConstraintsViaJdo(String db_name, String tbl_name)
+          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");
+      query.declareParameters("java.lang.String tbl_name, java.lang.String db_name");
+      Collection<?> constraints = (Collection<?>) query.execute(tbl_name, db_name);
+      pm.retrieveAll(constraints);
+      notNullConstraints = new ArrayList<SQLNotNullConstraint>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currPK = (MConstraint) i.next();
+        int enableValidateRely = currPK.getEnableValidateRely();
+        boolean enable = (enableValidateRely & 4) != 0;
+        boolean validate = (enableValidateRely & 2) != 0;
+        boolean rely = (enableValidateRely & 1) != 0;
+        notNullConstraints.add(new SQLNotNullConstraint(db_name,
+         tbl_name,
+         currPK.getParentColumn().getCols().get(currPK.getParentIntegerIndex()).getName(),
+         currPK.getConstraintName(), enable, validate, rely));
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return notNullConstraints;
+  }
+
+  @Override
   public void dropConstraint(String dbName, String tableName,
     String constraintName) throws NoSuchObjectException {
     boolean success = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index 964ffb2..67506f2 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -54,7 +54,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -702,8 +704,15 @@ public interface RawStore extends Configurable {
     String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
     throws MetaException;
 
+  public abstract List<SQLUniqueConstraint> getUniqueConstraints(String db_name,
+    String tbl_name) throws MetaException;
+
+  public abstract List<SQLNotNullConstraint> getNotNullConstraints(String db_name,
+    String tbl_name) throws MetaException;
+
   void createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
-    List<SQLForeignKey> foreignKeys) throws InvalidObjectException, MetaException;
+    List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints) throws InvalidObjectException, MetaException;
 
   void dropConstraint(String dbName, String tableName, String constraintName) throws NoSuchObjectException;
 
@@ -711,6 +720,10 @@ public interface RawStore extends Configurable {
 
   void addForeignKeys(List<SQLForeignKey> fks) throws InvalidObjectException, MetaException;
 
+  void addUniqueConstraints(List<SQLUniqueConstraint> uks) throws InvalidObjectException, MetaException;
+
+  void addNotNullConstraints(List<SQLNotNullConstraint> 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/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 78aab91..f00f08f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -70,7 +70,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
@@ -1846,11 +1848,28 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    // TODO constraintCache
+    return rawStore.getUniqueConstraints(db_name, tbl_name);
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    // TODO constraintCache
+    return rawStore.getNotNullConstraints(db_name, tbl_name);
+  }
+
+  @Override
   public void createTableWithConstraints(Table tbl,
-      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints,
+      List<SQLNotNullConstraint> notNullConstraints)
       throws InvalidObjectException, MetaException {
     // TODO constraintCache
-    rawStore.createTableWithConstraints(tbl, primaryKeys, foreignKeys);
+    rawStore.createTableWithConstraints(tbl, primaryKeys, foreignKeys,
+            uniqueConstraints, notNullConstraints);
     SharedCache.addTableToCache(HiveStringUtils.normalizeIdentifier(tbl.getDbName()),
         HiveStringUtils.normalizeIdentifier(tbl.getTableName()), tbl);
   }
@@ -1877,6 +1896,20 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks)
+      throws InvalidObjectException, MetaException {
+    // TODO constraintCache
+    rawStore.addUniqueConstraints(uks);
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns)
+      throws InvalidObjectException, MetaException {
+    // TODO constraintCache
+    rawStore.addNotNullConstraints(nns);
+  }
+
+  @Override
   public Map<String, List<ColumnStatisticsObj>> getColStatsForTablePartitions(String dbName,
       String tableName) throws MetaException, NoSuchObjectException {
     return rawStore.getColStatsForTablePartitions(dbName, tableName);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index e687a69..d711805 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -24,7 +24,9 @@ import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -147,6 +149,8 @@ public class HBaseReadWrite implements MetadataStore {
   private final static byte[] MASTER_KEY_COL = "mk".getBytes(HBaseUtils.ENCODING);
   private final static byte[] PRIMARY_KEY_COL = "pk".getBytes(HBaseUtils.ENCODING);
   private final static byte[] FOREIGN_KEY_COL = "fk".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] UNIQUE_CONSTRAINT_COL = "uk".getBytes(HBaseUtils.ENCODING);
+  private final static byte[] NOT_NULL_CONSTRAINT_COL = "nn".getBytes(HBaseUtils.ENCODING);
   private final static byte[] GLOBAL_PRIVS_KEY = "gp".getBytes(HBaseUtils.ENCODING);
   private final static byte[] SEQUENCES_KEY = "seq".getBytes(HBaseUtils.ENCODING);
   private final static int TABLES_TO_CACHE = 10;
@@ -2550,7 +2554,7 @@ public class HBaseReadWrite implements MetadataStore {
   }
 
   /**********************************************************************************************
-   * Constraints (pk/fk) related methods
+   * Constraints related methods
    *********************************************************************************************/
 
   /**
@@ -2582,6 +2586,34 @@ public class HBaseReadWrite implements MetadataStore {
   }
 
   /**
+   * Fetch a unique constraint
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @return List of unique constraints objects
+   * @throws IOException if there's a read error
+   */
+  List<SQLUniqueConstraint> getUniqueConstraint(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    byte[] serialized = read(TABLE_TABLE, key, CATALOG_CF, UNIQUE_CONSTRAINT_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializeUniqueConstraint(dbName, tableName, serialized);
+  }
+
+  /**
+   * Fetch a not null constraint
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @return List of not null constraints objects
+   * @throws IOException if there's a read error
+   */
+  List<SQLNotNullConstraint> getNotNullConstraint(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    byte[] serialized = read(TABLE_TABLE, key, CATALOG_CF, NOT_NULL_CONSTRAINT_COL);
+    if (serialized == null) return null;
+    return HBaseUtils.deserializeNotNullConstraint(dbName, tableName, serialized);
+  }
+
+  /**
    * Create a primary key on a table.
    * @param pk Primary key for this table
    * @throws IOException if unable to write the data to the store.
@@ -2605,6 +2637,26 @@ public class HBaseReadWrite implements MetadataStore {
   }
 
   /**
+   * Create one or more unique constraints on a table.
+   * @param uks Unique constraints for this table
+   * @throws IOException if unable to write the data to the store.
+   */
+  void putUniqueConstraints(List<SQLUniqueConstraint> uks) throws IOException {
+    byte[][] serialized = HBaseUtils.serializeUniqueConstraints(uks);
+    store(TABLE_TABLE, serialized[0], CATALOG_CF, UNIQUE_CONSTRAINT_COL, serialized[1]);
+  }
+
+  /**
+   * Create one or more not null constraints on a table.
+   * @param nns Not null constraints for this table
+   * @throws IOException if unable to write the data to the store.
+   */
+  void putNotNullConstraints(List<SQLNotNullConstraint> nns) throws IOException {
+    byte[][] serialized = HBaseUtils.serializeNotNullConstraints(nns);
+    store(TABLE_TABLE, serialized[0], CATALOG_CF, NOT_NULL_CONSTRAINT_COL, serialized[1]);
+  }
+
+  /**
    * Drop the primary key from a table.
    * @param dbName database the table is in
    * @param tableName table name
@@ -2629,6 +2681,28 @@ public class HBaseReadWrite implements MetadataStore {
     delete(TABLE_TABLE, key, CATALOG_CF, FOREIGN_KEY_COL);
   }
 
+  /**
+   * Drop the unique constraint from a table.
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @throws IOException if unable to delete from the store
+   */
+  void deleteUniqueConstraint(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    delete(TABLE_TABLE, key, CATALOG_CF, UNIQUE_CONSTRAINT_COL);
+  }
+
+  /**
+   * Drop the not null constraint from a table.
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @throws IOException if unable to delete from the store
+   */
+  void deleteNotNullConstraint(String dbName, String tableName) throws IOException {
+    byte[] key = HBaseUtils.buildKey(dbName, tableName);
+    delete(TABLE_TABLE, key, CATALOG_CF, NOT_NULL_CONSTRAINT_COL);
+  }
+
   /**********************************************************************************************
    * Cache methods
    *********************************************************************************************/

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index a7681dd..5a45051 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheLoader;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -65,7 +64,9 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -2739,15 +2740,56 @@ public class HBaseStore implements RawStore {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    db_name = HiveStringUtils.normalizeIdentifier(db_name);
+    tbl_name = HiveStringUtils.normalizeIdentifier(tbl_name);
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLUniqueConstraint> uk = getHBase().getUniqueConstraint(db_name, tbl_name);
+      commit = true;
+      return uk;
+    } catch (IOException e) {
+      LOG.error("Unable to get unique constraint", e);
+      throw new MetaException("Error reading db " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+          throws MetaException {
+    db_name = HiveStringUtils.normalizeIdentifier(db_name);
+    tbl_name = HiveStringUtils.normalizeIdentifier(tbl_name);
+    boolean commit = false;
+    openTransaction();
+    try {
+      List<SQLNotNullConstraint> nn = getHBase().getNotNullConstraint(db_name, tbl_name);
+      commit = true;
+      return nn;
+    } catch (IOException e) {
+      LOG.error("Unable to get not null constraint", e);
+      throw new MetaException("Error reading db " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
+  @Override
   public void createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
-                                         List<SQLForeignKey> foreignKeys)
-      throws InvalidObjectException, MetaException {
+      List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
+      List<SQLNotNullConstraint> notNullConstraints)
+          throws InvalidObjectException, MetaException {
     boolean commit = false;
     openTransaction();
     try {
       createTable(tbl);
       if (primaryKeys != null) addPrimaryKeys(primaryKeys);
       if (foreignKeys != null) addForeignKeys(foreignKeys);
+      if (uniqueConstraints != null) addUniqueConstraints(uniqueConstraints);
+      if (notNullConstraints != null) addNotNullConstraints(notNullConstraints);
       commit = true;
     } finally {
       commitOrRoleBack(commit);
@@ -2787,6 +2829,20 @@ public class HBaseStore implements RawStore {
         return;
       }
 
+      List<SQLUniqueConstraint> uk = getHBase().getUniqueConstraint(dbName, tableName);
+      if (uk != null && uk.size() > 0 && uk.get(0).getUk_name().equals(constraintName)) {
+        getHBase().deleteUniqueConstraint(dbName, tableName);
+        commit = true;
+        return;
+      }
+
+      List<SQLNotNullConstraint> nn = getHBase().getNotNullConstraint(dbName, tableName);
+      if (nn != null && nn.size() > 0 && nn.get(0).getNn_name().equals(constraintName)) {
+        getHBase().deleteNotNullConstraint(dbName, tableName);
+        commit = true;
+        return;
+      }
+
       commit = true;
       throw new NoSuchObjectException("Unable to find constraint named " + constraintName +
         " on table " + tableNameForErrorMsg(dbName, tableName));
@@ -2853,6 +2909,47 @@ public class HBaseStore implements RawStore {
     }
   }
 
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks) throws InvalidObjectException, MetaException {
+    boolean commit = false;
+    for (SQLUniqueConstraint uk : uks) {
+      uk.setTable_db(HiveStringUtils.normalizeIdentifier(uk.getTable_db()));
+      uk.setTable_name(HiveStringUtils.normalizeIdentifier(uk.getTable_name()));
+      uk.setColumn_name(HiveStringUtils.normalizeIdentifier(uk.getColumn_name()));
+      uk.setUk_name(HiveStringUtils.normalizeIdentifier(uk.getUk_name()));
+    }
+    openTransaction();
+    try {
+      getHBase().putUniqueConstraints(uks);
+      commit = true;
+    } catch (IOException e) {
+      LOG.error("Error writing unique constraints", e);
+      throw new MetaException("Error writing unique constraints: " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns) throws InvalidObjectException, MetaException {
+    boolean commit = false;
+    for (SQLNotNullConstraint nn : nns) {
+      nn.setTable_db(HiveStringUtils.normalizeIdentifier(nn.getTable_db()));
+      nn.setTable_name(HiveStringUtils.normalizeIdentifier(nn.getTable_name()));
+      nn.setColumn_name(HiveStringUtils.normalizeIdentifier(nn.getColumn_name()));
+      nn.setNn_name(HiveStringUtils.normalizeIdentifier(nn.getNn_name()));
+    }
+    openTransaction();
+    try {
+      getHBase().putNotNullConstraints(nns);
+      commit = true;
+    } catch (IOException e) {
+      LOG.error("Error writing not null constraints", e);
+      throw new MetaException("Error writing not null constraints: " + e.getMessage());
+    } finally {
+      commitOrRoleBack(commit);
+    }
+  }
+
   @Override
   public Map<String, List<ColumnStatisticsObj>> getColStatsForTablePartitions(String dbName,
       String tableName) throws MetaException, NoSuchObjectException {

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 64082e8..6b7eb9e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -37,7 +37,6 @@ import java.util.TreeSet;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
@@ -63,7 +62,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -1586,6 +1587,99 @@ public class HBaseUtils {
     return result;
   }
 
+  /**
+   * Serialize the unique constraint(s) for a table.
+   * @param uks Unique constraint columns.  These may belong to multiple unique constraints.
+   * @return two byte arrays, first contains the key, the second the serialized value.
+   */
+  static byte[][] serializeUniqueConstraints(List<SQLUniqueConstraint> uks) {
+    // First, figure out the dbName and tableName.  We expect this to match for all list entries.
+    byte[][] result = new byte[2][];
+    String dbName = uks.get(0).getTable_db();
+    String tableName = uks.get(0).getTable_name();
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(dbName),
+        HiveStringUtils.normalizeIdentifier(tableName));
+
+    HbaseMetastoreProto.UniqueConstraints.Builder builder =
+        HbaseMetastoreProto.UniqueConstraints.newBuilder();
+
+    // Encode any foreign keys we find.  This can be complex because there may be more than
+    // one foreign key in here, so we need to detect that.
+    Map<String, HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.Builder> ukBuilders = new HashMap<>();
+
+    for (SQLUniqueConstraint ukcol : uks) {
+      HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.Builder ukBuilder =
+          ukBuilders.get(ukcol.getUk_name());
+      if (ukBuilder == null) {
+        // We haven't seen this key before, so add it
+        ukBuilder = HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.newBuilder();
+        ukBuilder.setUkName(ukcol.getUk_name());
+        ukBuilder.setEnableConstraint(ukcol.isEnable_cstr());
+        ukBuilder.setValidateConstraint(ukcol.isValidate_cstr());
+        ukBuilder.setRelyConstraint(ukcol.isRely_cstr());
+        ukBuilders.put(ukcol.getUk_name(), ukBuilder);
+      }
+      assert dbName.equals(ukcol.getTable_db()) : "You switched databases on me!";
+      assert tableName.equals(ukcol.getTable_name()) : "You switched tables on me!";
+      HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder ukColBuilder =
+          HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.newBuilder();
+      ukColBuilder.setColumnName(ukcol.getColumn_name());
+      ukColBuilder.setKeySeq(ukcol.getKey_seq());
+      ukBuilder.addCols(ukColBuilder);
+    }
+    for (HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.Builder ukBuilder : ukBuilders.values()) {
+      builder.addUks(ukBuilder);
+    }
+    result[1] = builder.build().toByteArray();
+    return result;
+  }
+
+  /**
+   * Serialize the not null constraint(s) for a table.
+   * @param nns Not null constraint columns.  These may belong to multiple constraints.
+   * @return two byte arrays, first contains the constraint, the second the serialized value.
+   */
+  static byte[][] serializeNotNullConstraints(List<SQLNotNullConstraint> nns) {
+    // First, figure out the dbName and tableName.  We expect this to match for all list entries.
+    byte[][] result = new byte[2][];
+    String dbName = nns.get(0).getTable_db();
+    String tableName = nns.get(0).getTable_name();
+    result[0] = buildKey(HiveStringUtils.normalizeIdentifier(dbName),
+        HiveStringUtils.normalizeIdentifier(tableName));
+
+    HbaseMetastoreProto.NotNullConstraints.Builder builder =
+        HbaseMetastoreProto.NotNullConstraints.newBuilder();
+
+    // Encode any foreign keys we find.  This can be complex because there may be more than
+    // one foreign key in here, so we need to detect that.
+    Map<String, HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.Builder> nnBuilders = new HashMap<>();
+
+    for (SQLNotNullConstraint nncol : nns) {
+      HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.Builder nnBuilder =
+          nnBuilders.get(nncol.getNn_name());
+      if (nnBuilder == null) {
+        // We haven't seen this key before, so add it
+        nnBuilder = HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.newBuilder();
+        nnBuilder.setNnName(nncol.getNn_name());
+        nnBuilder.setEnableConstraint(nncol.isEnable_cstr());
+        nnBuilder.setValidateConstraint(nncol.isValidate_cstr());
+        nnBuilder.setRelyConstraint(nncol.isRely_cstr());
+        nnBuilders.put(nncol.getNn_name(), nnBuilder);
+      }
+      assert dbName.equals(nncol.getTable_db()) : "You switched databases on me!";
+      assert tableName.equals(nncol.getTable_name()) : "You switched tables on me!";
+      HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.NotNullConstraintColumn.Builder nnColBuilder =
+          HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.NotNullConstraintColumn.newBuilder();
+      nnColBuilder.setColumnName(nncol.getColumn_name());
+      nnBuilder.addCols(nnColBuilder);
+    }
+    for (HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.Builder nnBuilder : nnBuilders.values()) {
+      builder.addNns(nnBuilder);
+    }
+    result[1] = builder.build().toByteArray();
+    return result;
+  }
+
   static List<SQLPrimaryKey> deserializePrimaryKey(String dbName, String tableName, byte[] value)
       throws InvalidProtocolBufferException {
     HbaseMetastoreProto.PrimaryKey proto = HbaseMetastoreProto.PrimaryKey.parseFrom(value);
@@ -1599,6 +1693,41 @@ public class HBaseUtils {
     return result;
   }
 
+  static List<SQLUniqueConstraint> deserializeUniqueConstraint(String dbName, String tableName, byte[] value)
+      throws InvalidProtocolBufferException {
+    List<SQLUniqueConstraint> result = new ArrayList<>();
+    HbaseMetastoreProto.UniqueConstraints protoConstraints =
+        HbaseMetastoreProto.UniqueConstraints.parseFrom(value);
+
+    for (HbaseMetastoreProto.UniqueConstraints.UniqueConstraint proto : protoConstraints.getUksList()) {
+      for (HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn protoUkCol :
+          proto.getColsList()) {
+        result.add(new SQLUniqueConstraint(dbName, tableName, protoUkCol.getColumnName(),
+            protoUkCol.getKeySeq(),
+            proto.getUkName(), proto.getEnableConstraint(),
+            proto.getValidateConstraint(), proto.getRelyConstraint()));
+      }
+    }
+    return result;
+  }
+
+  static List<SQLNotNullConstraint> deserializeNotNullConstraint(String dbName, String tableName, byte[] value)
+      throws InvalidProtocolBufferException {
+    List<SQLNotNullConstraint> result = new ArrayList<>();
+    HbaseMetastoreProto.NotNullConstraints protoConstraints =
+        HbaseMetastoreProto.NotNullConstraints.parseFrom(value);
+
+    for (HbaseMetastoreProto.NotNullConstraints.NotNullConstraint proto : protoConstraints.getNnsList()) {
+      for (HbaseMetastoreProto.NotNullConstraints.NotNullConstraint.NotNullConstraintColumn protoNnCol :
+          proto.getColsList()) {
+        result.add(new SQLNotNullConstraint(dbName, tableName, protoNnCol.getColumnName(),
+            proto.getNnName(), proto.getEnableConstraint(),
+            proto.getValidateConstraint(), proto.getRelyConstraint()));
+      }
+    }
+    return result;
+  }
+
   static List<SQLForeignKey> deserializeForeignKeys(String dbName, String tableName, byte[] value)
       throws InvalidProtocolBufferException {
     List<SQLForeignKey> result = new ArrayList<>();


[07/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/test/results/clientpositive/llap/sysdb.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/sysdb.q.out b/ql/src/test/results/clientpositive/llap/sysdb.q.out
index 0ddc373..e6f408b 100644
--- a/ql/src/test/results/clientpositive/llap/sysdb.q.out
+++ b/ql/src/test/results/clientpositive/llap/sysdb.q.out
@@ -146,7 +146,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
   `SD_ID` bigint,
   `BUCKET_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -166,7 +166,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
   `SD_ID` bigint,
   `BUCKET_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -184,7 +184,7 @@ POSTHOOK: Output: SYS@BUCKETING_COLS
 POSTHOOK: Output: database:sys
 PREHOOK: query: CREATE TABLE IF NOT EXISTS `CDS` (
   `CD_ID` bigint,
-  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -200,7 +200,7 @@ PREHOOK: Output: SYS@CDS
 PREHOOK: Output: database:sys
 POSTHOOK: query: CREATE TABLE IF NOT EXISTS `CDS` (
   `CD_ID` bigint,
-  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -220,7 +220,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
   `COLUMN_NAME` string,
   `TYPE_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -244,7 +244,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
   `COLUMN_NAME` string,
   `TYPE_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -266,7 +266,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
   `DB_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -286,7 +286,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
   `DB_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -308,7 +308,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `DBS` (
   `NAME` string,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -332,7 +332,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DBS` (
   `NAME` string,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -360,7 +360,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `DB_PRIV` string,
-  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -392,7 +392,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `DB_PRIV` string,
-  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -423,7 +423,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `USER_PRIV` string,
-  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -453,7 +453,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `USER_PRIV` string,
-  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -484,7 +484,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `IDXS` (
   `LAST_ACCESS_TIME` int,
   `ORIG_TBL_ID` bigint,
   `SD_ID` bigint,
-  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -516,7 +516,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `IDXS` (
   `LAST_ACCESS_TIME` int,
   `ORIG_TBL_ID` bigint,
   `SD_ID` bigint,
-  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -542,7 +542,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
   `INDEX_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -562,7 +562,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
   `INDEX_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -585,7 +585,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITIONS` (
   `PART_NAME` string,
   `SD_ID` bigint,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -611,7 +611,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITIONS` (
   `PART_NAME` string,
   `SD_ID` bigint,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -636,7 +636,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
   `PKEY_NAME` string,
   `PKEY_TYPE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -660,7 +660,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
   `PKEY_NAME` string,
   `PKEY_TYPE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -682,7 +682,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
   `PART_ID` bigint,
   `PART_KEY_VAL` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -702,7 +702,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
   `PART_ID` bigint,
   `PART_KEY_VAL` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -722,7 +722,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
   `PART_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -742,7 +742,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
   `PART_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -769,7 +769,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_COL_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -803,7 +803,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_COL_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -836,7 +836,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -868,7 +868,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -895,7 +895,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `ROLES` (
   `CREATE_TIME` int,
   `OWNER_NAME` string,
   `ROLE_NAME` string,
-  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -917,7 +917,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `ROLES` (
   `CREATE_TIME` int,
   `OWNER_NAME` string,
   `ROLE_NAME` string,
-  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -943,7 +943,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `ROLE_ID` bigint,
-  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -973,7 +973,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `ROLE_ID` bigint,
-  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1004,7 +1004,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SDS` (
   `NUM_BUCKETS` int,
   `OUTPUT_FORMAT` string,
   `SERDE_ID` bigint,
-  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1036,7 +1036,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SDS` (
   `NUM_BUCKETS` int,
   `OUTPUT_FORMAT` string,
   `SERDE_ID` bigint,
-  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1062,7 +1062,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
   `SD_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1082,7 +1082,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
   `SD_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1101,7 +1101,7 @@ POSTHOOK: Output: database:sys
 PREHOOK: query: CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
   `SEQUENCE_NAME` string,
   `NEXT_VAL` bigint,
-  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1119,7 +1119,7 @@ PREHOOK: Output: database:sys
 POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
   `SEQUENCE_NAME` string,
   `NEXT_VAL` bigint,
-  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1138,7 +1138,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SERDES` (
   `SERDE_ID` bigint,
   `NAME` string,
   `SLIB` string,
-  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1158,7 +1158,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SERDES` (
   `SERDE_ID` bigint,
   `NAME` string,
   `SLIB` string,
-  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1178,7 +1178,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
   `SERDE_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1198,7 +1198,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
   `SERDE_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1218,7 +1218,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
   `SD_ID` bigint,
   `SKEWED_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1238,7 +1238,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
   `SD_ID` bigint,
   `SKEWED_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1258,7 +1258,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
   `SD_ID` bigint,
   `STRING_LIST_ID_KID` bigint,
   `LOCATION` string,
-  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1278,7 +1278,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
   `SD_ID` bigint,
   `STRING_LIST_ID_KID` bigint,
   `LOCATION` string,
-  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1296,7 +1296,7 @@ POSTHOOK: Output: SYS@SKEWED_COL_VALUE_LOC_MAP
 POSTHOOK: Output: database:sys
 PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
   `STRING_LIST_ID` bigint,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1312,7 +1312,7 @@ PREHOOK: Output: SYS@SKEWED_STRING_LIST
 PREHOOK: Output: database:sys
 POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
   `STRING_LIST_ID` bigint,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1330,7 +1330,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
   `STRING_LIST_ID` bigint,
   `STRING_LIST_VALUE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1350,7 +1350,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
   `STRING_LIST_ID` bigint,
   `STRING_LIST_VALUE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1370,7 +1370,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
   `SD_ID_OID` bigint,
   `STRING_LIST_ID_EID` bigint,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1390,7 +1390,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
   `SD_ID_OID` bigint,
   `STRING_LIST_ID_EID` bigint,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1411,7 +1411,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `SORT_COLS` (
   `COLUMN_NAME` string,
   `ORDER` int,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1433,7 +1433,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SORT_COLS` (
   `COLUMN_NAME` string,
   `ORDER` int,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1454,7 +1454,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
   `TBL_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1474,7 +1474,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
   `TBL_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1503,7 +1503,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBLS` (
   `VIEW_EXPANDED_TEXT` string,
   `VIEW_ORIGINAL_TEXT` string,
   `IS_REWRITE_ENABLED` boolean,
-  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1540,7 +1540,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBLS` (
   `VIEW_EXPANDED_TEXT` string,
   `VIEW_ORIGINAL_TEXT` string,
   `IS_REWRITE_ENABLED` boolean,
-  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1575,7 +1575,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_COL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1609,7 +1609,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_COL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1642,7 +1642,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1674,7 +1674,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1716,7 +1716,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1768,7 +1768,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1821,7 +1821,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1875,7 +1875,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1912,7 +1912,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 )
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: SYS@VERSION
@@ -1921,7 +1921,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 )
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@VERSION
@@ -1939,7 +1939,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `DB_VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1959,7 +1959,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DB_VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -1984,7 +1984,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `FUNCS` (
   `FUNC_TYPE` int,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -2014,7 +2014,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `FUNCS` (
   `FUNC_TYPE` int,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -2049,7 +2049,7 @@ PREHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
-  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -2088,7 +2088,7 @@ POSTHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
-  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (


[26/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index 03e492e..9cf1ee2 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -41301,6 +41301,4617 @@ public final class HbaseMetastoreProto {
     // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.ForeignKeys)
   }
 
+  public interface UniqueConstraintsOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint> 
+        getUksList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint getUks(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    int getUksCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraintOrBuilder> 
+        getUksOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraintOrBuilder getUksOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints}
+   */
+  public static final class UniqueConstraints extends
+      com.google.protobuf.GeneratedMessage
+      implements UniqueConstraintsOrBuilder {
+    // Use UniqueConstraints.newBuilder() to construct.
+    private UniqueConstraints(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private UniqueConstraints(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final UniqueConstraints defaultInstance;
+    public static UniqueConstraints getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public UniqueConstraints getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private UniqueConstraints(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                uks_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              uks_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          uks_ = java.util.Collections.unmodifiableList(uks_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<UniqueConstraints> PARSER =
+        new com.google.protobuf.AbstractParser<UniqueConstraints>() {
+      public UniqueConstraints parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new UniqueConstraints(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<UniqueConstraints> getParserForType() {
+      return PARSER;
+    }
+
+    public interface UniqueConstraintOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string uk_name = 1;
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      boolean hasUkName();
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      java.lang.String getUkName();
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getUkNameBytes();
+
+      // repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> 
+          getColsList();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn getCols(int index);
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      int getColsCount();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder> 
+          getColsOrBuilderList();
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder getColsOrBuilder(
+          int index);
+
+      // optional bool enable_constraint = 3;
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      boolean hasEnableConstraint();
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      boolean getEnableConstraint();
+
+      // optional bool validate_constraint = 4;
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      boolean hasValidateConstraint();
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      boolean getValidateConstraint();
+
+      // optional bool rely_constraint = 5;
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      boolean hasRelyConstraint();
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      boolean getRelyConstraint();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint}
+     */
+    public static final class UniqueConstraint extends
+        com.google.protobuf.GeneratedMessage
+        implements UniqueConstraintOrBuilder {
+      // Use UniqueConstraint.newBuilder() to construct.
+      private UniqueConstraint(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private UniqueConstraint(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final UniqueConstraint defaultInstance;
+      public static UniqueConstraint getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public UniqueConstraint getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private UniqueConstraint(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                ukName_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                  cols_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn>();
+                  mutable_bitField0_ |= 0x00000002;
+                }
+                cols_.add(input.readMessage(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.PARSER, extensionRegistry));
+                break;
+              }
+              case 24: {
+                bitField0_ |= 0x00000002;
+                enableConstraint_ = input.readBool();
+                break;
+              }
+              case 32: {
+                bitField0_ |= 0x00000004;
+                validateConstraint_ = input.readBool();
+                break;
+              }
+              case 40: {
+                bitField0_ |= 0x00000008;
+                relyConstraint_ = input.readBool();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+            cols_ = java.util.Collections.unmodifiableList(cols_);
+          }
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<UniqueConstraint> PARSER =
+          new com.google.protobuf.AbstractParser<UniqueConstraint>() {
+        public UniqueConstraint parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new UniqueConstraint(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<UniqueConstraint> getParserForType() {
+        return PARSER;
+      }
+
+      public interface UniqueConstraintColumnOrBuilder
+          extends com.google.protobuf.MessageOrBuilder {
+
+        // required string column_name = 1;
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        boolean hasColumnName();
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        java.lang.String getColumnName();
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        com.google.protobuf.ByteString
+            getColumnNameBytes();
+
+        // required sint32 key_seq = 2;
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        boolean hasKeySeq();
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        int getKeySeq();
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn}
+       */
+      public static final class UniqueConstraintColumn extends
+          com.google.protobuf.GeneratedMessage
+          implements UniqueConstraintColumnOrBuilder {
+        // Use UniqueConstraintColumn.newBuilder() to construct.
+        private UniqueConstraintColumn(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+          super(builder);
+          this.unknownFields = builder.getUnknownFields();
+        }
+        private UniqueConstraintColumn(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+        private static final UniqueConstraintColumn defaultInstance;
+        public static UniqueConstraintColumn getDefaultInstance() {
+          return defaultInstance;
+        }
+
+        public UniqueConstraintColumn getDefaultInstanceForType() {
+          return defaultInstance;
+        }
+
+        private final com.google.protobuf.UnknownFieldSet unknownFields;
+        @java.lang.Override
+        public final com.google.protobuf.UnknownFieldSet
+            getUnknownFields() {
+          return this.unknownFields;
+        }
+        private UniqueConstraintColumn(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          initFields();
+          int mutable_bitField0_ = 0;
+          com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+              com.google.protobuf.UnknownFieldSet.newBuilder();
+          try {
+            boolean done = false;
+            while (!done) {
+              int tag = input.readTag();
+              switch (tag) {
+                case 0:
+                  done = true;
+                  break;
+                default: {
+                  if (!parseUnknownField(input, unknownFields,
+                                         extensionRegistry, tag)) {
+                    done = true;
+                  }
+                  break;
+                }
+                case 10: {
+                  bitField0_ |= 0x00000001;
+                  columnName_ = input.readBytes();
+                  break;
+                }
+                case 16: {
+                  bitField0_ |= 0x00000002;
+                  keySeq_ = input.readSInt32();
+                  break;
+                }
+              }
+            }
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            throw e.setUnfinishedMessage(this);
+          } catch (java.io.IOException e) {
+            throw new com.google.protobuf.InvalidProtocolBufferException(
+                e.getMessage()).setUnfinishedMessage(this);
+          } finally {
+            this.unknownFields = unknownFields.build();
+            makeExtensionsImmutable();
+          }
+        }
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_UniqueConstraintColumn_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_UniqueConstraintColumn_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder.class);
+        }
+
+        public static com.google.protobuf.Parser<UniqueConstraintColumn> PARSER =
+            new com.google.protobuf.AbstractParser<UniqueConstraintColumn>() {
+          public UniqueConstraintColumn parsePartialFrom(
+              com.google.protobuf.CodedInputStream input,
+              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+              throws com.google.protobuf.InvalidProtocolBufferException {
+            return new UniqueConstraintColumn(input, extensionRegistry);
+          }
+        };
+
+        @java.lang.Override
+        public com.google.protobuf.Parser<UniqueConstraintColumn> getParserForType() {
+          return PARSER;
+        }
+
+        private int bitField0_;
+        // required string column_name = 1;
+        public static final int COLUMN_NAME_FIELD_NUMBER = 1;
+        private java.lang.Object columnName_;
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public boolean hasColumnName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public java.lang.String getColumnName() {
+          java.lang.Object ref = columnName_;
+          if (ref instanceof java.lang.String) {
+            return (java.lang.String) ref;
+          } else {
+            com.google.protobuf.ByteString bs = 
+                (com.google.protobuf.ByteString) ref;
+            java.lang.String s = bs.toStringUtf8();
+            if (bs.isValidUtf8()) {
+              columnName_ = s;
+            }
+            return s;
+          }
+        }
+        /**
+         * <code>required string column_name = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getColumnNameBytes() {
+          java.lang.Object ref = columnName_;
+          if (ref instanceof java.lang.String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            columnName_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+
+        // required sint32 key_seq = 2;
+        public static final int KEY_SEQ_FIELD_NUMBER = 2;
+        private int keySeq_;
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public boolean hasKeySeq() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required sint32 key_seq = 2;</code>
+         */
+        public int getKeySeq() {
+          return keySeq_;
+        }
+
+        private void initFields() {
+          columnName_ = "";
+          keySeq_ = 0;
+        }
+        private byte memoizedIsInitialized = -1;
+        public final boolean isInitialized() {
+          byte isInitialized = memoizedIsInitialized;
+          if (isInitialized != -1) return isInitialized == 1;
+
+          if (!hasColumnName()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+          if (!hasKeySeq()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+          memoizedIsInitialized = 1;
+          return true;
+        }
+
+        public void writeTo(com.google.protobuf.CodedOutputStream output)
+                            throws java.io.IOException {
+          getSerializedSize();
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            output.writeBytes(1, getColumnNameBytes());
+          }
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            output.writeSInt32(2, keySeq_);
+          }
+          getUnknownFields().writeTo(output);
+        }
+
+        private int memoizedSerializedSize = -1;
+        public int getSerializedSize() {
+          int size = memoizedSerializedSize;
+          if (size != -1) return size;
+
+          size = 0;
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            size += com.google.protobuf.CodedOutputStream
+              .computeBytesSize(1, getColumnNameBytes());
+          }
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            size += com.google.protobuf.CodedOutputStream
+              .computeSInt32Size(2, keySeq_);
+          }
+          size += getUnknownFields().getSerializedSize();
+          memoizedSerializedSize = size;
+          return size;
+        }
+
+        private static final long serialVersionUID = 0L;
+        @java.lang.Override
+        protected java.lang.Object writeReplace()
+            throws java.io.ObjectStreamException {
+          return super.writeReplace();
+        }
+
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            com.google.protobuf.ByteString data)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return PARSER.parseFrom(data);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            com.google.protobuf.ByteString data,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return PARSER.parseFrom(data, extensionRegistry);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(byte[] data)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return PARSER.parseFrom(data);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            byte[] data,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return PARSER.parseFrom(data, extensionRegistry);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(java.io.InputStream input)
+            throws java.io.IOException {
+          return PARSER.parseFrom(input);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            java.io.InputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          return PARSER.parseFrom(input, extensionRegistry);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseDelimitedFrom(java.io.InputStream input)
+            throws java.io.IOException {
+          return PARSER.parseDelimitedFrom(input);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseDelimitedFrom(
+            java.io.InputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          return PARSER.parseDelimitedFrom(input, extensionRegistry);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            com.google.protobuf.CodedInputStream input)
+            throws java.io.IOException {
+          return PARSER.parseFrom(input);
+        }
+        public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parseFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          return PARSER.parseFrom(input, extensionRegistry);
+        }
+
+        public static Builder newBuilder() { return Builder.create(); }
+        public Builder newBuilderForType() { return newBuilder(); }
+        public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn prototype) {
+          return newBuilder().mergeFrom(prototype);
+        }
+        public Builder toBuilder() { return newBuilder(this); }
+
+        @java.lang.Override
+        protected Builder newBuilderForType(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          Builder builder = new Builder(parent);
+          return builder;
+        }
+        /**
+         * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn}
+         */
+        public static final class Builder extends
+            com.google.protobuf.GeneratedMessage.Builder<Builder>
+           implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder {
+          public static final com.google.protobuf.Descriptors.Descriptor
+              getDescriptor() {
+            return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_UniqueConstraintColumn_descriptor;
+          }
+
+          protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+              internalGetFieldAccessorTable() {
+            return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_UniqueConstraintColumn_fieldAccessorTable
+                .ensureFieldAccessorsInitialized(
+                    org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder.class);
+          }
+
+          // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.newBuilder()
+          private Builder() {
+            maybeForceBuilderInitialization();
+          }
+
+          private Builder(
+              com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+            super(parent);
+            maybeForceBuilderInitialization();
+          }
+          private void maybeForceBuilderInitialization() {
+            if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+            }
+          }
+          private static Builder create() {
+            return new Builder();
+          }
+
+          public Builder clear() {
+            super.clear();
+            columnName_ = "";
+            bitField0_ = (bitField0_ & ~0x00000001);
+            keySeq_ = 0;
+            bitField0_ = (bitField0_ & ~0x00000002);
+            return this;
+          }
+
+          public Builder clone() {
+            return create().mergeFrom(buildPartial());
+          }
+
+          public com.google.protobuf.Descriptors.Descriptor
+              getDescriptorForType() {
+            return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_UniqueConstraintColumn_descriptor;
+          }
+
+          public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn getDefaultInstanceForType() {
+            return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.getDefaultInstance();
+          }
+
+          public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn build() {
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn result = buildPartial();
+            if (!result.isInitialized()) {
+              throw newUninitializedMessageException(result);
+            }
+            return result;
+          }
+
+          public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn buildPartial() {
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn(this);
+            int from_bitField0_ = bitField0_;
+            int to_bitField0_ = 0;
+            if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+              to_bitField0_ |= 0x00000001;
+            }
+            result.columnName_ = columnName_;
+            if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+              to_bitField0_ |= 0x00000002;
+            }
+            result.keySeq_ = keySeq_;
+            result.bitField0_ = to_bitField0_;
+            onBuilt();
+            return result;
+          }
+
+          public Builder mergeFrom(com.google.protobuf.Message other) {
+            if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn) {
+              return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn)other);
+            } else {
+              super.mergeFrom(other);
+              return this;
+            }
+          }
+
+          public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn other) {
+            if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.getDefaultInstance()) return this;
+            if (other.hasColumnName()) {
+              bitField0_ |= 0x00000001;
+              columnName_ = other.columnName_;
+              onChanged();
+            }
+            if (other.hasKeySeq()) {
+              setKeySeq(other.getKeySeq());
+            }
+            this.mergeUnknownFields(other.getUnknownFields());
+            return this;
+          }
+
+          public final boolean isInitialized() {
+            if (!hasColumnName()) {
+              
+              return false;
+            }
+            if (!hasKeySeq()) {
+              
+              return false;
+            }
+            return true;
+          }
+
+          public Builder mergeFrom(
+              com.google.protobuf.CodedInputStream input,
+              com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+              throws java.io.IOException {
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn parsedMessage = null;
+            try {
+              parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+            } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+              parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn) e.getUnfinishedMessage();
+              throw e;
+            } finally {
+              if (parsedMessage != null) {
+                mergeFrom(parsedMessage);
+              }
+            }
+            return this;
+          }
+          private int bitField0_;
+
+          // required string column_name = 1;
+          private java.lang.Object columnName_ = "";
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public boolean hasColumnName() {
+            return ((bitField0_ & 0x00000001) == 0x00000001);
+          }
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public java.lang.String getColumnName() {
+            java.lang.Object ref = columnName_;
+            if (!(ref instanceof java.lang.String)) {
+              java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                  .toStringUtf8();
+              columnName_ = s;
+              return s;
+            } else {
+              return (java.lang.String) ref;
+            }
+          }
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public com.google.protobuf.ByteString
+              getColumnNameBytes() {
+            java.lang.Object ref = columnName_;
+            if (ref instanceof String) {
+              com.google.protobuf.ByteString b = 
+                  com.google.protobuf.ByteString.copyFromUtf8(
+                      (java.lang.String) ref);
+              columnName_ = b;
+              return b;
+            } else {
+              return (com.google.protobuf.ByteString) ref;
+            }
+          }
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public Builder setColumnName(
+              java.lang.String value) {
+            if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+            columnName_ = value;
+            onChanged();
+            return this;
+          }
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public Builder clearColumnName() {
+            bitField0_ = (bitField0_ & ~0x00000001);
+            columnName_ = getDefaultInstance().getColumnName();
+            onChanged();
+            return this;
+          }
+          /**
+           * <code>required string column_name = 1;</code>
+           */
+          public Builder setColumnNameBytes(
+              com.google.protobuf.ByteString value) {
+            if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+            columnName_ = value;
+            onChanged();
+            return this;
+          }
+
+          // required sint32 key_seq = 2;
+          private int keySeq_ ;
+          /**
+           * <code>required sint32 key_seq = 2;</code>
+           */
+          public boolean hasKeySeq() {
+            return ((bitField0_ & 0x00000002) == 0x00000002);
+          }
+          /**
+           * <code>required sint32 key_seq = 2;</code>
+           */
+          public int getKeySeq() {
+            return keySeq_;
+          }
+          /**
+           * <code>required sint32 key_seq = 2;</code>
+           */
+          public Builder setKeySeq(int value) {
+            bitField0_ |= 0x00000002;
+            keySeq_ = value;
+            onChanged();
+            return this;
+          }
+          /**
+           * <code>required sint32 key_seq = 2;</code>
+           */
+          public Builder clearKeySeq() {
+            bitField0_ = (bitField0_ & ~0x00000002);
+            keySeq_ = 0;
+            onChanged();
+            return this;
+          }
+
+          // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn)
+        }
+
+        static {
+          defaultInstance = new UniqueConstraintColumn(true);
+          defaultInstance.initFields();
+        }
+
+        // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn)
+      }
+
+      private int bitField0_;
+      // required string uk_name = 1;
+      public static final int UK_NAME_FIELD_NUMBER = 1;
+      private java.lang.Object ukName_;
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      public boolean hasUkName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      public java.lang.String getUkName() {
+        java.lang.Object ref = ukName_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            ukName_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string uk_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getUkNameBytes() {
+        java.lang.Object ref = ukName_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          ukName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;
+      public static final int COLS_FIELD_NUMBER = 2;
+      private java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> cols_;
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> getColsList() {
+        return cols_;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder> 
+          getColsOrBuilderList() {
+        return cols_;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      public int getColsCount() {
+        return cols_.size();
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn getCols(int index) {
+        return cols_.get(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+       */
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder getColsOrBuilder(
+          int index) {
+        return cols_.get(index);
+      }
+
+      // optional bool enable_constraint = 3;
+      public static final int ENABLE_CONSTRAINT_FIELD_NUMBER = 3;
+      private boolean enableConstraint_;
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public boolean hasEnableConstraint() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool enable_constraint = 3;</code>
+       */
+      public boolean getEnableConstraint() {
+        return enableConstraint_;
+      }
+
+      // optional bool validate_constraint = 4;
+      public static final int VALIDATE_CONSTRAINT_FIELD_NUMBER = 4;
+      private boolean validateConstraint_;
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public boolean hasValidateConstraint() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool validate_constraint = 4;</code>
+       */
+      public boolean getValidateConstraint() {
+        return validateConstraint_;
+      }
+
+      // optional bool rely_constraint = 5;
+      public static final int RELY_CONSTRAINT_FIELD_NUMBER = 5;
+      private boolean relyConstraint_;
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public boolean hasRelyConstraint() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional bool rely_constraint = 5;</code>
+       */
+      public boolean getRelyConstraint() {
+        return relyConstraint_;
+      }
+
+      private void initFields() {
+        ukName_ = "";
+        cols_ = java.util.Collections.emptyList();
+        enableConstraint_ = false;
+        validateConstraint_ = false;
+        relyConstraint_ = false;
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasUkName()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        for (int i = 0; i < getColsCount(); i++) {
+          if (!getCols(i).isInitialized()) {
+            memoizedIsInitialized = 0;
+            return false;
+          }
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getUkNameBytes());
+        }
+        for (int i = 0; i < cols_.size(); i++) {
+          output.writeMessage(2, cols_.get(i));
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBool(3, enableConstraint_);
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          output.writeBool(4, validateConstraint_);
+        }
+        if (((bitField0_ & 0x00000008) == 0x00000008)) {
+          output.writeBool(5, relyConstraint_);
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getUkNameBytes());
+        }
+        for (int i = 0; i < cols_.size(); i++) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeMessageSize(2, cols_.get(i));
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBoolSize(3, enableConstraint_);
+        }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBoolSize(4, validateConstraint_);
+        }
+        if (((bitField0_ & 0x00000008) == 0x00000008)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBoolSize(5, relyConstraint_);
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraintOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+            getColsFieldBuilder();
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          ukName_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          if (colsBuilder_ == null) {
+            cols_ = java.util.Collections.emptyList();
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            colsBuilder_.clear();
+          }
+          enableConstraint_ = false;
+          bitField0_ = (bitField0_ & ~0x00000004);
+          validateConstraint_ = false;
+          bitField0_ = (bitField0_ & ~0x00000008);
+          relyConstraint_ = false;
+          bitField0_ = (bitField0_ & ~0x00000010);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_UniqueConstraint_descriptor;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint getDefaultInstanceForType() {
+          return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint build() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint buildPartial() {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.ukName_ = ukName_;
+          if (colsBuilder_ == null) {
+            if (((bitField0_ & 0x00000002) == 0x00000002)) {
+              cols_ = java.util.Collections.unmodifiableList(cols_);
+              bitField0_ = (bitField0_ & ~0x00000002);
+            }
+            result.cols_ = cols_;
+          } else {
+            result.cols_ = colsBuilder_.build();
+          }
+          if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.enableConstraint_ = enableConstraint_;
+          if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+            to_bitField0_ |= 0x00000004;
+          }
+          result.validateConstraint_ = validateConstraint_;
+          if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+            to_bitField0_ |= 0x00000008;
+          }
+          result.relyConstraint_ = relyConstraint_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint) {
+            return mergeFrom((org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint other) {
+          if (other == org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.getDefaultInstance()) return this;
+          if (other.hasUkName()) {
+            bitField0_ |= 0x00000001;
+            ukName_ = other.ukName_;
+            onChanged();
+          }
+          if (colsBuilder_ == null) {
+            if (!other.cols_.isEmpty()) {
+              if (cols_.isEmpty()) {
+                cols_ = other.cols_;
+                bitField0_ = (bitField0_ & ~0x00000002);
+              } else {
+                ensureColsIsMutable();
+                cols_.addAll(other.cols_);
+              }
+              onChanged();
+            }
+          } else {
+            if (!other.cols_.isEmpty()) {
+              if (colsBuilder_.isEmpty()) {
+                colsBuilder_.dispose();
+                colsBuilder_ = null;
+                cols_ = other.cols_;
+                bitField0_ = (bitField0_ & ~0x00000002);
+                colsBuilder_ = 
+                  com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                     getColsFieldBuilder() : null;
+              } else {
+                colsBuilder_.addAllMessages(other.cols_);
+              }
+            }
+          }
+          if (other.hasEnableConstraint()) {
+            setEnableConstraint(other.getEnableConstraint());
+          }
+          if (other.hasValidateConstraint()) {
+            setValidateConstraint(other.getValidateConstraint());
+          }
+          if (other.hasRelyConstraint()) {
+            setRelyConstraint(other.getRelyConstraint());
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasUkName()) {
+            
+            return false;
+          }
+          for (int i = 0; i < getColsCount(); i++) {
+            if (!getCols(i).isInitialized()) {
+              
+              return false;
+            }
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string uk_name = 1;
+        private java.lang.Object ukName_ = "";
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public boolean hasUkName() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public java.lang.String getUkName() {
+          java.lang.Object ref = ukName_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            ukName_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getUkNameBytes() {
+          java.lang.Object ref = ukName_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            ukName_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public Builder setUkName(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          ukName_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public Builder clearUkName() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          ukName_ = getDefaultInstance().getUkName();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string uk_name = 1;</code>
+         */
+        public Builder setUkNameBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          ukName_ = value;
+          onChanged();
+          return this;
+        }
+
+        // repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;
+        private java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> cols_ =
+          java.util.Collections.emptyList();
+        private void ensureColsIsMutable() {
+          if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+            cols_ = new java.util.ArrayList<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn>(cols_);
+            bitField0_ |= 0x00000002;
+           }
+        }
+
+        private com.google.protobuf.RepeatedFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder> colsBuilder_;
+
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> getColsList() {
+          if (colsBuilder_ == null) {
+            return java.util.Collections.unmodifiableList(cols_);
+          } else {
+            return colsBuilder_.getMessageList();
+          }
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public int getColsCount() {
+          if (colsBuilder_ == null) {
+            return cols_.size();
+          } else {
+            return colsBuilder_.getCount();
+          }
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn getCols(int index) {
+          if (colsBuilder_ == null) {
+            return cols_.get(index);
+          } else {
+            return colsBuilder_.getMessage(index);
+          }
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder setCols(
+            int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn value) {
+          if (colsBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            ensureColsIsMutable();
+            cols_.set(index, value);
+            onChanged();
+          } else {
+            colsBuilder_.setMessage(index, value);
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder setCols(
+            int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder builderForValue) {
+          if (colsBuilder_ == null) {
+            ensureColsIsMutable();
+            cols_.set(index, builderForValue.build());
+            onChanged();
+          } else {
+            colsBuilder_.setMessage(index, builderForValue.build());
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder addCols(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn value) {
+          if (colsBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            ensureColsIsMutable();
+            cols_.add(value);
+            onChanged();
+          } else {
+            colsBuilder_.addMessage(value);
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder addCols(
+            int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn value) {
+          if (colsBuilder_ == null) {
+            if (value == null) {
+              throw new NullPointerException();
+            }
+            ensureColsIsMutable();
+            cols_.add(index, value);
+            onChanged();
+          } else {
+            colsBuilder_.addMessage(index, value);
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder addCols(
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder builderForValue) {
+          if (colsBuilder_ == null) {
+            ensureColsIsMutable();
+            cols_.add(builderForValue.build());
+            onChanged();
+          } else {
+            colsBuilder_.addMessage(builderForValue.build());
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder addCols(
+            int index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder builderForValue) {
+          if (colsBuilder_ == null) {
+            ensureColsIsMutable();
+            cols_.add(index, builderForValue.build());
+            onChanged();
+          } else {
+            colsBuilder_.addMessage(index, builderForValue.build());
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder addAllCols(
+            java.lang.Iterable<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn> values) {
+          if (colsBuilder_ == null) {
+            ensureColsIsMutable();
+            super.addAll(values, cols_);
+            onChanged();
+          } else {
+            colsBuilder_.addAllMessages(values);
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder clearCols() {
+          if (colsBuilder_ == null) {
+            cols_ = java.util.Collections.emptyList();
+            bitField0_ = (bitField0_ & ~0x00000002);
+            onChanged();
+          } else {
+            colsBuilder_.clear();
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public Builder removeCols(int index) {
+          if (colsBuilder_ == null) {
+            ensureColsIsMutable();
+            cols_.remove(index);
+            onChanged();
+          } else {
+            colsBuilder_.remove(index);
+          }
+          return this;
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder getColsBuilder(
+            int index) {
+          return getColsFieldBuilder().getBuilder(index);
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder getColsOrBuilder(
+            int index) {
+          if (colsBuilder_ == null) {
+            return cols_.get(index);  } else {
+            return colsBuilder_.getMessageOrBuilder(index);
+          }
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder> 
+             getColsOrBuilderList() {
+          if (colsBuilder_ != null) {
+            return colsBuilder_.getMessageOrBuilderList();
+          } else {
+            return java.util.Collections.unmodifiableList(cols_);
+          }
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder addColsBuilder() {
+          return getColsFieldBuilder().addBuilder(
+              org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.getDefaultInstance());
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder addColsBuilder(
+            int index) {
+          return getColsFieldBuilder().addBuilder(
+              index, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.getDefaultInstance());
+        }
+        /**
+         * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn cols = 2;</code>
+         */
+        public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder> 
+             getColsBuilderList() {
+          return getColsFieldBuilder().getBuilderList();
+        }
+        private com.google.protobuf.RepeatedFieldBuilder<
+            org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder> 
+            getColsFieldBuilder() {
+          if (colsBuilder_ == null) {
+            colsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumn.Builder, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint.UniqueConstraintColumnOrBuilder>(
+                    cols_,
+                    ((bitField0_ & 0x00000002) == 0x00000002),
+                    getParentForChildren(),
+                    isClean());
+            cols_ = null;
+          }
+          return colsBuilder_;
+        }
+
+        // optional bool enable_constraint = 3;
+        private boolean enableConstraint_ ;
+        /**
+         * <code>optional bool enable_constraint = 3;</code>
+         */
+        public boolean hasEnableConstraint() {
+          return ((bitField0_ & 0x00000004) == 0x00000004);
+        }
+        /**
+         * <code>optional bool enable_constraint = 3;</code>
+         */
+        public boolean getEnableConstraint() {
+          return enableConstraint_;
+        }
+        /**
+         * <code>optional bool enable_constraint = 3;</code>
+         */
+        public Builder setEnableConstraint(boolean value) {
+          bitField0_ |= 0x00000004;
+          enableConstraint_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional bool enable_constraint = 3;</code>
+         */
+        public Builder clearEnableConstraint() {
+          bitField0_ = (bitField0_ & ~0x00000004);
+          enableConstraint_ = false;
+          onChanged();
+          return this;
+        }
+
+        // optional bool validate_constraint = 4;
+        private boolean validateConstraint_ ;
+        /**
+         * <code>optional bool validate_constraint = 4;</code>
+         */
+        public boolean hasValidateConstraint() {
+          return ((bitField0_ & 0x00000008) == 0x00000008);
+        }
+        /**
+         * <code>optional bool validate_constraint = 4;</code>
+         */
+        public boolean getValidateConstraint() {
+          return validateConstraint_;
+        }
+        /**
+         * <code>optional bool validate_constraint = 4;</code>
+         */
+        public Builder setValidateConstraint(boolean value) {
+          bitField0_ |= 0x00000008;
+          validateConstraint_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional bool validate_constraint = 4;</code>
+         */
+        public Builder clearValidateConstraint() {
+          bitField0_ = (bitField0_ & ~0x00000008);
+          validateConstraint_ = false;
+          onChanged();
+          return this;
+        }
+
+        // optional bool rely_constraint = 5;
+        private boolean relyConstraint_ ;
+        /**
+         * <code>optional bool rely_constraint = 5;</code>
+         */
+        public boolean hasRelyConstraint() {
+          return ((bitField0_ & 0x00000010) == 0x00000010);
+        }
+        /**
+         * <code>optional bool rely_constraint = 5;</code>
+         */
+        public boolean getRelyConstraint() {
+          return relyConstraint_;
+        }
+        /**
+         * <code>optional bool rely_constraint = 5;</code>
+         */
+        public Builder setRelyConstraint(boolean value) {
+          bitField0_ |= 0x00000010;
+          relyConstraint_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional bool rely_constraint = 5;</code>
+         */
+        public Builder clearRelyConstraint() {
+          bitField0_ = (bitField0_ & ~0x00000010);
+          relyConstraint_ = false;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint)
+      }
+
+      static {
+        defaultInstance = new UniqueConstraint(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint)
+    }
+
+    // repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;
+    public static final int UKS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint> uks_;
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint> getUksList() {
+      return uks_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraintOrBuilder> 
+        getUksOrBuilderList() {
+      return uks_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    public int getUksCount() {
+      return uks_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraint getUks(int index) {
+      return uks_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hive.metastore.hbase.UniqueConstraints.UniqueConstraint uks = 1;</code>
+     */
+    public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.UniqueConstraintOrBuilder getUksOrBuilder(
+        int index) {
+      return uks_.get(index);
+    }
+
+    private void initFields() {
+      uks_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getUksCount(); i++) {
+        if (!getUks(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < uks_.size(); i++) {
+        output.writeMessage(1, uks_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < uks_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, uks_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.UniqueConstraints}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraintsOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.class, org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUksFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (uksBuilder_ == null) {
+          uks_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          uksBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.internal_static_org_apache_hadoop_hive_metastore_hbase_UniqueConstraints_descriptor;
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints getDefaultInstanceForType() {
+        return org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints build() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints buildPartial() {
+        org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints result = new org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.UniqueConstraints(this);
+        int from_bitField0_ = bitField0_;
+        if (uksBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            uks_ = java.util.Collections.unmodifiableList(uks_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.uks_ = uks_;
+        } else {
+          result.uks_ = uksBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.prot

<TRUNCATED>

[20/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
new file mode 100644
index 0000000..b49895b
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.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)")
+public class NotNullConstraintsResponse implements org.apache.thrift.TBase<NotNullConstraintsResponse, NotNullConstraintsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<NotNullConstraintsResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotNullConstraintsResponse");
+
+  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)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NotNullConstraintsResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NotNullConstraintsResponseTupleSchemeFactory());
+  }
+
+  private List<SQLNotNullConstraint> notNullConstraints; // 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 {
+    NOT_NULL_CONSTRAINTS((short)1, "notNullConstraints");
+
+    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: // NOT_NULL_CONSTRAINTS
+          return NOT_NULL_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.NOT_NULL_CONSTRAINTS, new org.apache.thrift.meta_data.FieldMetaData("notNullConstraints", 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, SQLNotNullConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotNullConstraintsResponse.class, metaDataMap);
+  }
+
+  public NotNullConstraintsResponse() {
+  }
+
+  public NotNullConstraintsResponse(
+    List<SQLNotNullConstraint> notNullConstraints)
+  {
+    this();
+    this.notNullConstraints = notNullConstraints;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NotNullConstraintsResponse(NotNullConstraintsResponse other) {
+    if (other.isSetNotNullConstraints()) {
+      List<SQLNotNullConstraint> __this__notNullConstraints = new ArrayList<SQLNotNullConstraint>(other.notNullConstraints.size());
+      for (SQLNotNullConstraint other_element : other.notNullConstraints) {
+        __this__notNullConstraints.add(new SQLNotNullConstraint(other_element));
+      }
+      this.notNullConstraints = __this__notNullConstraints;
+    }
+  }
+
+  public NotNullConstraintsResponse deepCopy() {
+    return new NotNullConstraintsResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.notNullConstraints = null;
+  }
+
+  public int getNotNullConstraintsSize() {
+    return (this.notNullConstraints == null) ? 0 : this.notNullConstraints.size();
+  }
+
+  public java.util.Iterator<SQLNotNullConstraint> getNotNullConstraintsIterator() {
+    return (this.notNullConstraints == null) ? null : this.notNullConstraints.iterator();
+  }
+
+  public void addToNotNullConstraints(SQLNotNullConstraint elem) {
+    if (this.notNullConstraints == null) {
+      this.notNullConstraints = new ArrayList<SQLNotNullConstraint>();
+    }
+    this.notNullConstraints.add(elem);
+  }
+
+  public List<SQLNotNullConstraint> getNotNullConstraints() {
+    return this.notNullConstraints;
+  }
+
+  public void setNotNullConstraints(List<SQLNotNullConstraint> notNullConstraints) {
+    this.notNullConstraints = notNullConstraints;
+  }
+
+  public void unsetNotNullConstraints() {
+    this.notNullConstraints = null;
+  }
+
+  /** Returns true if field notNullConstraints is set (has been assigned a value) and false otherwise */
+  public boolean isSetNotNullConstraints() {
+    return this.notNullConstraints != null;
+  }
+
+  public void setNotNullConstraintsIsSet(boolean value) {
+    if (!value) {
+      this.notNullConstraints = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NOT_NULL_CONSTRAINTS:
+      if (value == null) {
+        unsetNotNullConstraints();
+      } else {
+        setNotNullConstraints((List<SQLNotNullConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NOT_NULL_CONSTRAINTS:
+      return getNotNullConstraints();
+
+    }
+    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 NOT_NULL_CONSTRAINTS:
+      return isSetNotNullConstraints();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NotNullConstraintsResponse)
+      return this.equals((NotNullConstraintsResponse)that);
+    return false;
+  }
+
+  public boolean equals(NotNullConstraintsResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_notNullConstraints = true && this.isSetNotNullConstraints();
+    boolean that_present_notNullConstraints = true && that.isSetNotNullConstraints();
+    if (this_present_notNullConstraints || that_present_notNullConstraints) {
+      if (!(this_present_notNullConstraints && that_present_notNullConstraints))
+        return false;
+      if (!this.notNullConstraints.equals(that.notNullConstraints))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_notNullConstraints = true && (isSetNotNullConstraints());
+    list.add(present_notNullConstraints);
+    if (present_notNullConstraints)
+      list.add(notNullConstraints);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NotNullConstraintsResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetNotNullConstraints()).compareTo(other.isSetNotNullConstraints());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNotNullConstraints()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.notNullConstraints, other.notNullConstraints);
+      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("NotNullConstraintsResponse(");
+    boolean first = true;
+
+    sb.append("notNullConstraints:");
+    if (this.notNullConstraints == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.notNullConstraints);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetNotNullConstraints()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'notNullConstraints' 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 NotNullConstraintsResponseStandardSchemeFactory implements SchemeFactory {
+    public NotNullConstraintsResponseStandardScheme getScheme() {
+      return new NotNullConstraintsResponseStandardScheme();
+    }
+  }
+
+  private static class NotNullConstraintsResponseStandardScheme extends StandardScheme<NotNullConstraintsResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NotNullConstraintsResponse 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: // NOT_NULL_CONSTRAINTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list346.size);
+                SQLNotNullConstraint _elem347;
+                for (int _i348 = 0; _i348 < _list346.size; ++_i348)
+                {
+                  _elem347 = new SQLNotNullConstraint();
+                  _elem347.read(iprot);
+                  struct.notNullConstraints.add(_elem347);
+                }
+                iprot.readListEnd();
+              }
+              struct.setNotNullConstraintsIsSet(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, NotNullConstraintsResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.notNullConstraints != null) {
+        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 _iter349 : struct.notNullConstraints)
+          {
+            _iter349.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NotNullConstraintsResponseTupleSchemeFactory implements SchemeFactory {
+    public NotNullConstraintsResponseTupleScheme getScheme() {
+      return new NotNullConstraintsResponseTupleScheme();
+    }
+  }
+
+  private static class NotNullConstraintsResponseTupleScheme extends TupleScheme<NotNullConstraintsResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.notNullConstraints.size());
+        for (SQLNotNullConstraint _iter350 : struct.notNullConstraints)
+        {
+          _iter350.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list351.size);
+        SQLNotNullConstraint _elem352;
+        for (int _i353 = 0; _i353 < _list351.size; ++_i353)
+        {
+          _elem352 = new SQLNotNullConstraint();
+          _elem352.read(iprot);
+          struct.notNullConstraints.add(_elem352);
+        }
+      }
+      struct.setNotNullConstraintsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index edc548a..bcfc75b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list550 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list550.size);
-                NotificationEvent _elem551;
-                for (int _i552 = 0; _i552 < _list550.size; ++_i552)
+                org.apache.thrift.protocol.TList _list582 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list582.size);
+                NotificationEvent _elem583;
+                for (int _i584 = 0; _i584 < _list582.size; ++_i584)
                 {
-                  _elem551 = new NotificationEvent();
-                  _elem551.read(iprot);
-                  struct.events.add(_elem551);
+                  _elem583 = new NotificationEvent();
+                  _elem583.read(iprot);
+                  struct.events.add(_elem583);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter553 : struct.events)
+          for (NotificationEvent _iter585 : struct.events)
           {
-            _iter553.write(oprot);
+            _iter585.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter554 : struct.events)
+        for (NotificationEvent _iter586 : struct.events)
         {
-          _iter554.write(oprot);
+          _iter586.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
     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 _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list555.size);
-        NotificationEvent _elem556;
-        for (int _i557 = 0; _i557 < _list555.size; ++_i557)
+        org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list587.size);
+        NotificationEvent _elem588;
+        for (int _i589 = 0; _i589 < _list587.size; ++_i589)
         {
-          _elem556 = new NotificationEvent();
-          _elem556.read(iprot);
-          struct.events.add(_elem556);
+          _elem588 = new NotificationEvent();
+          _elem588.read(iprot);
+          struct.events.add(_elem588);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index a8af71b..c23482a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -351,13 +351,13 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list476.size);
-                long _elem477;
-                for (int _i478 = 0; _i478 < _list476.size; ++_i478)
+                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list508.size);
+                long _elem509;
+                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
                 {
-                  _elem477 = iprot.readI64();
-                  struct.txn_ids.add(_elem477);
+                  _elem509 = iprot.readI64();
+                  struct.txn_ids.add(_elem509);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
         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 _iter479 : struct.txn_ids)
+          for (long _iter511 : struct.txn_ids)
           {
-            oprot.writeI64(_iter479);
+            oprot.writeI64(_iter511);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter480 : struct.txn_ids)
+        for (long _iter512 : struct.txn_ids)
         {
-          oprot.writeI64(_iter480);
+          oprot.writeI64(_iter512);
         }
       }
     }
@@ -421,13 +421,13 @@ public class OpenTxnsResponse implements org.apache.thrift.TBase<OpenTxnsRespons
     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 _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list481.size);
-        long _elem482;
-        for (int _i483 = 0; _i483 < _list481.size; ++_i483)
+        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list513.size);
+        long _elem514;
+        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
         {
-          _elem482 = iprot.readI64();
-          struct.txn_ids.add(_elem482);
+          _elem514 = iprot.readI64();
+          struct.txn_ids.add(_elem514);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index 0533053..7df156f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@ -439,14 +439,14 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list354.size);
-                Partition _elem355;
-                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+                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)
                 {
-                  _elem355 = new Partition();
-                  _elem355.read(iprot);
-                  struct.partitions.add(_elem355);
+                  _elem387 = new Partition();
+                  _elem387.read(iprot);
+                  struct.partitions.add(_elem387);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter357 : struct.partitions)
+          for (Partition _iter389 : struct.partitions)
           {
-            _iter357.write(oprot);
+            _iter389.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter358 : struct.partitions)
+        for (Partition _iter390 : struct.partitions)
         {
-          _iter358.write(oprot);
+          _iter390.write(oprot);
         }
       }
       oprot.writeBool(struct.hasUnknownPartitions);
@@ -522,14 +522,14 @@ public class PartitionsByExprResult implements org.apache.thrift.TBase<Partition
     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 _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list359.size);
-        Partition _elem360;
-        for (int _i361 = 0; _i361 < _list359.size; ++_i361)
+        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)
         {
-          _elem360 = new Partition();
-          _elem360.read(iprot);
-          struct.partitions.add(_elem360);
+          _elem392 = new Partition();
+          _elem392.read(iprot);
+          struct.partitions.add(_elem392);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index 65b8a54..b43f6c0 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -639,13 +639,13 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list396 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list396.size);
-                String _elem397;
-                for (int _i398 = 0; _i398 < _list396.size; ++_i398)
+                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)
                 {
-                  _elem397 = iprot.readString();
-                  struct.colNames.add(_elem397);
+                  _elem429 = iprot.readString();
+                  struct.colNames.add(_elem429);
                 }
                 iprot.readListEnd();
               }
@@ -657,13 +657,13 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
           case 4: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list399 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list399.size);
-                String _elem400;
-                for (int _i401 = 0; _i401 < _list399.size; ++_i401)
+                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)
                 {
-                  _elem400 = iprot.readString();
-                  struct.partNames.add(_elem400);
+                  _elem432 = iprot.readString();
+                  struct.partNames.add(_elem432);
                 }
                 iprot.readListEnd();
               }
@@ -699,9 +699,9 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
         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 _iter402 : struct.colNames)
+          for (String _iter434 : struct.colNames)
           {
-            oprot.writeString(_iter402);
+            oprot.writeString(_iter434);
           }
           oprot.writeListEnd();
         }
@@ -711,9 +711,9 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
         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 _iter403 : struct.partNames)
+          for (String _iter435 : struct.partNames)
           {
-            oprot.writeString(_iter403);
+            oprot.writeString(_iter435);
           }
           oprot.writeListEnd();
         }
@@ -740,16 +740,16 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter404 : struct.colNames)
+        for (String _iter436 : struct.colNames)
         {
-          oprot.writeString(_iter404);
+          oprot.writeString(_iter436);
         }
       }
       {
         oprot.writeI32(struct.partNames.size());
-        for (String _iter405 : struct.partNames)
+        for (String _iter437 : struct.partNames)
         {
-          oprot.writeString(_iter405);
+          oprot.writeString(_iter437);
         }
       }
     }
@@ -762,24 +762,24 @@ public class PartitionsStatsRequest implements org.apache.thrift.TBase<Partition
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list406 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list406.size);
-        String _elem407;
-        for (int _i408 = 0; _i408 < _list406.size; ++_i408)
+        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)
         {
-          _elem407 = iprot.readString();
-          struct.colNames.add(_elem407);
+          _elem439 = iprot.readString();
+          struct.colNames.add(_elem439);
         }
       }
       struct.setColNamesIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list409 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partNames = new ArrayList<String>(_list409.size);
-        String _elem410;
-        for (int _i411 = 0; _i411 < _list409.size; ++_i411)
+        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)
         {
-          _elem410 = iprot.readString();
-          struct.partNames.add(_elem410);
+          _elem442 = iprot.readString();
+          struct.partNames.add(_elem442);
         }
       }
       struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index 1d5e6ce..63ed662 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@ -363,26 +363,26 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
           case 1: // PART_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map370 = iprot.readMapBegin();
-                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map370.size);
-                String _key371;
-                List<ColumnStatisticsObj> _val372;
-                for (int _i373 = 0; _i373 < _map370.size; ++_i373)
+                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)
                 {
-                  _key371 = iprot.readString();
+                  _key403 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list374 = iprot.readListBegin();
-                    _val372 = new ArrayList<ColumnStatisticsObj>(_list374.size);
-                    ColumnStatisticsObj _elem375;
-                    for (int _i376 = 0; _i376 < _list374.size; ++_i376)
+                    org.apache.thrift.protocol.TList _list406 = iprot.readListBegin();
+                    _val404 = new ArrayList<ColumnStatisticsObj>(_list406.size);
+                    ColumnStatisticsObj _elem407;
+                    for (int _i408 = 0; _i408 < _list406.size; ++_i408)
                     {
-                      _elem375 = new ColumnStatisticsObj();
-                      _elem375.read(iprot);
-                      _val372.add(_elem375);
+                      _elem407 = new ColumnStatisticsObj();
+                      _elem407.read(iprot);
+                      _val404.add(_elem407);
                     }
                     iprot.readListEnd();
                   }
-                  struct.partStats.put(_key371, _val372);
+                  struct.partStats.put(_key403, _val404);
                 }
                 iprot.readMapEnd();
               }
@@ -408,14 +408,14 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
         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>> _iter377 : struct.partStats.entrySet())
+          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter409 : struct.partStats.entrySet())
           {
-            oprot.writeString(_iter377.getKey());
+            oprot.writeString(_iter409.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter377.getValue().size()));
-              for (ColumnStatisticsObj _iter378 : _iter377.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter409.getValue().size()));
+              for (ColumnStatisticsObj _iter410 : _iter409.getValue())
               {
-                _iter378.write(oprot);
+                _iter410.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -443,14 +443,14 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partStats.size());
-        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter379 : struct.partStats.entrySet())
+        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter411 : struct.partStats.entrySet())
         {
-          oprot.writeString(_iter379.getKey());
+          oprot.writeString(_iter411.getKey());
           {
-            oprot.writeI32(_iter379.getValue().size());
-            for (ColumnStatisticsObj _iter380 : _iter379.getValue())
+            oprot.writeI32(_iter411.getValue().size());
+            for (ColumnStatisticsObj _iter412 : _iter411.getValue())
             {
-              _iter380.write(oprot);
+              _iter412.write(oprot);
             }
           }
         }
@@ -461,25 +461,25 @@ public class PartitionsStatsResult implements org.apache.thrift.TBase<Partitions
     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 _map381 = 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*_map381.size);
-        String _key382;
-        List<ColumnStatisticsObj> _val383;
-        for (int _i384 = 0; _i384 < _map381.size; ++_i384)
+        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)
         {
-          _key382 = iprot.readString();
+          _key414 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list385 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val383 = new ArrayList<ColumnStatisticsObj>(_list385.size);
-            ColumnStatisticsObj _elem386;
-            for (int _i387 = 0; _i387 < _list385.size; ++_i387)
+            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)
             {
-              _elem386 = new ColumnStatisticsObj();
-              _elem386.read(iprot);
-              _val383.add(_elem386);
+              _elem418 = new ColumnStatisticsObj();
+              _elem418.read(iprot);
+              _val415.add(_elem418);
             }
           }
-          struct.partStats.put(_key382, _val383);
+          struct.partStats.put(_key414, _val415);
         }
       }
       struct.setPartStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 0a1302f..0c06ff3 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list618.size);
-                long _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list650.size);
+                long _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem619 = iprot.readI64();
-                  struct.fileIds.add(_elem619);
+                  _elem651 = iprot.readI64();
+                  struct.fileIds.add(_elem651);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list621 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list621.size);
-                ByteBuffer _elem622;
-                for (int _i623 = 0; _i623 < _list621.size; ++_i623)
+                org.apache.thrift.protocol.TList _list653 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list653.size);
+                ByteBuffer _elem654;
+                for (int _i655 = 0; _i655 < _list653.size; ++_i655)
                 {
-                  _elem622 = iprot.readBinary();
-                  struct.metadata.add(_elem622);
+                  _elem654 = iprot.readBinary();
+                  struct.metadata.add(_elem654);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         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 _iter624 : struct.fileIds)
+          for (long _iter656 : struct.fileIds)
           {
-            oprot.writeI64(_iter624);
+            oprot.writeI64(_iter656);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter625 : struct.metadata)
+          for (ByteBuffer _iter657 : struct.metadata)
           {
-            oprot.writeBinary(_iter625);
+            oprot.writeBinary(_iter657);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter626 : struct.fileIds)
+        for (long _iter658 : struct.fileIds)
         {
-          oprot.writeI64(_iter626);
+          oprot.writeI64(_iter658);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter627 : struct.metadata)
+        for (ByteBuffer _iter659 : struct.metadata)
         {
-          oprot.writeBinary(_iter627);
+          oprot.writeBinary(_iter659);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     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 _list628 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list628.size);
-        long _elem629;
-        for (int _i630 = 0; _i630 < _list628.size; ++_i630)
+        org.apache.thrift.protocol.TList _list660 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list660.size);
+        long _elem661;
+        for (int _i662 = 0; _i662 < _list660.size; ++_i662)
         {
-          _elem629 = iprot.readI64();
-          struct.fileIds.add(_elem629);
+          _elem661 = iprot.readI64();
+          struct.fileIds.add(_elem661);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list631.size);
-        ByteBuffer _elem632;
-        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
+        org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list663.size);
+        ByteBuffer _elem664;
+        for (int _i665 = 0; _i665 < _list663.size; ++_i665)
         {
-          _elem632 = iprot.readBinary();
-          struct.metadata.add(_elem632);
+          _elem664 = iprot.readBinary();
+          struct.metadata.add(_elem664);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
index 7368e9a..77067c6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
@@ -168,13 +168,13 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
           if (field.type == NAMES_FIELD_DESC.type) {
             List<String> names;
             {
-              org.apache.thrift.protocol.TList _list436 = iprot.readListBegin();
-              names = new ArrayList<String>(_list436.size);
-              String _elem437;
-              for (int _i438 = 0; _i438 < _list436.size; ++_i438)
+              org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
+              names = new ArrayList<String>(_list468.size);
+              String _elem469;
+              for (int _i470 = 0; _i470 < _list468.size; ++_i470)
               {
-                _elem437 = iprot.readString();
-                names.add(_elem437);
+                _elem469 = iprot.readString();
+                names.add(_elem469);
               }
               iprot.readListEnd();
             }
@@ -187,14 +187,14 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
           if (field.type == EXPRS_FIELD_DESC.type) {
             List<DropPartitionsExpr> exprs;
             {
-              org.apache.thrift.protocol.TList _list439 = iprot.readListBegin();
-              exprs = new ArrayList<DropPartitionsExpr>(_list439.size);
-              DropPartitionsExpr _elem440;
-              for (int _i441 = 0; _i441 < _list439.size; ++_i441)
+              org.apache.thrift.protocol.TList _list471 = iprot.readListBegin();
+              exprs = new ArrayList<DropPartitionsExpr>(_list471.size);
+              DropPartitionsExpr _elem472;
+              for (int _i473 = 0; _i473 < _list471.size; ++_i473)
               {
-                _elem440 = new DropPartitionsExpr();
-                _elem440.read(iprot);
-                exprs.add(_elem440);
+                _elem472 = new DropPartitionsExpr();
+                _elem472.read(iprot);
+                exprs.add(_elem472);
               }
               iprot.readListEnd();
             }
@@ -219,9 +219,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter442 : names)
+          for (String _iter474 : names)
           {
-            oprot.writeString(_iter442);
+            oprot.writeString(_iter474);
           }
           oprot.writeListEnd();
         }
@@ -230,9 +230,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter443 : exprs)
+          for (DropPartitionsExpr _iter475 : exprs)
           {
-            _iter443.write(oprot);
+            _iter475.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -250,13 +250,13 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         case NAMES:
           List<String> names;
           {
-            org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
-            names = new ArrayList<String>(_list444.size);
-            String _elem445;
-            for (int _i446 = 0; _i446 < _list444.size; ++_i446)
+            org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
+            names = new ArrayList<String>(_list476.size);
+            String _elem477;
+            for (int _i478 = 0; _i478 < _list476.size; ++_i478)
             {
-              _elem445 = iprot.readString();
-              names.add(_elem445);
+              _elem477 = iprot.readString();
+              names.add(_elem477);
             }
             iprot.readListEnd();
           }
@@ -264,14 +264,14 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         case EXPRS:
           List<DropPartitionsExpr> exprs;
           {
-            org.apache.thrift.protocol.TList _list447 = iprot.readListBegin();
-            exprs = new ArrayList<DropPartitionsExpr>(_list447.size);
-            DropPartitionsExpr _elem448;
-            for (int _i449 = 0; _i449 < _list447.size; ++_i449)
+            org.apache.thrift.protocol.TList _list479 = iprot.readListBegin();
+            exprs = new ArrayList<DropPartitionsExpr>(_list479.size);
+            DropPartitionsExpr _elem480;
+            for (int _i481 = 0; _i481 < _list479.size; ++_i481)
             {
-              _elem448 = new DropPartitionsExpr();
-              _elem448.read(iprot);
-              exprs.add(_elem448);
+              _elem480 = new DropPartitionsExpr();
+              _elem480.read(iprot);
+              exprs.add(_elem480);
             }
             iprot.readListEnd();
           }
@@ -291,9 +291,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter450 : names)
+          for (String _iter482 : names)
           {
-            oprot.writeString(_iter450);
+            oprot.writeString(_iter482);
           }
           oprot.writeListEnd();
         }
@@ -302,9 +302,9 @@ public class RequestPartsSpec extends org.apache.thrift.TUnion<RequestPartsSpec,
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter451 : exprs)
+          for (DropPartitionsExpr _iter483 : exprs)
           {
-            _iter451.write(oprot);
+            _iter483.write(oprot);
           }
           oprot.writeListEnd();
         }


[31/31] hive git commit: HIVE-14671 : Merge branch 'master' into hive-14535 (Wei Zheng)

Posted by we...@apache.org.
HIVE-14671 : Merge branch 'master' into hive-14535 (Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: 52e0f8f34076fda56888b773f9e603323d7a0fc5
Parents: 21c209e 2fa4dc2
Author: Wei Zheng <we...@apache.org>
Authored: Thu May 25 13:48:45 2017 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Thu May 25 13:48:45 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/jsonexplain/Op.java      |     4 +-
 .../hadoop/hive/common/jsonexplain/TestOp.java  |    81 +
 .../hive/common/jsonexplain/TestStage.java      |   194 +
 .../hive/common/jsonexplain/TestVertex.java     |   108 +
 .../jsonexplain/tez/TestTezJsonParser.java      |    53 +
 errata.txt                                      |     3 +-
 .../listener/DummyRawStoreFailEvent.java        |    28 +-
 .../hive/ql/parse/TestReplicationScenarios.java |   171 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |     2 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |     2 +
 .../hive/llap/daemon/impl/LlapDaemon.java       |     4 +-
 metastore/if/hive_metastore.thrift              |    61 +-
 .../upgrade/derby/042-HIVE-16575.derby.sql      |     4 +
 .../upgrade/derby/hive-schema-3.0.0.derby.sql   |     4 +-
 .../derby/upgrade-2.3.0-to-3.0.0.derby.sql      |     2 +-
 .../upgrade/hive/hive-schema-3.0.0.hive.sql     |    78 +-
 .../upgrade/mssql/027-HIVE-16575.mssql.sql      |     1 +
 .../upgrade/mssql/hive-schema-3.0.0.mssql.sql   |     2 +
 .../mssql/upgrade-2.3.0-to-3.0.0.mssql.sql      |     1 +
 .../upgrade/mysql/042-HIVE-16575.mysql.sql      |     1 +
 .../upgrade/mysql/hive-schema-3.0.0.mysql.sql   |     2 +
 .../mysql/upgrade-2.3.0-to-3.0.0.mysql.sql      |     1 +
 .../upgrade/oracle/042-HIVE-16575.oracle.sql    |     1 +
 .../upgrade/oracle/hive-schema-3.0.0.oracle.sql |     2 +
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |     1 +
 .../postgres/041-HIVE-16575.postgres.sql        |     1 +
 .../postgres/hive-schema-3.0.0.postgres.sql     |     2 +
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |     1 +
 .../metastore/hbase/HbaseMetastoreProto.java    |  4700 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  5850 ++-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   584 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    22 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  7378 +--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   436 +
 .../hive/metastore/api/AbortTxnsRequest.java    |    32 +-
 .../metastore/api/AddDynamicPartitions.java     |    32 +-
 .../metastore/api/AddForeignKeyRequest.java     |    36 +-
 .../api/AddNotNullConstraintRequest.java        |   443 +
 .../metastore/api/AddPartitionsRequest.java     |    36 +-
 .../hive/metastore/api/AddPartitionsResult.java |    36 +-
 .../metastore/api/AddPrimaryKeyRequest.java     |    36 +-
 .../api/AddUniqueConstraintRequest.java         |   443 +
 .../metastore/api/ClearFileMetadataRequest.java |    32 +-
 .../hive/metastore/api/ClientCapabilities.java  |    32 +-
 .../hive/metastore/api/CompactionRequest.java   |    44 +-
 .../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 +-
 .../api/HeartbeatTxnRangeResponse.java          |    64 +-
 .../metastore/api/InsertEventRequestData.java   |    64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |    36 +-
 .../api/NotNullConstraintsRequest.java          |   490 +
 .../api/NotNullConstraintsResponse.java         |   443 +
 .../api/NotificationEventResponse.java          |    36 +-
 .../hive/metastore/api/OpenTxnsResponse.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/SQLNotNullConstraint.java     |  1005 +
 .../hive/metastore/api/SQLUniqueConstraint.java |  1103 +
 .../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/ThriftHiveMetastore.java | 43424 +++++++++--------
 .../metastore/api/UniqueConstraintsRequest.java |   490 +
 .../api/UniqueConstraintsResponse.java          |   443 +
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2554 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  9511 ++--
 .../hive_metastore/ThriftHiveMetastore-remote   |    36 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  2096 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  2207 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   166 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   274 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   128 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    37 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    21 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   111 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   324 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    15 +-
 .../hive/metastore/cache/CachedStore.java       |    37 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |    76 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   102 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |   131 +-
 .../hive/metastore/model/MConstraint.java       |     4 +
 .../metastore/hbase/hbase_metastore_proto.proto |    34 +
 .../DummyRawStoreControlledCommit.java          |    32 +-
 .../DummyRawStoreForJdoConnection.java          |    31 +-
 .../InjectableBehaviourObjectStore.java         |    20 +
 .../hive/metastore/hbase/TestHBaseStore.java    |   138 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |     4 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    78 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |    19 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |     4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |     8 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    75 +-
 .../hive/ql/metadata/NotNullConstraint.java     |    86 +
 .../hive/ql/metadata/UniqueConstraint.java      |   111 +
 .../formatting/JsonMetaDataFormatter.java       |    12 +-
 .../formatting/MetaDataFormatUtils.java         |    63 +-
 .../metadata/formatting/MetaDataFormatter.java  |     8 +-
 .../formatting/TextMetaDataFormatter.java       |    44 +-
 .../index/IndexWhereTaskDispatcher.java         |    34 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   447 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    85 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |     1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   158 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    31 +-
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |    44 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |    41 +-
 .../hadoop/hive/ql/plan/ImportTableDesc.java    |     6 +-
 .../txn/compactor/HouseKeeperServiceBase.java   |     6 +-
 .../ql/TestTxnCommands2WithSplitUpdate.java     |    10 +-
 .../hadoop/hive/ql/exec/TestExplainTask.java    |   293 +-
 .../hadoop/hive/ql/hooks/TestATSHook.java       |    59 +
 .../hive/ql/parse/TestHiveDecimalParse.java     |     2 +-
 .../TestSQL11ReservedKeyWordsNegative.java      |    13 +
 .../alter_table_constraint_duplicate_pk.q       |     2 +-
 .../alter_table_constraint_invalid_fk_col1.q    |     4 +-
 .../alter_table_constraint_invalid_fk_col2.q    |     4 +-
 .../alter_table_constraint_invalid_fk_tbl1.q    |     4 +-
 .../alter_table_constraint_invalid_fk_tbl2.q    |     4 +-
 .../alter_table_constraint_invalid_pk_tbl.q     |     2 +-
 .../create_with_constraints_duplicate_name.q    |     4 +-
 .../create_with_constraints_enable.q            |     2 +-
 .../clientnegative/create_with_fk_constraint.q  |     2 +
 .../create_with_multi_pk_constraint.q           |     1 +
 .../clientnegative/drop_invalid_constraint1.q   |     2 +-
 .../clientnegative/drop_invalid_constraint2.q   |     2 +-
 .../clientnegative/drop_invalid_constraint3.q   |     2 +-
 .../clientnegative/drop_invalid_constraint4.q   |     4 +-
 .../clientpositive/create_with_constraints.q    |    84 +-
 .../alter_table_constraint_duplicate_pk.q.out   |     4 +-
 ...alter_table_constraint_invalid_fk_col1.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_col2.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_tbl1.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_tbl2.q.out |     8 +-
 .../alter_table_constraint_invalid_pk_tbl.q.out |     4 +-
 ...create_with_constraints_duplicate_name.q.out |     6 +-
 .../create_with_constraints_enable.q.out        |     2 +-
 .../create_with_constraints_validate.q.out      |     2 +-
 .../create_with_fk_constraint.q.out             |    13 +
 .../create_with_multi_pk_constraint.q.out       |     1 +
 .../drop_invalid_constraint1.q.out              |     4 +-
 .../drop_invalid_constraint2.q.out              |     4 +-
 .../drop_invalid_constraint3.q.out              |     4 +-
 .../drop_invalid_constraint4.q.out              |     8 +-
 .../create_with_constraints.q.out               |  1241 +-
 .../clientpositive/llap/mm_conversions.q.out    |   135 +
 .../results/clientpositive/llap/sysdb.q.out     |   152 +-
 .../clientpositive/tez/explainuser_3.q.out      |     5 -
 .../hive/spark/client/SparkClientImpl.java      |    12 +-
 .../org/apache/hive/spark/client/rpc/Rpc.java   |     7 +-
 .../hive/spark/client/rpc/SaslHandler.java      |     2 +-
 165 files changed, 58646 insertions(+), 32439 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index e44cb9b,52bfb26..feea615
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@@ -6900,18 -7036,57 +6965,68 @@@ public class HiveMetaStore extends Thri
        return new ForeignKeysResponse(ret);
      }
  
 +    private void throwMetaException(Exception e) throws MetaException,
 +        NoSuchObjectException {
 +      if (e instanceof MetaException) {
 +        throw (MetaException) e;
 +      } else if (e instanceof NoSuchObjectException) {
 +        throw (NoSuchObjectException) e;
 +      } else {
 +        throw newMetaException(e);
 +      }
 +    }
 +
      @Override
+     public UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request)
+         throws MetaException, NoSuchObjectException, TException {
+       String db_name = request.getDb_name();
+       String tbl_name = request.getTbl_name();
+       startTableFunction("get_unique_constraints", db_name, tbl_name);
+       List<SQLUniqueConstraint> ret = null;
+       Exception ex = null;
+       try {
+         ret = getMS().getUniqueConstraints(db_name, tbl_name);
+       } catch (Exception e) {
+         ex = e;
+         if (e instanceof MetaException) {
+           throw (MetaException) e;
+         } else if (e instanceof NoSuchObjectException) {
+           throw (NoSuchObjectException) e;
+         } else {
+           throw newMetaException(e);
+         }
+       } finally {
+         endFunction("get_unique_constraints", ret != null, ex, tbl_name);
+       }
+       return new UniqueConstraintsResponse(ret);
+     }
+ 
+     @Override
+     public NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest request)
+         throws MetaException, NoSuchObjectException, TException {
+       String db_name = request.getDb_name();
+       String tbl_name = request.getTbl_name();
+       startTableFunction("get_not_null_constraints", db_name, tbl_name);
+       List<SQLNotNullConstraint> ret = null;
+       Exception ex = null;
+       try {
+         ret = getMS().getNotNullConstraints(db_name, tbl_name);
+       } catch (Exception e) {
+         ex = e;
+         if (e instanceof MetaException) {
+           throw (MetaException) e;
+         } else if (e instanceof NoSuchObjectException) {
+           throw (NoSuchObjectException) e;
+         } else {
+           throw newMetaException(e);
+         }
+       } finally {
+         endFunction("get_not_null_constraints", ret != null, ex, tbl_name);
+       }
+       return new NotNullConstraintsResponse(ret);
+     }
+ 
+     @Override
      public String get_metastore_db_uuid() throws MetaException, TException {
        try {
          return getMS().getMetastoreDbUuid();

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index f67831e,617555e..0b7a031
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@@ -3994,238 -4009,41 +4002,249 @@@ public class DDLTask extends Task<DDLWo
        throw new HiveException(ErrorMsg.UNSUPPORTED_ALTER_TBL_OP, alterTbl.getOp().toString());
      }
  
 -    return 0;
 +    return null;
 +  }
 +
 +  private List<Task<?>> alterTableDropProps(AlterTableDesc alterTbl, Table tbl,
 +      Partition part, EnvironmentContext environmentContext) throws HiveException {
 +    if (StatsSetupConst.USER.equals(environmentContext.getProperties()
 +        .get(StatsSetupConst.STATS_GENERATED))) {
 +      // drop a stats parameter, which triggers recompute stats update automatically
 +      environmentContext.getProperties().remove(StatsSetupConst.DO_NOT_UPDATE_STATS);
 +    }
 +
 +    List<Task<?>> result = null;
 +    if (part == null) {
 +      Set<String> removedSet = alterTbl.getProps().keySet();
 +      boolean isFromMmTable = MetaStoreUtils.isInsertOnlyTable(tbl.getParameters()),
 +          isRemoved = MetaStoreUtils.isRemovedInsertOnlyTable(removedSet);
 +      if (isFromMmTable && isRemoved) {
 +        result = generateRemoveMmTasks(tbl);
 +      }
 +    }
 +    Iterator<String> keyItr = alterTbl.getProps().keySet().iterator();
 +    while (keyItr.hasNext()) {
 +      if (part != null) {
 +        part.getTPartition().getParameters().remove(keyItr.next());
 +      } else {
 +        tbl.getTTable().getParameters().remove(keyItr.next());
 +      }
 +    }
 +    return result;
 +  }
 +
 +  private List<Task<?>> generateRemoveMmTasks(Table tbl) throws HiveException {
 +    // To avoid confusion from nested MM directories when table is converted back and forth, we
 +    // want to rename mm_ dirs to remove the prefix; however, given the unpredictable nested
 +    // directory handling in Hive/MR, we will instead move all the files into the root directory.
 +    // We will also delete any directories that are not committed. 
 +    // Note that this relies on locks. Note also that we only do the renames AFTER the metastore
 +    // operation commits. Deleting uncommitted things is safe, but moving stuff before we convert
 +    // could cause data loss.
 +    List<Path> allMmDirs = new ArrayList<>();
 +    if (tbl.isStoredAsSubDirectories()) {
 +      // TODO: support this? we only bail because it's a PITA and hardly anyone seems to care.
 +      throw new HiveException("Converting list bucketed tables stored as subdirectories "
 +          + " to and from MM is not supported");
 +    }
 +    List<String> bucketCols = tbl.getBucketCols();
 +    if (bucketCols != null && !bucketCols.isEmpty()
 +        && HiveConf.getBoolVar(conf, ConfVars.HIVE_STRICT_CHECKS_BUCKETING)) {
 +      throw new HiveException("Converting bucketed tables from MM is not supported by default; "
 +          + "copying files from multiple MM directories may potentially break the buckets. You "
 +          + "can set " + ConfVars.HIVE_STRICT_CHECKS_BUCKETING.varname
 +          + " to false for this query if you want to force the conversion.");
 +    }
 +    Hive db = getHive();
 +    String value = conf.get(ValidTxnList.VALID_TXNS_KEY);
 +    ValidTxnList validTxnList = value == null ? new ValidReadTxnList() : new ValidReadTxnList(value);
 +    if (tbl.getPartitionKeys().size() > 0) {
 +      PartitionIterable parts = new PartitionIterable(db, tbl, null,
 +          HiveConf.getIntVar(conf, ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
 +      Iterator<Partition> partIter = parts.iterator();
 +      while (partIter.hasNext()) {
 +        Partition part = partIter.next();
 +        checkMmLb(part);
 +        handleRemoveMm(part.getDataLocation(), validTxnList, allMmDirs);
 +      }
 +    } else {
 +      checkMmLb(tbl);
 +      handleRemoveMm(tbl.getDataLocation(), validTxnList, allMmDirs);
 +    }
 +    List<Path> targetPaths = new ArrayList<>(allMmDirs.size());
 +    List<String> targetPrefix = new ArrayList<>(allMmDirs.size());
 +    int prefixLen = JavaUtils.DELTA_PREFIX.length();
 +    for (int i = 0; i < allMmDirs.size(); ++i) {
 +      Path src = allMmDirs.get(i);
 +      Path tgt = src.getParent();
 +      String prefix = src.getName().substring(prefixLen + 1) + "_";
 +      Utilities.LOG14535.info("Will move " + src + " to " + tgt + " (prefix " + prefix + ")");
 +      targetPaths.add(tgt);
 +      targetPrefix.add(prefix);
 +    }
 +    // Don't set inputs and outputs - the locks have already been taken so it's pointless.
 +    MoveWork mw = new MoveWork(null, null, null, null, false);
 +    mw.setMultiFilesDesc(new LoadMultiFilesDesc(
 +        allMmDirs, targetPaths, targetPrefix, true, null, null));
 +    return Lists.<Task<?>>newArrayList(TaskFactory.get(mw, conf));
 +  }
 +
 +  private void checkMmLb(Table tbl) throws HiveException {
 +    if (!tbl.isStoredAsSubDirectories()) return;
 +    // TODO: support this?
 +    throw new HiveException("Converting list bucketed tables stored as subdirectories "
 +        + " to and from MM is not supported");
 +  }
 +
 +  private void checkMmLb(Partition part) throws HiveException {
 +    if (!part.isStoredAsSubDirectories()) return;
 +    // TODO: support this?
 +    throw new HiveException("Converting list bucketed tables stored as subdirectories "
 +        + " to and from MM is not supported. Please create a table in the desired format.");
 +  }
 +
 +  private void handleRemoveMm(
 +      Path path, ValidTxnList validTxnList, List<Path> result) throws HiveException {
 +    // Note: doesn't take LB into account; that is not presently supported here (throws above).
 +    try {
 +      FileSystem fs = path.getFileSystem(conf);
 +      for (FileStatus file : fs.listStatus(path)) {
 +        Path childPath = file.getPath();
 +        if (!file.isDirectory()) {
 +          ensureDelete(fs, childPath, "a non-directory file");
 +          continue;
 +        }
 +        Long writeId = JavaUtils.extractTxnId(childPath);
 +        if (writeId == null) {
 +          ensureDelete(fs, childPath, "an unknown directory");
 +        } else if (!validTxnList.isTxnValid(writeId)) {
 +          // Assume no concurrent active writes - we rely on locks here. We could check and fail.
 +          ensureDelete(fs, childPath, "an uncommitted directory");
 +        } else {
 +          result.add(childPath);
 +        }
 +      }
 +    } catch (IOException ex) {
 +      throw new HiveException(ex);
 +    }
 +  }
 +
 +  private static void ensureDelete(FileSystem fs, Path path, String what) throws IOException {
 +    Utilities.LOG14535.info("Deleting " + what + " " + path);
 +    try {
 +      if (!fs.delete(path, true)) throw new IOException("delete returned false");
 +    } catch (Exception ex) {
 +      String error = "Couldn't delete " + path + "; cannot remove MM setting from the table";
 +      LOG.error(error, ex);
 +      throw (ex instanceof IOException) ? (IOException)ex : new IOException(ex);
 +    }
 +  }
 +
 +  private List<Task<?>> generateAddMmTasks(Table tbl) throws HiveException {
 +    // We will move all the files in the table/partition directories into the first MM
 +    // directory, then commit the first write ID.
 +    List<Path> srcs = new ArrayList<>(), tgts = new ArrayList<>();
 +    long mmWriteId = 0;
 +    try {
 +      HiveTxnManager txnManager = SessionState.get().getTxnMgr();
 +      mmWriteId = txnManager.openTxn(new Context(conf), conf.getUser());
 +      txnManager.commitTxn();
 +    } catch (Exception e) {
 +      String errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage();
 +      console.printError(errorMessage, "\n"
 +          + org.apache.hadoop.util.StringUtils.stringifyException(e));
 +    }
 +    int stmtId = 0;
 +    String mmDir = AcidUtils.deltaSubdir(mmWriteId, mmWriteId, stmtId);
 +    Hive db = getHive();
 +    if (tbl.getPartitionKeys().size() > 0) {
 +      PartitionIterable parts = new PartitionIterable(db, tbl, null,
 +          HiveConf.getIntVar(conf, ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
 +      Iterator<Partition> partIter = parts.iterator();
 +      while (partIter.hasNext()) {
 +        Partition part = partIter.next();
 +        checkMmLb(part);
 +        Path src = part.getDataLocation(), tgt = new Path(src, mmDir);
 +        srcs.add(src);
 +        tgts.add(tgt);
 +        Utilities.LOG14535.info("Will move " + src + " to " + tgt);
 +      }
 +    } else {
 +      checkMmLb(tbl);
 +      Path src = tbl.getDataLocation(), tgt = new Path(src, mmDir);
 +      srcs.add(src);
 +      tgts.add(tgt);
 +      Utilities.LOG14535.info("Will move " + src + " to " + tgt);
 +    }
 +    // Don't set inputs and outputs - the locks have already been taken so it's pointless.
 +    MoveWork mw = new MoveWork(null, null, null, null, false);
 +    mw.setMultiFilesDesc(new LoadMultiFilesDesc(srcs, tgts, true, null, null));
 +    ImportCommitWork icw = new ImportCommitWork(tbl.getDbName(), tbl.getTableName(), mmWriteId, stmtId);
 +    Task<?> mv = TaskFactory.get(mw, conf), ic = TaskFactory.get(icw, conf);
 +    mv.addDependentTask(ic);
 +    return Lists.<Task<?>>newArrayList(mv);
 +  }
 +
 +  private List<Task<?>> alterTableAddProps(AlterTableDesc alterTbl, Table tbl,
 +      Partition part, EnvironmentContext environmentContext) throws HiveException {
 +    if (StatsSetupConst.USER.equals(environmentContext.getProperties()
 +        .get(StatsSetupConst.STATS_GENERATED))) {
 +      environmentContext.getProperties().remove(StatsSetupConst.DO_NOT_UPDATE_STATS);
 +    }
 +    if(alterTbl.getProps().containsKey(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY)) {
 +      NanoTimeUtils.validateTimeZone(
 +          alterTbl.getProps().get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY));
 +    }
 +    List<Task<?>> result = null;
 +    if (part != null) {
 +      part.getTPartition().getParameters().putAll(alterTbl.getProps());
 +    } else {
 +      boolean isFromMmTable = MetaStoreUtils.isInsertOnlyTable(tbl.getParameters());
 +      Boolean isToMmTable = MetaStoreUtils.isToInsertOnlyTable(alterTbl.getProps());
 +      if (isToMmTable != null) {
 +        if (!isFromMmTable && isToMmTable) {
 +          result = generateAddMmTasks(tbl);
 +        } else if (isFromMmTable && !isToMmTable) {
 +          result = generateRemoveMmTasks(tbl);
 +        }
 +      }
 +      tbl.getTTable().getParameters().putAll(alterTbl.getProps());
 +    }
 +    return result;
    }
  
-    private int dropConstraint(Hive db, AlterTableDesc alterTbl)
-     throws SemanticException, HiveException {
-      try {
-       db.dropConstraint(Utilities.getDatabaseName(alterTbl.getOldName()),
-         Utilities.getTableName(alterTbl.getOldName()),
-           alterTbl.getConstraintName());
-       } catch (NoSuchObjectException e) {
-         throw new HiveException(e);
-       }
-      return 0;
-    }
- 
-    private int addConstraint(Hive db, AlterTableDesc alterTbl)
-     throws SemanticException, HiveException {
+   private int dropConstraint(Hive db, AlterTableDesc alterTbl)
+           throws SemanticException, HiveException {
      try {
-     // This is either an alter table add foreign key or add primary key command.
-     if (!alterTbl.getForeignKeyCols().isEmpty()) {
-        db.addForeignKey(alterTbl.getForeignKeyCols());
-      } else if (!alterTbl.getPrimaryKeyCols().isEmpty()) {
-        db.addPrimaryKey(alterTbl.getPrimaryKeyCols());
+      db.dropConstraint(Utilities.getDatabaseName(alterTbl.getOldName()),
+        Utilities.getTableName(alterTbl.getOldName()),
+          alterTbl.getConstraintName());
+      } catch (NoSuchObjectException e) {
+        throw new HiveException(e);
       }
+     return 0;
+   }
+ 
+   private int addConstraints(Hive db, AlterTableDesc alterTbl)
+            throws SemanticException, HiveException {
+     try {
+       // This is either an alter table add foreign key or add primary key command.
+       if (alterTbl.getForeignKeyCols() != null
+               && !alterTbl.getForeignKeyCols().isEmpty()) {
+         db.addForeignKey(alterTbl.getForeignKeyCols());
+       }
+       if (alterTbl.getPrimaryKeyCols() != null
+               && !alterTbl.getPrimaryKeyCols().isEmpty()) {
+         db.addPrimaryKey(alterTbl.getPrimaryKeyCols());
+       }
+       if (alterTbl.getUniqueConstraintCols() != null
+               && !alterTbl.getUniqueConstraintCols().isEmpty()) {
+         db.addUniqueConstraint(alterTbl.getUniqueConstraintCols());
+       }
+       if (alterTbl.getNotNullConstraintCols() != null
+               && !alterTbl.getNotNullConstraintCols().isEmpty()) {
+         db.addNotNullConstraint(alterTbl.getNotNullConstraintCols());
+       }
      } catch (NoSuchObjectException e) {
        throw new HiveException(e);
      }

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index e723e2f,da00bb3..f325c0e
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@@ -345,11 -344,8 +345,11 @@@ public class AcidUtils 
      public static final String SPLIT_UPDATE_STRING = "split_update";
      public static final int HASH_BASED_MERGE_BIT = 0x02;
      public static final String HASH_BASED_MERGE_STRING = "hash_merge";
-     public static final int INSERT_ONLY_BIT = 0x03;
++    public static final int INSERT_ONLY_BIT = 0x04;
 +    public static final String INSERT_ONLY_STRING = "insert_only";
      public static final String DEFAULT_VALUE_STRING = TransactionalValidationListener.DEFAULT_TRANSACTIONAL_PROPERTY;
      public static final String LEGACY_VALUE_STRING = TransactionalValidationListener.LEGACY_TRANSACTIONAL_PROPERTY;
 +    public static final String INSERTONLY_VALUE_STRING = TransactionalValidationListener.INSERTONLY_TRANSACTIONAL_PROPERTY;
  
      private AcidOperationalProperties() {
      }
@@@ -374,17 -370,6 +374,18 @@@
        AcidOperationalProperties obj = new AcidOperationalProperties();
        obj.setSplitUpdate(true);
        obj.setHashBasedMerge(false);
++      obj.setInsertOnly(false);
 +      return obj;
 +    }
 +
 +    /**
 +     * Returns an acidOperationalProperties object for tables that uses ACID framework but only
 +     * supports INSERT operation and does not require ORC or bucketing
 +     * @return the acidOperationalProperties object
 +     */
 +    public static AcidOperationalProperties getInsertOnly() {
 +      AcidOperationalProperties obj = new AcidOperationalProperties();
 +      obj.setInsertOnly(true);
        return obj;
      }
  
@@@ -417,6 -399,6 +418,8 @@@
            case HASH_BASED_MERGE_STRING:
              obj.setHashBasedMerge(true);
              break;
++          case INSERT_ONLY_STRING:
++            obj.setInsertOnly(true);
            default:
              throw new IllegalArgumentException(
                  "Unexpected value " + option + " for ACID operational properties!");
@@@ -438,6 -420,6 +441,9 @@@
        if ((properties & HASH_BASED_MERGE_BIT)  > 0) {
          obj.setHashBasedMerge(true);
        }
++      if ((properties & INSERT_ONLY_BIT) > 0) {
++        obj.setInsertOnly(true);
++      }
        return obj;
      }
  

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index a16bf91,3f032c8..66c662f
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@@ -65,10 -62,8 +65,9 @@@ import org.apache.hadoop.fs.FileStatus
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.Path;
  import org.apache.hadoop.fs.PathFilter;
- import org.apache.hadoop.hive.common.BlobStorageUtils;
  import org.apache.hadoop.hive.common.FileUtils;
  import org.apache.hadoop.hive.common.HiveStatsUtils;
 +import org.apache.hadoop.hive.common.JavaUtils;
  import org.apache.hadoop.hive.common.ObjectPair;
  import org.apache.hadoop.hive.common.StatsSetupConst;
  import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index e36623c,dee6a10..d5947bd
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@@ -41,9 -42,10 +42,11 @@@ import org.apache.hadoop.hive.metastore
  import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
  import org.apache.hadoop.hive.metastore.api.Order;
  import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
  import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+ import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
  import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
  import org.apache.hadoop.hive.ql.Driver;
  import org.apache.hadoop.hive.ql.ErrorMsg;
  import org.apache.hadoop.hive.ql.QueryState;

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
index 344893c,7b46fcd..f9314a7
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
@@@ -96,10 -98,8 +98,12 @@@ public class CreateTableDesc extends DD
    private boolean isCTAS = false;
    List<SQLPrimaryKey> primaryKeys;
    List<SQLForeignKey> foreignKeys;
+   List<SQLUniqueConstraint> uniqueConstraints;
+   List<SQLNotNullConstraint> notNullConstraints;
 +  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.
 +  private transient FileSinkDesc writer;
  
    public CreateTableDesc() {
    }

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2WithSplitUpdate.java
----------------------------------------------------------------------
diff --cc ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2WithSplitUpdate.java
index ea5ecbc,ea5ecbc..d9be605
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2WithSplitUpdate.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2WithSplitUpdate.java
@@@ -444,11 -444,11 +444,11 @@@ public class TestTxnCommands2WithSplitU
          FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.STAGING_DIR_PATH_FILTER);
          Arrays.sort(buckets);
          if (numDelta == 1) {
--          Assert.assertEquals("delta_0000022_0000022_0000", status[i].getPath().getName());
++          Assert.assertEquals("delta_0000024_0000024_0000", status[i].getPath().getName());
            Assert.assertEquals(BUCKET_COUNT - 1, buckets.length);
            Assert.assertEquals("bucket_00001", buckets[0].getPath().getName());
          } else if (numDelta == 2) {
--          Assert.assertEquals("delta_0000023_0000023_0000", status[i].getPath().getName());
++          Assert.assertEquals("delta_0000025_0000025_0000", status[i].getPath().getName());
            Assert.assertEquals(1, buckets.length);
            Assert.assertEquals("bucket_00001", buckets[0].getPath().getName());
          }
@@@ -457,7 -457,7 +457,7 @@@
          FileStatus[] buckets = fs.listStatus(status[i].getPath(), FileUtils.STAGING_DIR_PATH_FILTER);
          Arrays.sort(buckets);
          if (numDeleteDelta == 1) {
--          Assert.assertEquals("delete_delta_0000022_0000022_0000", status[i].getPath().getName());
++          Assert.assertEquals("delete_delta_0000024_0000024_0000", status[i].getPath().getName());
            Assert.assertEquals(BUCKET_COUNT - 1, buckets.length);
            Assert.assertEquals("bucket_00001", buckets[0].getPath().getName());
          }
@@@ -504,7 -504,7 +504,7 @@@
            Assert.assertEquals("bucket_00001", buckets[0].getPath().getName());
          } else if (numBase == 2) {
            // The new base dir now has two bucket files, since the delta dir has two bucket files
--          Assert.assertEquals("base_0000023", status[i].getPath().getName());
++          Assert.assertEquals("base_0000025", status[i].getPath().getName());
            Assert.assertEquals(1, buckets.length);
            Assert.assertEquals("bucket_00001", buckets[0].getPath().getName());
          }
@@@ -530,7 -530,7 +530,7 @@@
      status = fs.listStatus(new Path(TEST_WAREHOUSE_DIR + "/" +
          (Table.NONACIDORCTBL).toString().toLowerCase()), FileUtils.STAGING_DIR_PATH_FILTER);
      Assert.assertEquals(1, status.length);
--    Assert.assertEquals("base_0000023", status[0].getPath().getName());
++    Assert.assertEquals("base_0000025", status[0].getPath().getName());
      FileStatus[] buckets = fs.listStatus(status[0].getPath(), FileUtils.STAGING_DIR_PATH_FILTER);
      Arrays.sort(buckets);
      Assert.assertEquals(1, buckets.length);

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/test/results/clientpositive/llap/mm_conversions.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/llap/mm_conversions.q.out
index 861acaf,0000000..1610672
mode 100644,000000..100644
--- a/ql/src/test/results/clientpositive/llap/mm_conversions.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_conversions.q.out
@@@ -1,720 -1,0 +1,855 @@@
 +PREHOOK: query: drop table intermediate
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermediate
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table intermediate(key int) partitioned by (p int) stored as orc
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@intermediate
 +POSTHOOK: query: create table intermediate(key int) partitioned by (p int) stored as orc
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@intermediate
 +PREHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 1
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=455
 +POSTHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 1
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=455
 +POSTHOOK: Lineage: intermediate PARTITION(p=455).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 1
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=456
 +POSTHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 1
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=456
 +POSTHOOK: Lineage: intermediate PARTITION(p=456).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: insert into table intermediate partition(p='457') select distinct key from src where key >= 100 order by key asc limit 1
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=457
 +POSTHOOK: query: insert into table intermediate partition(p='457') select distinct key from src where key >= 100 order by key asc limit 1
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=457
 +POSTHOOK: Lineage: intermediate PARTITION(p=457).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: drop table simple_from_mm1
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table simple_from_mm1
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table simple_from_mm1(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: create table simple_from_mm1(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@simple_from_mm1
 +PREHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm1
 +POSTHOOK: Lineage: simple_from_mm1.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm1
 +POSTHOOK: Lineage: simple_from_mm1.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_from_mm1 s1 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm1 s1 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
++0
++0
++98
++98
++100
++100
 +PREHOOK: query: alter table simple_from_mm1 unset tblproperties('transactional_properties', 'transactional')
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@simple_from_mm1
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: alter table simple_from_mm1 unset tblproperties('transactional_properties', 'transactional')
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@simple_from_mm1
 +POSTHOOK: Output: default@simple_from_mm1
 +PREHOOK: query: select * from simple_from_mm1 s2 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm1 s2 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
++0
++0
++98
++98
++100
++100
 +PREHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: insert into table simple_from_mm1 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm1
 +POSTHOOK: Lineage: simple_from_mm1.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_from_mm1 s3 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm1 s3 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm1
 +#### A masked pattern was here ####
++0
++0
++0
++98
++98
++98
++100
++100
++100
 +PREHOOK: query: drop table simple_from_mm1
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@simple_from_mm1
 +PREHOOK: Output: default@simple_from_mm1
 +POSTHOOK: query: drop table simple_from_mm1
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@simple_from_mm1
 +POSTHOOK: Output: default@simple_from_mm1
 +PREHOOK: query: drop table simple_from_mm2
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table simple_from_mm2
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table simple_from_mm2(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: create table simple_from_mm2(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@simple_from_mm2
 +PREHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm2
 +POSTHOOK: Lineage: simple_from_mm2.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm2
 +POSTHOOK: Lineage: simple_from_mm2.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_from_mm2 s1 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm2 s1 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
++0
++0
++98
++98
++100
++100
 +PREHOOK: query: alter table simple_from_mm2 set tblproperties("transactional"="false", 'transactional_properties'='false')
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@simple_from_mm2
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: alter table simple_from_mm2 set tblproperties("transactional"="false", 'transactional_properties'='false')
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@simple_from_mm2
 +POSTHOOK: Output: default@simple_from_mm2
 +PREHOOK: query: select * from simple_from_mm2 s2 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm2 s2 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
++0
++0
++98
++98
++100
++100
 +PREHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: insert into table simple_from_mm2 select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_from_mm2
 +POSTHOOK: Lineage: simple_from_mm2.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_from_mm2 s3 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_from_mm2 s3 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_from_mm2
 +#### A masked pattern was here ####
++0
++0
++0
++98
++98
++98
++100
++100
++100
 +PREHOOK: query: drop table simple_from_mm2
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@simple_from_mm2
 +PREHOOK: Output: default@simple_from_mm2
 +POSTHOOK: query: drop table simple_from_mm2
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@simple_from_mm2
 +POSTHOOK: Output: default@simple_from_mm2
 +PREHOOK: query: drop table simple_to_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table simple_to_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table simple_to_mm(key int) stored as orc
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@simple_to_mm
 +POSTHOOK: query: create table simple_to_mm(key int) stored as orc
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@simple_to_mm
 +PREHOOK: query: insert into table simple_to_mm select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_to_mm
 +POSTHOOK: query: insert into table simple_to_mm select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_to_mm
 +POSTHOOK: Lineage: simple_to_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_to_mm s1 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_to_mm s1 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
++0
++98
++100
 +PREHOOK: query: alter table simple_to_mm set tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@simple_to_mm
 +PREHOOK: Output: default@simple_to_mm
 +FAILED: Error in acquiring locks: Transaction already opened. txnid:30
 +POSTHOOK: query: alter table simple_to_mm set tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@simple_to_mm
 +POSTHOOK: Output: default@simple_to_mm
 +PREHOOK: query: select * from simple_to_mm s2 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_to_mm s2 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
++0
++98
++100
 +PREHOOK: query: insert into table simple_to_mm select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_to_mm
 +POSTHOOK: query: insert into table simple_to_mm select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_to_mm
 +POSTHOOK: Lineage: simple_to_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table simple_to_mm select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@simple_to_mm
 +POSTHOOK: query: insert into table simple_to_mm select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@simple_to_mm
 +POSTHOOK: Lineage: simple_to_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_to_mm s3 order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_to_mm s3 order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_to_mm
 +#### A masked pattern was here ####
++0
++0
++0
++98
++98
++98
++100
++100
++100
 +PREHOOK: query: drop table simple_to_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@simple_to_mm
 +PREHOOK: Output: default@simple_to_mm
 +POSTHOOK: query: drop table simple_to_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@simple_to_mm
 +POSTHOOK: Output: default@simple_to_mm
 +PREHOOK: query: drop table part_from_mm1
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table part_from_mm1
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table part_from_mm1(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@part_from_mm1
 +POSTHOOK: query: create table part_from_mm1(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@part_from_mm1
 +PREHOOK: query: insert into table part_from_mm1 partition(key_mm='455') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm1@key_mm=455
 +POSTHOOK: query: insert into table part_from_mm1 partition(key_mm='455') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm1@key_mm=455
 +POSTHOOK: Lineage: part_from_mm1 PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table part_from_mm1 partition(key_mm='455') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm1@key_mm=455
 +POSTHOOK: query: insert into table part_from_mm1 partition(key_mm='455') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm1@key_mm=455
 +POSTHOOK: Lineage: part_from_mm1 PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table part_from_mm1 partition(key_mm='456') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm1@key_mm=456
 +POSTHOOK: query: insert into table part_from_mm1 partition(key_mm='456') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm1@key_mm=456
 +POSTHOOK: Lineage: part_from_mm1 PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_from_mm1 s1 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm1
 +PREHOOK: Input: default@part_from_mm1@key_mm=455
 +PREHOOK: Input: default@part_from_mm1@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm1 s1 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm1
 +POSTHOOK: Input: default@part_from_mm1@key_mm=455
 +POSTHOOK: Input: default@part_from_mm1@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	455
++0	456
++98	455
++98	455
++98	456
++100	455
++100	455
++100	456
 +PREHOOK: query: alter table part_from_mm1 unset tblproperties('transactional_properties', 'transactional')
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@part_from_mm1
 +PREHOOK: Output: default@part_from_mm1
 +POSTHOOK: query: alter table part_from_mm1 unset tblproperties('transactional_properties', 'transactional')
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@part_from_mm1
 +POSTHOOK: Output: default@part_from_mm1
 +PREHOOK: query: select * from part_from_mm1 s2 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm1
 +PREHOOK: Input: default@part_from_mm1@key_mm=455
 +PREHOOK: Input: default@part_from_mm1@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm1 s2 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm1
 +POSTHOOK: Input: default@part_from_mm1@key_mm=455
 +POSTHOOK: Input: default@part_from_mm1@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	455
++0	456
++98	455
++98	455
++98	456
++100	455
++100	455
++100	456
 +PREHOOK: query: insert into table part_from_mm1 partition(key_mm='456') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm1@key_mm=456
 +POSTHOOK: query: insert into table part_from_mm1 partition(key_mm='456') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm1@key_mm=456
 +POSTHOOK: Lineage: part_from_mm1 PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table part_from_mm1 partition(key_mm='457') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm1@key_mm=457
 +POSTHOOK: query: insert into table part_from_mm1 partition(key_mm='457') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm1@key_mm=457
 +POSTHOOK: Lineage: part_from_mm1 PARTITION(key_mm=457).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_from_mm1 s3 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm1
 +PREHOOK: Input: default@part_from_mm1@key_mm=455
 +PREHOOK: Input: default@part_from_mm1@key_mm=456
 +PREHOOK: Input: default@part_from_mm1@key_mm=457
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm1 s3 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm1
 +POSTHOOK: Input: default@part_from_mm1@key_mm=455
 +POSTHOOK: Input: default@part_from_mm1@key_mm=456
 +POSTHOOK: Input: default@part_from_mm1@key_mm=457
 +#### A masked pattern was here ####
++0	455
++0	455
++0	456
++0	456
++0	457
++98	455
++98	455
++98	456
++98	456
++98	457
++100	455
++100	455
++100	456
++100	456
++100	457
 +PREHOOK: query: drop table part_from_mm1
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@part_from_mm1
 +PREHOOK: Output: default@part_from_mm1
 +POSTHOOK: query: drop table part_from_mm1
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@part_from_mm1
 +POSTHOOK: Output: default@part_from_mm1
 +PREHOOK: query: drop table part_from_mm2
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table part_from_mm2
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table part_from_mm2(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@part_from_mm2
 +POSTHOOK: query: create table part_from_mm2(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@part_from_mm2
 +PREHOOK: query: insert into table part_from_mm2 partition(key_mm='456') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm2@key_mm=456
 +POSTHOOK: query: insert into table part_from_mm2 partition(key_mm='456') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm2@key_mm=456
 +POSTHOOK: Lineage: part_from_mm2 PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: --fails here
 +insert into table part_from_mm2 partition(key_mm='455') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm2@key_mm=455
 +POSTHOOK: query: --fails here
 +insert into table part_from_mm2 partition(key_mm='455') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm2@key_mm=455
 +POSTHOOK: Lineage: part_from_mm2 PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_from_mm2 s1 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm2
 +PREHOOK: Input: default@part_from_mm2@key_mm=455
 +PREHOOK: Input: default@part_from_mm2@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm2 s1 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm2
 +POSTHOOK: Input: default@part_from_mm2@key_mm=455
 +POSTHOOK: Input: default@part_from_mm2@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	456
++98	455
++98	456
++100	455
++100	456
 +PREHOOK: query: alter table part_from_mm2 set tblproperties("transactional"="false", 'transactional_properties'='false')
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@part_from_mm2
 +PREHOOK: Output: default@part_from_mm2
 +POSTHOOK: query: alter table part_from_mm2 set tblproperties("transactional"="false", 'transactional_properties'='false')
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@part_from_mm2
 +POSTHOOK: Output: default@part_from_mm2
 +PREHOOK: query: select * from part_from_mm2 s2 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm2
 +PREHOOK: Input: default@part_from_mm2@key_mm=455
 +PREHOOK: Input: default@part_from_mm2@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm2 s2 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm2
 +POSTHOOK: Input: default@part_from_mm2@key_mm=455
 +POSTHOOK: Input: default@part_from_mm2@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	456
++98	455
++98	456
++100	455
++100	456
 +PREHOOK: query: insert into table part_from_mm2 partition(key_mm='457') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_from_mm2@key_mm=457
 +POSTHOOK: query: insert into table part_from_mm2 partition(key_mm='457') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_from_mm2@key_mm=457
 +POSTHOOK: Lineage: part_from_mm2 PARTITION(key_mm=457).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_from_mm2 s3 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_from_mm2
 +PREHOOK: Input: default@part_from_mm2@key_mm=455
 +PREHOOK: Input: default@part_from_mm2@key_mm=456
 +PREHOOK: Input: default@part_from_mm2@key_mm=457
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_from_mm2 s3 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_from_mm2
 +POSTHOOK: Input: default@part_from_mm2@key_mm=455
 +POSTHOOK: Input: default@part_from_mm2@key_mm=456
 +POSTHOOK: Input: default@part_from_mm2@key_mm=457
 +#### A masked pattern was here ####
++0	455
++0	456
++0	457
++98	455
++98	456
++98	457
++100	455
++100	456
++100	457
 +PREHOOK: query: drop table part_from_mm2
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@part_from_mm2
 +PREHOOK: Output: default@part_from_mm2
 +POSTHOOK: query: drop table part_from_mm2
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@part_from_mm2
 +POSTHOOK: Output: default@part_from_mm2
 +PREHOOK: query: drop table part_to_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table part_to_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table part_to_mm(key int) partitioned by (key_mm int) stored as orc
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@part_to_mm
 +POSTHOOK: query: create table part_to_mm(key int) partitioned by (key_mm int) stored as orc
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@part_to_mm
 +PREHOOK: query: insert into table part_to_mm partition(key_mm='455') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_to_mm@key_mm=455
 +POSTHOOK: query: insert into table part_to_mm partition(key_mm='455') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_to_mm@key_mm=455
 +POSTHOOK: Lineage: part_to_mm PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table part_to_mm partition(key_mm='456') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_to_mm@key_mm=456
 +POSTHOOK: query: insert into table part_to_mm partition(key_mm='456') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_to_mm@key_mm=456
 +POSTHOOK: Lineage: part_to_mm PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_to_mm s1 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_to_mm
 +PREHOOK: Input: default@part_to_mm@key_mm=455
 +PREHOOK: Input: default@part_to_mm@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_to_mm s1 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_to_mm
 +POSTHOOK: Input: default@part_to_mm@key_mm=455
 +POSTHOOK: Input: default@part_to_mm@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	456
++98	455
++98	456
++100	455
++100	456
 +PREHOOK: query: alter table part_to_mm set tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: ALTERTABLE_PROPERTIES
 +PREHOOK: Input: default@part_to_mm
 +PREHOOK: Output: default@part_to_mm
 +FAILED: Error in acquiring locks: Transaction already opened. txnid:63
 +POSTHOOK: query: alter table part_to_mm set tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: ALTERTABLE_PROPERTIES
 +POSTHOOK: Input: default@part_to_mm
 +POSTHOOK: Output: default@part_to_mm
 +PREHOOK: query: select * from part_to_mm s2 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_to_mm
 +PREHOOK: Input: default@part_to_mm@key_mm=455
 +PREHOOK: Input: default@part_to_mm@key_mm=456
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_to_mm s2 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_to_mm
 +POSTHOOK: Input: default@part_to_mm@key_mm=455
 +POSTHOOK: Input: default@part_to_mm@key_mm=456
 +#### A masked pattern was here ####
++0	455
++0	456
++98	455
++98	456
++100	455
++100	456
 +PREHOOK: query: insert into table part_to_mm partition(key_mm='456') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_to_mm@key_mm=456
 +POSTHOOK: query: insert into table part_to_mm partition(key_mm='456') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_to_mm@key_mm=456
 +POSTHOOK: Lineage: part_to_mm PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table part_to_mm partition(key_mm='457') select key from intermediate
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: default@part_to_mm@key_mm=457
 +POSTHOOK: query: insert into table part_to_mm partition(key_mm='457') select key from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: default@part_to_mm@key_mm=457
 +POSTHOOK: Lineage: part_to_mm PARTITION(key_mm=457).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from part_to_mm s3 order by key, key_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@part_to_mm
 +PREHOOK: Input: default@part_to_mm@key_mm=455
 +PREHOOK: Input: default@part_to_mm@key_mm=456
 +PREHOOK: Input: default@part_to_mm@key_mm=457
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from part_to_mm s3 order by key, key_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@part_to_mm
 +POSTHOOK: Input: default@part_to_mm@key_mm=455
 +POSTHOOK: Input: default@part_to_mm@key_mm=456
 +POSTHOOK: Input: default@part_to_mm@key_mm=457
 +#### A masked pattern was here ####
++0	455
++0	456
++0	456
++0	457
++98	455
++98	456
++98	456
++98	457
++100	455
++100	456
++100	456
++100	457
 +PREHOOK: query: drop table part_to_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@part_to_mm
 +PREHOOK: Output: default@part_to_mm
 +POSTHOOK: query: drop table part_to_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@part_to_mm
 +POSTHOOK: Output: default@part_to_mm
 +PREHOOK: query: drop table intermediate
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Output: default@intermediate
 +POSTHOOK: query: drop table intermediate
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Output: default@intermediate

http://git-wip-us.apache.org/repos/asf/hive/blob/52e0f8f3/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index cb867fd,65c9114..f68d2bb
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@@ -509,16 -509,11 +509,11 @@@ Stage-
                    Conditional Operator
                      Stage-1
                        Map 1 vectorized
 -                      File Output Operator [FS_10]
 +                      File Output Operator [FS_8]
                          table:{"name:":"default.orc_merge5"}
 -                        Select Operator [SEL_9] (rows=306 width=268)
 +                        Select Operator [SEL_7] (rows=306 width=268)
                            Output:["_col0","_col1","_col2","_col3","_col4"]
 -                          Filter Operator [FIL_8] (rows=306 width=268)
 +                          Filter Operator [FIL_6] (rows=306 width=268)
-                       File Output Operator [FS_3]
-                         table:{"name:":"default.orc_merge5"}
-                         Select Operator [SEL_2] (rows=306 width=268)
-                           Output:["_col0","_col1","_col2","_col3","_col4"]
-                           Filter Operator [FIL_4] (rows=306 width=268)
                              predicate:(userid <= 13)
                              TableScan [TS_0] (rows=919 width=268)
                                default@orc_merge5,orc_merge5,Tbl:COMPLETE,Col:NONE,Output:["userid","string1","subtype","decimal1","ts"]


[15/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index 74f0028..9321301 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -1003,70 +1003,109 @@ class SQLForeignKey {
 
 }
 
-class Type {
+class SQLUniqueConstraint {
   static $_TSPEC;
 
   /**
    * @var string
    */
-  public $name = null;
+  public $table_db = null;
   /**
    * @var string
    */
-  public $type1 = null;
+  public $table_name = null;
   /**
    * @var string
    */
-  public $type2 = null;
+  public $column_name = null;
   /**
-   * @var \metastore\FieldSchema[]
+   * @var int
    */
-  public $fields = null;
+  public $key_seq = null;
+  /**
+   * @var string
+   */
+  public $uk_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' => 'name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'type1',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'type2',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'fields',
-          'type' => TType::LST,
-          'etype' => TType::STRUCT,
-          'elem' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\FieldSchema',
-            ),
+          'var' => 'key_seq',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'uk_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['name'])) {
-        $this->name = $vals['name'];
+      if (isset($vals['table_db'])) {
+        $this->table_db = $vals['table_db'];
       }
-      if (isset($vals['type1'])) {
-        $this->type1 = $vals['type1'];
+      if (isset($vals['table_name'])) {
+        $this->table_name = $vals['table_name'];
       }
-      if (isset($vals['type2'])) {
-        $this->type2 = $vals['type2'];
+      if (isset($vals['column_name'])) {
+        $this->column_name = $vals['column_name'];
       }
-      if (isset($vals['fields'])) {
-        $this->fields = $vals['fields'];
+      if (isset($vals['key_seq'])) {
+        $this->key_seq = $vals['key_seq'];
+      }
+      if (isset($vals['uk_name'])) {
+        $this->uk_name = $vals['uk_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 'Type';
+    return 'SQLUniqueConstraint';
   }
 
   public function read($input)
@@ -1086,39 +1125,56 @@ class Type {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->type1);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->type2);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::LST) {
-            $this->fields = array();
-            $_size0 = 0;
-            $_etype3 = 0;
-            $xfer += $input->readListBegin($_etype3, $_size0);
-            for ($_i4 = 0; $_i4 < $_size0; ++$_i4)
-            {
-              $elem5 = null;
-              $elem5 = new \metastore\FieldSchema();
-              $xfer += $elem5->read($input);
-              $this->fields []= $elem5;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->key_seq);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->uk_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);
           }
@@ -1135,37 +1191,45 @@ class Type {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('Type');
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
-      $xfer += $output->writeString($this->name);
+    $xfer += $output->writeStructBegin('SQLUniqueConstraint');
+    if ($this->table_db !== null) {
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->type1 !== null) {
-      $xfer += $output->writeFieldBegin('type1', TType::STRING, 2);
-      $xfer += $output->writeString($this->type1);
+    if ($this->table_name !== null) {
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->type2 !== null) {
-      $xfer += $output->writeFieldBegin('type2', TType::STRING, 3);
-      $xfer += $output->writeString($this->type2);
+    if ($this->column_name !== null) {
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->fields !== null) {
-      if (!is_array($this->fields)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('fields', TType::LST, 4);
-      {
-        $output->writeListBegin(TType::STRUCT, count($this->fields));
-        {
-          foreach ($this->fields as $iter6)
-          {
-            $xfer += $iter6->write($output);
-          }
-        }
-        $output->writeListEnd();
-      }
+    if ($this->key_seq !== null) {
+      $xfer += $output->writeFieldBegin('key_seq', TType::I32, 4);
+      $xfer += $output->writeI32($this->key_seq);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->uk_name !== null) {
+      $xfer += $output->writeFieldBegin('uk_name', TType::STRING, 5);
+      $xfer += $output->writeString($this->uk_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();
@@ -1175,80 +1239,98 @@ class Type {
 
 }
 
-class HiveObjectRef {
+class SQLNotNullConstraint {
   static $_TSPEC;
 
   /**
-   * @var int
+   * @var string
    */
-  public $objectType = null;
+  public $table_db = null;
   /**
    * @var string
    */
-  public $dbName = null;
+  public $table_name = null;
   /**
    * @var string
    */
-  public $objectName = null;
+  public $column_name = null;
   /**
-   * @var string[]
+   * @var string
    */
-  public $partValues = null;
+  public $nn_name = null;
   /**
-   * @var string
+   * @var bool
    */
-  public $columnName = null;
+  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' => 'objectType',
-          'type' => TType::I32,
+          'var' => 'table_db',
+          'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'dbName',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'objectName',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'partValues',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
+          'var' => 'nn_name',
+          'type' => TType::STRING,
           ),
         5 => array(
-          'var' => 'columnName',
-          'type' => TType::STRING,
+          'var' => 'enable_cstr',
+          'type' => TType::BOOL,
+          ),
+        6 => array(
+          'var' => 'validate_cstr',
+          'type' => TType::BOOL,
+          ),
+        7 => array(
+          'var' => 'rely_cstr',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['objectType'])) {
-        $this->objectType = $vals['objectType'];
+      if (isset($vals['table_db'])) {
+        $this->table_db = $vals['table_db'];
       }
-      if (isset($vals['dbName'])) {
-        $this->dbName = $vals['dbName'];
+      if (isset($vals['table_name'])) {
+        $this->table_name = $vals['table_name'];
       }
-      if (isset($vals['objectName'])) {
-        $this->objectName = $vals['objectName'];
+      if (isset($vals['column_name'])) {
+        $this->column_name = $vals['column_name'];
       }
-      if (isset($vals['partValues'])) {
-        $this->partValues = $vals['partValues'];
+      if (isset($vals['nn_name'])) {
+        $this->nn_name = $vals['nn_name'];
       }
-      if (isset($vals['columnName'])) {
-        $this->columnName = $vals['columnName'];
+      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 'HiveObjectRef';
+    return 'SQLNotNullConstraint';
   }
 
   public function read($input)
@@ -1267,46 +1349,50 @@ class HiveObjectRef {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->objectType);
+          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->dbName);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->objectName);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::LST) {
-            $this->partValues = array();
-            $_size7 = 0;
-            $_etype10 = 0;
-            $xfer += $input->readListBegin($_etype10, $_size7);
-            for ($_i11 = 0; $_i11 < $_size7; ++$_i11)
-            {
-              $elem12 = null;
-              $xfer += $input->readString($elem12);
-              $this->partValues []= $elem12;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->nn_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->columnName);
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->enable_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->validate_cstr);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rely_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1323,42 +1409,40 @@ class HiveObjectRef {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('HiveObjectRef');
-    if ($this->objectType !== null) {
-      $xfer += $output->writeFieldBegin('objectType', TType::I32, 1);
-      $xfer += $output->writeI32($this->objectType);
+    $xfer += $output->writeStructBegin('SQLNotNullConstraint');
+    if ($this->table_db !== null) {
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
-      $xfer += $output->writeString($this->dbName);
+    if ($this->table_name !== null) {
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->objectName !== null) {
-      $xfer += $output->writeFieldBegin('objectName', TType::STRING, 3);
-      $xfer += $output->writeString($this->objectName);
+    if ($this->column_name !== null) {
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->partValues !== null) {
-      if (!is_array($this->partValues)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('partValues', TType::LST, 4);
-      {
-        $output->writeListBegin(TType::STRING, count($this->partValues));
-        {
-          foreach ($this->partValues as $iter13)
-          {
-            $xfer += $output->writeString($iter13);
-          }
-        }
-        $output->writeListEnd();
-      }
+    if ($this->nn_name !== null) {
+      $xfer += $output->writeFieldBegin('nn_name', TType::STRING, 4);
+      $xfer += $output->writeString($this->nn_name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->columnName !== null) {
-      $xfer += $output->writeFieldBegin('columnName', TType::STRING, 5);
-      $xfer += $output->writeString($this->columnName);
+    if ($this->enable_cstr !== null) {
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 5);
+      $xfer += $output->writeBool($this->enable_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validate_cstr !== null) {
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 6);
+      $xfer += $output->writeBool($this->validate_cstr);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rely_cstr !== null) {
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1368,76 +1452,70 @@ class HiveObjectRef {
 
 }
 
-class PrivilegeGrantInfo {
+class Type {
   static $_TSPEC;
 
   /**
    * @var string
    */
-  public $privilege = null;
-  /**
-   * @var int
-   */
-  public $createTime = null;
+  public $name = null;
   /**
    * @var string
    */
-  public $grantor = null;
+  public $type1 = null;
   /**
-   * @var int
+   * @var string
    */
-  public $grantorType = null;
+  public $type2 = null;
   /**
-   * @var bool
+   * @var \metastore\FieldSchema[]
    */
-  public $grantOption = null;
+  public $fields = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'privilege',
+          'var' => 'name',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'createTime',
-          'type' => TType::I32,
+          'var' => 'type1',
+          'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'grantor',
+          'var' => 'type2',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'grantorType',
-          'type' => TType::I32,
-          ),
-        5 => array(
-          'var' => 'grantOption',
-          'type' => TType::BOOL,
+          'var' => 'fields',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\FieldSchema',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['privilege'])) {
-        $this->privilege = $vals['privilege'];
-      }
-      if (isset($vals['createTime'])) {
-        $this->createTime = $vals['createTime'];
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
       }
-      if (isset($vals['grantor'])) {
-        $this->grantor = $vals['grantor'];
+      if (isset($vals['type1'])) {
+        $this->type1 = $vals['type1'];
       }
-      if (isset($vals['grantorType'])) {
-        $this->grantorType = $vals['grantorType'];
+      if (isset($vals['type2'])) {
+        $this->type2 = $vals['type2'];
       }
-      if (isset($vals['grantOption'])) {
-        $this->grantOption = $vals['grantOption'];
+      if (isset($vals['fields'])) {
+        $this->fields = $vals['fields'];
       }
     }
   }
 
   public function getName() {
-    return 'PrivilegeGrantInfo';
+    return 'Type';
   }
 
   public function read($input)
@@ -1457,35 +1535,39 @@ class PrivilegeGrantInfo {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->privilege);
+            $xfer += $input->readString($this->name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->createTime);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->type1);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->grantor);
+            $xfer += $input->readString($this->type2);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->grantorType);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 5:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->grantOption);
+          if ($ftype == TType::LST) {
+            $this->fields = array();
+            $_size0 = 0;
+            $_etype3 = 0;
+            $xfer += $input->readListBegin($_etype3, $_size0);
+            for ($_i4 = 0; $_i4 < $_size0; ++$_i4)
+            {
+              $elem5 = null;
+              $elem5 = new \metastore\FieldSchema();
+              $xfer += $elem5->read($input);
+              $this->fields []= $elem5;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1502,30 +1584,37 @@ class PrivilegeGrantInfo {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrivilegeGrantInfo');
-    if ($this->privilege !== null) {
-      $xfer += $output->writeFieldBegin('privilege', TType::STRING, 1);
-      $xfer += $output->writeString($this->privilege);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
-      $xfer += $output->writeI32($this->createTime);
+    $xfer += $output->writeStructBegin('Type');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantor !== null) {
-      $xfer += $output->writeFieldBegin('grantor', TType::STRING, 3);
-      $xfer += $output->writeString($this->grantor);
+    if ($this->type1 !== null) {
+      $xfer += $output->writeFieldBegin('type1', TType::STRING, 2);
+      $xfer += $output->writeString($this->type1);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantorType !== null) {
-      $xfer += $output->writeFieldBegin('grantorType', TType::I32, 4);
-      $xfer += $output->writeI32($this->grantorType);
+    if ($this->type2 !== null) {
+      $xfer += $output->writeFieldBegin('type2', TType::STRING, 3);
+      $xfer += $output->writeString($this->type2);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantOption !== null) {
-      $xfer += $output->writeFieldBegin('grantOption', TType::BOOL, 5);
-      $xfer += $output->writeBool($this->grantOption);
+    if ($this->fields !== null) {
+      if (!is_array($this->fields)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fields', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->fields));
+        {
+          foreach ($this->fields as $iter6)
+          {
+            $xfer += $iter6->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1535,67 +1624,80 @@ class PrivilegeGrantInfo {
 
 }
 
-class HiveObjectPrivilege {
+class HiveObjectRef {
   static $_TSPEC;
 
   /**
-   * @var \metastore\HiveObjectRef
+   * @var int
    */
-  public $hiveObject = null;
+  public $objectType = null;
   /**
    * @var string
    */
-  public $principalName = null;
+  public $dbName = null;
   /**
-   * @var int
+   * @var string
    */
-  public $principalType = null;
+  public $objectName = null;
   /**
-   * @var \metastore\PrivilegeGrantInfo
+   * @var string[]
    */
-  public $grantInfo = null;
+  public $partValues = null;
+  /**
+   * @var string
+   */
+  public $columnName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'hiveObject',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\HiveObjectRef',
+          'var' => 'objectType',
+          'type' => TType::I32,
           ),
         2 => array(
-          'var' => 'principalName',
+          'var' => 'dbName',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'principalType',
-          'type' => TType::I32,
+          'var' => 'objectName',
+          'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'grantInfo',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\PrivilegeGrantInfo',
+          'var' => 'partValues',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        5 => array(
+          'var' => 'columnName',
+          'type' => TType::STRING,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['hiveObject'])) {
-        $this->hiveObject = $vals['hiveObject'];
+      if (isset($vals['objectType'])) {
+        $this->objectType = $vals['objectType'];
       }
-      if (isset($vals['principalName'])) {
-        $this->principalName = $vals['principalName'];
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
       }
-      if (isset($vals['principalType'])) {
-        $this->principalType = $vals['principalType'];
+      if (isset($vals['objectName'])) {
+        $this->objectName = $vals['objectName'];
       }
-      if (isset($vals['grantInfo'])) {
-        $this->grantInfo = $vals['grantInfo'];
+      if (isset($vals['partValues'])) {
+        $this->partValues = $vals['partValues'];
+      }
+      if (isset($vals['columnName'])) {
+        $this->columnName = $vals['columnName'];
       }
     }
   }
 
   public function getName() {
-    return 'HiveObjectPrivilege';
+    return 'HiveObjectRef';
   }
 
   public function read($input)
@@ -1614,31 +1716,46 @@ class HiveObjectPrivilege {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->hiveObject = new \metastore\HiveObjectRef();
-            $xfer += $this->hiveObject->read($input);
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->objectType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->principalName);
+            $xfer += $input->readString($this->dbName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->principalType);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->objectName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
-          if ($ftype == TType::STRUCT) {
-            $this->grantInfo = new \metastore\PrivilegeGrantInfo();
-            $xfer += $this->grantInfo->read($input);
+          if ($ftype == TType::LST) {
+            $this->partValues = array();
+            $_size7 = 0;
+            $_etype10 = 0;
+            $xfer += $input->readListBegin($_etype10, $_size7);
+            for ($_i11 = 0; $_i11 < $_size7; ++$_i11)
+            {
+              $elem12 = null;
+              $xfer += $input->readString($elem12);
+              $this->partValues []= $elem12;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->columnName);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1655,31 +1772,42 @@ class HiveObjectPrivilege {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('HiveObjectPrivilege');
-    if ($this->hiveObject !== null) {
-      if (!is_object($this->hiveObject)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1);
-      $xfer += $this->hiveObject->write($output);
+    $xfer += $output->writeStructBegin('HiveObjectRef');
+    if ($this->objectType !== null) {
+      $xfer += $output->writeFieldBegin('objectType', TType::I32, 1);
+      $xfer += $output->writeI32($this->objectType);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->principalName !== null) {
-      $xfer += $output->writeFieldBegin('principalName', TType::STRING, 2);
-      $xfer += $output->writeString($this->principalName);
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
+      $xfer += $output->writeString($this->dbName);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->principalType !== null) {
-      $xfer += $output->writeFieldBegin('principalType', TType::I32, 3);
-      $xfer += $output->writeI32($this->principalType);
+    if ($this->objectName !== null) {
+      $xfer += $output->writeFieldBegin('objectName', TType::STRING, 3);
+      $xfer += $output->writeString($this->objectName);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->grantInfo !== null) {
-      if (!is_object($this->grantInfo)) {
+    if ($this->partValues !== null) {
+      if (!is_array($this->partValues)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('grantInfo', TType::STRUCT, 4);
-      $xfer += $this->grantInfo->write($output);
+      $xfer += $output->writeFieldBegin('partValues', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRING, count($this->partValues));
+        {
+          foreach ($this->partValues as $iter13)
+          {
+            $xfer += $output->writeString($iter13);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->columnName !== null) {
+      $xfer += $output->writeFieldBegin('columnName', TType::STRING, 5);
+      $xfer += $output->writeString($this->columnName);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -1689,37 +1817,76 @@ class HiveObjectPrivilege {
 
 }
 
-class PrivilegeBag {
+class PrivilegeGrantInfo {
   static $_TSPEC;
 
   /**
-   * @var \metastore\HiveObjectPrivilege[]
+   * @var string
    */
-  public $privileges = null;
+  public $privilege = null;
+  /**
+   * @var int
+   */
+  public $createTime = null;
+  /**
+   * @var string
+   */
+  public $grantor = null;
+  /**
+   * @var int
+   */
+  public $grantorType = null;
+  /**
+   * @var bool
+   */
+  public $grantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'privileges',
-          'type' => TType::LST,
-          'etype' => TType::STRUCT,
-          'elem' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\HiveObjectPrivilege',
-            ),
+          'var' => 'privilege',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'createTime',
+          'type' => TType::I32,
+          ),
+        3 => array(
+          'var' => 'grantor',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'grantorType',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'grantOption',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['privileges'])) {
-        $this->privileges = $vals['privileges'];
+      if (isset($vals['privilege'])) {
+        $this->privilege = $vals['privilege'];
+      }
+      if (isset($vals['createTime'])) {
+        $this->createTime = $vals['createTime'];
+      }
+      if (isset($vals['grantor'])) {
+        $this->grantor = $vals['grantor'];
+      }
+      if (isset($vals['grantorType'])) {
+        $this->grantorType = $vals['grantorType'];
+      }
+      if (isset($vals['grantOption'])) {
+        $this->grantOption = $vals['grantOption'];
       }
     }
   }
 
   public function getName() {
-    return 'PrivilegeBag';
+    return 'PrivilegeGrantInfo';
   }
 
   public function read($input)
@@ -1738,19 +1905,36 @@ class PrivilegeBag {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::LST) {
-            $this->privileges = array();
-            $_size14 = 0;
-            $_etype17 = 0;
-            $xfer += $input->readListBegin($_etype17, $_size14);
-            for ($_i18 = 0; $_i18 < $_size14; ++$_i18)
-            {
-              $elem19 = null;
-              $elem19 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem19->read($input);
-              $this->privileges []= $elem19;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->privilege);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->createTime);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->grantor);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->grantorType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->grantOption);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -1767,118 +1951,100 @@ class PrivilegeBag {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrivilegeBag');
-    if ($this->privileges !== null) {
-      if (!is_array($this->privileges)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('privileges', TType::LST, 1);
-      {
-        $output->writeListBegin(TType::STRUCT, count($this->privileges));
-        {
-          foreach ($this->privileges as $iter20)
-          {
-            $xfer += $iter20->write($output);
-          }
-        }
-        $output->writeListEnd();
-      }
+    $xfer += $output->writeStructBegin('PrivilegeGrantInfo');
+    if ($this->privilege !== null) {
+      $xfer += $output->writeFieldBegin('privilege', TType::STRING, 1);
+      $xfer += $output->writeString($this->privilege);
       $xfer += $output->writeFieldEnd();
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class PrincipalPrivilegeSet {
-  static $_TSPEC;
-
-  /**
-   * @var array
-   */
-  public $userPrivileges = null;
+    if ($this->createTime !== null) {
+      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
+      $xfer += $output->writeI32($this->createTime);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantor !== null) {
+      $xfer += $output->writeFieldBegin('grantor', TType::STRING, 3);
+      $xfer += $output->writeString($this->grantor);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantorType !== null) {
+      $xfer += $output->writeFieldBegin('grantorType', TType::I32, 4);
+      $xfer += $output->writeI32($this->grantorType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantOption !== null) {
+      $xfer += $output->writeFieldBegin('grantOption', TType::BOOL, 5);
+      $xfer += $output->writeBool($this->grantOption);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class HiveObjectPrivilege {
+  static $_TSPEC;
+
   /**
-   * @var array
+   * @var \metastore\HiveObjectRef
    */
-  public $groupPrivileges = null;
+  public $hiveObject = null;
   /**
-   * @var array
+   * @var string
    */
-  public $rolePrivileges = null;
+  public $principalName = null;
+  /**
+   * @var int
+   */
+  public $principalType = null;
+  /**
+   * @var \metastore\PrivilegeGrantInfo
+   */
+  public $grantInfo = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'userPrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+          'var' => 'hiveObject',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\HiveObjectRef',
           ),
         2 => array(
-          'var' => 'groupPrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+          'var' => 'principalName',
+          'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'rolePrivileges',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::LST,
-          'key' => array(
-            'type' => TType::STRING,
+          'var' => 'principalType',
+          'type' => TType::I32,
           ),
-          'val' => array(
-            'type' => TType::LST,
-            'etype' => TType::STRUCT,
-            'elem' => array(
-              'type' => TType::STRUCT,
-              'class' => '\metastore\PrivilegeGrantInfo',
-              ),
-            ),
+        4 => array(
+          'var' => 'grantInfo',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PrivilegeGrantInfo',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['userPrivileges'])) {
-        $this->userPrivileges = $vals['userPrivileges'];
+      if (isset($vals['hiveObject'])) {
+        $this->hiveObject = $vals['hiveObject'];
       }
-      if (isset($vals['groupPrivileges'])) {
-        $this->groupPrivileges = $vals['groupPrivileges'];
+      if (isset($vals['principalName'])) {
+        $this->principalName = $vals['principalName'];
       }
-      if (isset($vals['rolePrivileges'])) {
-        $this->rolePrivileges = $vals['rolePrivileges'];
+      if (isset($vals['principalType'])) {
+        $this->principalType = $vals['principalType'];
+      }
+      if (isset($vals['grantInfo'])) {
+        $this->grantInfo = $vals['grantInfo'];
       }
     }
   }
 
   public function getName() {
-    return 'PrincipalPrivilegeSet';
+    return 'HiveObjectPrivilege';
   }
 
   public function read($input)
@@ -1897,94 +2063,31 @@ class PrincipalPrivilegeSet {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::MAP) {
-            $this->userPrivileges = array();
-            $_size21 = 0;
-            $_ktype22 = 0;
-            $_vtype23 = 0;
-            $xfer += $input->readMapBegin($_ktype22, $_vtype23, $_size21);
-            for ($_i25 = 0; $_i25 < $_size21; ++$_i25)
-            {
-              $key26 = '';
-              $val27 = array();
-              $xfer += $input->readString($key26);
-              $val27 = array();
-              $_size28 = 0;
-              $_etype31 = 0;
-              $xfer += $input->readListBegin($_etype31, $_size28);
-              for ($_i32 = 0; $_i32 < $_size28; ++$_i32)
-              {
-                $elem33 = null;
-                $elem33 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem33->read($input);
-                $val27 []= $elem33;
-              }
-              $xfer += $input->readListEnd();
-              $this->userPrivileges[$key26] = $val27;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->hiveObject = new \metastore\HiveObjectRef();
+            $xfer += $this->hiveObject->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::MAP) {
-            $this->groupPrivileges = array();
-            $_size34 = 0;
-            $_ktype35 = 0;
-            $_vtype36 = 0;
-            $xfer += $input->readMapBegin($_ktype35, $_vtype36, $_size34);
-            for ($_i38 = 0; $_i38 < $_size34; ++$_i38)
-            {
-              $key39 = '';
-              $val40 = array();
-              $xfer += $input->readString($key39);
-              $val40 = array();
-              $_size41 = 0;
-              $_etype44 = 0;
-              $xfer += $input->readListBegin($_etype44, $_size41);
-              for ($_i45 = 0; $_i45 < $_size41; ++$_i45)
-              {
-                $elem46 = null;
-                $elem46 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem46->read($input);
-                $val40 []= $elem46;
-              }
-              $xfer += $input->readListEnd();
-              $this->groupPrivileges[$key39] = $val40;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->principalName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::MAP) {
-            $this->rolePrivileges = array();
-            $_size47 = 0;
-            $_ktype48 = 0;
-            $_vtype49 = 0;
-            $xfer += $input->readMapBegin($_ktype48, $_vtype49, $_size47);
-            for ($_i51 = 0; $_i51 < $_size47; ++$_i51)
-            {
-              $key52 = '';
-              $val53 = array();
-              $xfer += $input->readString($key52);
-              $val53 = array();
-              $_size54 = 0;
-              $_etype57 = 0;
-              $xfer += $input->readListBegin($_etype57, $_size54);
-              for ($_i58 = 0; $_i58 < $_size54; ++$_i58)
-              {
-                $elem59 = null;
-                $elem59 = new \metastore\PrivilegeGrantInfo();
-                $xfer += $elem59->read($input);
-                $val53 []= $elem59;
-              }
-              $xfer += $input->readListEnd();
-              $this->rolePrivileges[$key52] = $val53;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->principalType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRUCT) {
+            $this->grantInfo = new \metastore\PrivilegeGrantInfo();
+            $xfer += $this->grantInfo->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -2001,86 +2104,31 @@ class PrincipalPrivilegeSet {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('PrincipalPrivilegeSet');
-    if ($this->userPrivileges !== null) {
-      if (!is_array($this->userPrivileges)) {
+    $xfer += $output->writeStructBegin('HiveObjectPrivilege');
+    if ($this->hiveObject !== null) {
+      if (!is_object($this->hiveObject)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('userPrivileges', TType::MAP, 1);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->userPrivileges));
-        {
-          foreach ($this->userPrivileges as $kiter60 => $viter61)
-          {
-            $xfer += $output->writeString($kiter60);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter61));
-              {
-                foreach ($viter61 as $iter62)
-                {
-                  $xfer += $iter62->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1);
+      $xfer += $this->hiveObject->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->groupPrivileges !== null) {
-      if (!is_array($this->groupPrivileges)) {
+    if ($this->principalName !== null) {
+      $xfer += $output->writeFieldBegin('principalName', TType::STRING, 2);
+      $xfer += $output->writeString($this->principalName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->principalType !== null) {
+      $xfer += $output->writeFieldBegin('principalType', TType::I32, 3);
+      $xfer += $output->writeI32($this->principalType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantInfo !== null) {
+      if (!is_object($this->grantInfo)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('groupPrivileges', TType::MAP, 2);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->groupPrivileges));
-        {
-          foreach ($this->groupPrivileges as $kiter63 => $viter64)
-          {
-            $xfer += $output->writeString($kiter63);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter64));
-              {
-                foreach ($viter64 as $iter65)
-                {
-                  $xfer += $iter65->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->rolePrivileges !== null) {
-      if (!is_array($this->rolePrivileges)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('rolePrivileges', TType::MAP, 3);
-      {
-        $output->writeMapBegin(TType::STRING, TType::LST, count($this->rolePrivileges));
-        {
-          foreach ($this->rolePrivileges as $kiter66 => $viter67)
-          {
-            $xfer += $output->writeString($kiter66);
-            {
-              $output->writeListBegin(TType::STRUCT, count($viter67));
-              {
-                foreach ($viter67 as $iter68)
-                {
-                  $xfer += $iter68->write($output);
-                }
-              }
-              $output->writeListEnd();
-            }
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('grantInfo', TType::STRUCT, 4);
+      $xfer += $this->grantInfo->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -2090,55 +2138,37 @@ class PrincipalPrivilegeSet {
 
 }
 
-class GrantRevokePrivilegeRequest {
+class PrivilegeBag {
   static $_TSPEC;
 
   /**
-   * @var int
-   */
-  public $requestType = null;
-  /**
-   * @var \metastore\PrivilegeBag
+   * @var \metastore\HiveObjectPrivilege[]
    */
   public $privileges = null;
-  /**
-   * @var bool
-   */
-  public $revokeGrantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'requestType',
-          'type' => TType::I32,
-          ),
-        2 => array(
           'var' => 'privileges',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\PrivilegeBag',
-          ),
-        3 => array(
-          'var' => 'revokeGrantOption',
-          'type' => TType::BOOL,
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\HiveObjectPrivilege',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['requestType'])) {
-        $this->requestType = $vals['requestType'];
-      }
       if (isset($vals['privileges'])) {
         $this->privileges = $vals['privileges'];
       }
-      if (isset($vals['revokeGrantOption'])) {
-        $this->revokeGrantOption = $vals['revokeGrantOption'];
-      }
     }
   }
 
   public function getName() {
-    return 'GrantRevokePrivilegeRequest';
+    return 'PrivilegeBag';
   }
 
   public function read($input)
@@ -2157,23 +2187,19 @@ class GrantRevokePrivilegeRequest {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->requestType);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::STRUCT) {
-            $this->privileges = new \metastore\PrivilegeBag();
-            $xfer += $this->privileges->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->revokeGrantOption);
+          if ($ftype == TType::LST) {
+            $this->privileges = array();
+            $_size14 = 0;
+            $_etype17 = 0;
+            $xfer += $input->readListBegin($_etype17, $_size14);
+            for ($_i18 = 0; $_i18 < $_size14; ++$_i18)
+            {
+              $elem19 = null;
+              $elem19 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem19->read($input);
+              $this->privileges []= $elem19;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -2190,23 +2216,22 @@ class GrantRevokePrivilegeRequest {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('GrantRevokePrivilegeRequest');
-    if ($this->requestType !== null) {
-      $xfer += $output->writeFieldBegin('requestType', TType::I32, 1);
-      $xfer += $output->writeI32($this->requestType);
-      $xfer += $output->writeFieldEnd();
-    }
+    $xfer += $output->writeStructBegin('PrivilegeBag');
     if ($this->privileges !== null) {
-      if (!is_object($this->privileges)) {
+      if (!is_array($this->privileges)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 2);
-      $xfer += $this->privileges->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->revokeGrantOption !== null) {
-      $xfer += $output->writeFieldBegin('revokeGrantOption', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->revokeGrantOption);
+      $xfer += $output->writeFieldBegin('privileges', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->privileges));
+        {
+          foreach ($this->privileges as $iter20)
+          {
+            $xfer += $iter20->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -2216,32 +2241,93 @@ class GrantRevokePrivilegeRequest {
 
 }
 
-class GrantRevokePrivilegeResponse {
+class PrincipalPrivilegeSet {
   static $_TSPEC;
 
   /**
-   * @var bool
+   * @var array
    */
-  public $success = null;
+  public $userPrivileges = null;
+  /**
+   * @var array
+   */
+  public $groupPrivileges = null;
+  /**
+   * @var array
+   */
+  public $rolePrivileges = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'success',
-          'type' => TType::BOOL,
+          'var' => 'userPrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
+          ),
+        2 => array(
+          'var' => 'groupPrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
+          ),
+        3 => array(
+          'var' => 'rolePrivileges',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::LST,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::LST,
+            'etype' => TType::STRUCT,
+            'elem' => array(
+              'type' => TType::STRUCT,
+              'class' => '\metastore\PrivilegeGrantInfo',
+              ),
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
+      if (isset($vals['userPrivileges'])) {
+        $this->userPrivileges = $vals['userPrivileges'];
+      }
+      if (isset($vals['groupPrivileges'])) {
+        $this->groupPrivileges = $vals['groupPrivileges'];
+      }
+      if (isset($vals['rolePrivileges'])) {
+        $this->rolePrivileges = $vals['rolePrivileges'];
       }
     }
   }
 
   public function getName() {
-    return 'GrantRevokePrivilegeResponse';
+    return 'PrincipalPrivilegeSet';
   }
 
   public function read($input)
@@ -2260,28 +2346,190 @@ class GrantRevokePrivilegeResponse {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->success);
+          if ($ftype == TType::MAP) {
+            $this->userPrivileges = array();
+            $_size21 = 0;
+            $_ktype22 = 0;
+            $_vtype23 = 0;
+            $xfer += $input->readMapBegin($_ktype22, $_vtype23, $_size21);
+            for ($_i25 = 0; $_i25 < $_size21; ++$_i25)
+            {
+              $key26 = '';
+              $val27 = array();
+              $xfer += $input->readString($key26);
+              $val27 = array();
+              $_size28 = 0;
+              $_etype31 = 0;
+              $xfer += $input->readListBegin($_etype31, $_size28);
+              for ($_i32 = 0; $_i32 < $_size28; ++$_i32)
+              {
+                $elem33 = null;
+                $elem33 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem33->read($input);
+                $val27 []= $elem33;
+              }
+              $xfer += $input->readListEnd();
+              $this->userPrivileges[$key26] = $val27;
+            }
+            $xfer += $input->readMapEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
+        case 2:
+          if ($ftype == TType::MAP) {
+            $this->groupPrivileges = array();
+            $_size34 = 0;
+            $_ktype35 = 0;
+            $_vtype36 = 0;
+            $xfer += $input->readMapBegin($_ktype35, $_vtype36, $_size34);
+            for ($_i38 = 0; $_i38 < $_size34; ++$_i38)
+            {
+              $key39 = '';
+              $val40 = array();
+              $xfer += $input->readString($key39);
+              $val40 = array();
+              $_size41 = 0;
+              $_etype44 = 0;
+              $xfer += $input->readListBegin($_etype44, $_size41);
+              for ($_i45 = 0; $_i45 < $_size41; ++$_i45)
+              {
+                $elem46 = null;
+                $elem46 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem46->read($input);
+                $val40 []= $elem46;
+              }
+              $xfer += $input->readListEnd();
+              $this->groupPrivileges[$key39] = $val40;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::MAP) {
+            $this->rolePrivileges = array();
+            $_size47 = 0;
+            $_ktype48 = 0;
+            $_vtype49 = 0;
+            $xfer += $input->readMapBegin($_ktype48, $_vtype49, $_size47);
+            for ($_i51 = 0; $_i51 < $_size47; ++$_i51)
+            {
+              $key52 = '';
+              $val53 = array();
+              $xfer += $input->readString($key52);
+              $val53 = array();
+              $_size54 = 0;
+              $_etype57 = 0;
+              $xfer += $input->readListBegin($_etype57, $_size54);
+              for ($_i58 = 0; $_i58 < $_size54; ++$_i58)
+              {
+                $elem59 = null;
+                $elem59 = new \metastore\PrivilegeGrantInfo();
+                $xfer += $elem59->read($input);
+                $val53 []= $elem59;
+              }
+              $xfer += $input->readListEnd();
+              $this->rolePrivileges[$key52] = $val53;
+            }
+            $xfer += $input->readMapEnd();
+          } 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('GrantRevokePrivilegeResponse');
-    if ($this->success !== null) {
-      $xfer += $output->writeFieldBegin('success', TType::BOOL, 1);
-      $xfer += $output->writeBool($this->success);
+    $xfer += $output->writeStructBegin('PrincipalPrivilegeSet');
+    if ($this->userPrivileges !== null) {
+      if (!is_array($this->userPrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('userPrivileges', TType::MAP, 1);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->userPrivileges));
+        {
+          foreach ($this->userPrivileges as $kiter60 => $viter61)
+          {
+            $xfer += $output->writeString($kiter60);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter61));
+              {
+                foreach ($viter61 as $iter62)
+                {
+                  $xfer += $iter62->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->groupPrivileges !== null) {
+      if (!is_array($this->groupPrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('groupPrivileges', TType::MAP, 2);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->groupPrivileges));
+        {
+          foreach ($this->groupPrivileges as $kiter63 => $viter64)
+          {
+            $xfer += $output->writeString($kiter63);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter64));
+              {
+                foreach ($viter64 as $iter65)
+                {
+                  $xfer += $iter65->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->rolePrivileges !== null) {
+      if (!is_array($this->rolePrivileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('rolePrivileges', TType::MAP, 3);
+      {
+        $output->writeMapBegin(TType::STRING, TType::LST, count($this->rolePrivileges));
+        {
+          foreach ($this->rolePrivileges as $kiter66 => $viter67)
+          {
+            $xfer += $output->writeString($kiter66);
+            {
+              $output->writeListBegin(TType::STRUCT, count($viter67));
+              {
+                foreach ($viter67 as $iter68)
+                {
+                  $xfer += $iter68->write($output);
+                }
+              }
+              $output->writeListEnd();
+            }
+          }
+        }
+        $output->writeMapEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -2291,54 +2539,55 @@ class GrantRevokePrivilegeResponse {
 
 }
 
-class Role {
+class GrantRevokePrivilegeRequest {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var int
    */
-  public $roleName = null;
+  public $requestType = null;
   /**
-   * @var int
+   * @var \metastore\PrivilegeBag
    */
-  public $createTime = null;
+  public $privileges = null;
   /**
-   * @var string
+   * @var bool
    */
-  public $ownerName = null;
+  public $revokeGrantOption = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'roleName',
-          'type' => TType::STRING,
+          'var' => 'requestType',
+          'type' => TType::I32,
           ),
         2 => array(
-          'var' => 'createTime',
-          'type' => TType::I32,
+          'var' => 'privileges',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PrivilegeBag',
           ),
         3 => array(
-          'var' => 'ownerName',
-          'type' => TType::STRING,
+          'var' => 'revokeGrantOption',
+          'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['roleName'])) {
-        $this->roleName = $vals['roleName'];
+      if (isset($vals['requestType'])) {
+        $this->requestType = $vals['requestType'];
       }
-      if (isset($vals['createTime'])) {
-        $this->createTime = $vals['createTime'];
+      if (isset($vals['privileges'])) {
+        $this->privileges = $vals['privileges'];
       }
-      if (isset($vals['ownerName'])) {
-        $this->ownerName = $vals['ownerName'];
+      if (isset($vals['revokeGrantOption'])) {
+        $this->revokeGrantOption = $vals['revokeGrantOption'];
       }
     }
   }
 
   public function getName() {
-    return 'Role';
+    return 'GrantRevokePrivilegeRequest';
   }
 
   public function read($input)
@@ -2357,22 +2606,23 @@ class Role {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->roleName);
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->requestType);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->createTime);
+          if ($ftype == TType::STRUCT) {
+            $this->privileges = new \metastore\PrivilegeBag();
+            $xfer += $this->privileges->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->ownerName);
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->revokeGrantOption);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -2389,20 +2639,23 @@ class Role {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('Role');
-    if ($this->roleName !== null) {
-      $xfer += $output->writeFieldBegin('roleName', TType::STRING, 1);
-      $xfer += $output->writeString($this->roleName);
+    $xfer += $output->writeStructBegin('GrantRevokePrivilegeRequest');
+    if ($this->requestType !== null) {
+      $xfer += $output->writeFieldBegin('requestType', TType::I32, 1);
+      $xfer += $output->writeI32($this->requestType);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
-      $xfer += $output->writeI32($this->createTime);
+    if ($this->privileges !== null) {
+      if (!is_object($this->privileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 2);
+      $xfer += $this->privileges->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->ownerName !== null) {
-      $xfer += $output->writeFieldBegin('ownerName', TType::STRING, 3);
-      $xfer += $output->writeString($this->ownerName);
+    if ($this->revokeGrantOption !== null) {
+      $xfer += $output->writeFieldBegin('revokeGrantOption', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->revokeGrantOption);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -2412,34 +2665,230 @@ class Role {
 
 }
 
-class RolePrincipalGrant {
+class GrantRevokePrivilegeResponse {
   static $_TSPEC;
 
   /**
-   * @var string
-   */
-  public $roleName = null;
-  /**
-   * @var string
-   */
-  public $principalName = null;
-  /**
-   * @var int
-   */
-  public $principalType = null;
-  /**
    * @var bool
    */
-  public $grantOption = null;
-  /**
-   * @var int
-   */
-  public $grantTime = null;
-  /**
-   * @var string
-   */
-  public $grantorName = null;
-  /**
+  public $success = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'success',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GrantRevokePrivilegeResponse';
+  }
+
+  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::BOOL) {
+            $xfer += $input->readBool($this->success);
+          } 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('GrantRevokePrivilegeResponse');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class Role {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $roleName = null;
+  /**
+   * @var int
+   */
+  public $createTime = null;
+  /**
+   * @var string
+   */
+  public $ownerName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'roleName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'createTime',
+          'type' => TType::I32,
+          ),
+        3 => array(
+          'var' => 'ownerName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['roleName'])) {
+        $this->roleName = $vals['roleName'];
+      }
+      if (isset($vals['createTime'])) {
+        $this->createTime = $vals['createTime'];
+      }
+      if (isset($vals['ownerName'])) {
+        $this->ownerName = $vals['ownerName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Role';
+  }
+
+  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->roleName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->createTime);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->ownerName);
+          } 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('Role');
+    if ($this->roleName !== null) {
+      $xfer += $output->writeFieldBegin('roleName', TType::STRING, 1);
+      $xfer += $output->writeString($this->roleName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->createTime !== null) {
+      $xfer += $output->writeFieldBegin('createTime', TType::I32, 2);
+      $xfer += $output->writeI32($this->createTime);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->ownerName !== null) {
+      $xfer += $output->writeFieldBegin('ownerName', TType::STRING, 3);
+      $xfer += $output->writeString($this->ownerName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class RolePrincipalGrant {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $roleName = null;
+  /**
+   * @var string
+   */
+  public $principalName = null;
+  /**
+   * @var int
+   */
+  public $principalType = null;
+  /**
+   * @var bool
+   */
+  public $grantOption = null;
+  /**
+   * @var int
+   */
+  public $grantTime = null;
+  /**
+   * @var string
+   */
+  public $grantorName = null;
+  /**
    * @var int
    */
   public $grantorPrincipalType = null;
@@ -2979,24 +3428,1524 @@ class GetPrincipalsInRoleResponse {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('GetPrincipalsInRoleResponse');
-    if ($this->principalGrants !== null) {
-      if (!is_array($this->principalGrants)) {
+    $xfer += $output->writeStructBegin('GetPrincipalsInRoleResponse');
+    if ($this->principalGrants !== null) {
+      if (!is_array($this->principalGrants)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('principalGrants', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->principalGrants));
+        {
+          foreach ($this->principalGrants as $iter82)
+          {
+            $xfer += $iter82->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GrantRevokeRoleRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $requestType = null;
+  /**
+   * @var string
+   */
+  public $roleName = null;
+  /**
+   * @var string
+   */
+  public $principalName = null;
+  /**
+   * @var int
+   */
+  public $principalType = null;
+  /**
+   * @var string
+   */
+  public $grantor = null;
+  /**
+   * @var int
+   */
+  public $grantorType = null;
+  /**
+   * @var bool
+   */
+  public $grantOption = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'requestType',
+          'type' => TType::I32,
+          ),
+        2 => array(
+          'var' => 'roleName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'principalName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'principalType',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'grantor',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'grantorType',
+          'type' => TType::I32,
+          ),
+        7 => array(
+          'var' => 'grantOption',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['requestType'])) {
+        $this->requestType = $vals['requestType'];
+      }
+      if (isset($vals['roleName'])) {
+        $this->roleName = $vals['roleName'];
+      }
+      if (isset($vals['principalName'])) {
+        $this->principalName = $vals['principalName'];
+      }
+      if (isset($vals['principalType'])) {
+        $this->principalType = $vals['principalType'];
+      }
+      if (isset($vals['grantor'])) {
+        $this->grantor = $vals['grantor'];
+      }
+      if (isset($vals['grantorType'])) {
+        $this->grantorType = $vals['grantorType'];
+      }
+      if (isset($vals['grantOption'])) {
+        $this->grantOption = $vals['grantOption'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GrantRevokeRoleRequest';
+  }
+
+  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::I32) {
+            $xfer += $input->readI32($this->requestType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->roleName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->principalName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->principalType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->grantor);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->grantorType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->grantOption);
+          } 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('GrantRevokeRoleRequest');
+    if ($this->requestType !== null) {
+      $xfer += $output->writeFieldBegin('requestType', TType::I32, 1);
+      $xfer += $output->writeI32($this->requestType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->roleName !== null) {
+      $xfer += $output->writeFieldBegin('roleName', TType::STRING, 2);
+      $xfer += $output->writeString($this->roleName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->principalName !== null) {
+      $xfer += $output->writeFieldBegin('principalName', TType::STRING, 3);
+      $xfer += $output->writeString($this->principalName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->principalType !== null) {
+      $xfer += $output->writeFieldBegin('principalType', TType::I32, 4);
+      $xfer += $output->writeI32($this->principalType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantor !== null) {
+      $xfer += $output->writeFieldBegin('grantor', TType::STRING, 5);
+      $xfer += $output->writeString($this->grantor);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantorType !== null) {
+      $xfer += $output->writeFieldBegin('grantorType', TType::I32, 6);
+      $xfer += $output->writeI32($this->grantorType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->grantOption !== null) {
+      $xfer += $output->writeFieldBegin('grantOption', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->grantOption);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GrantRevokeRoleResponse {
+  static $_TSPEC;
+
+  /**
+   * @var bool
+   */
+  public $success = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'success',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GrantRevokeRoleResponse';
+  }
+
+  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::BOOL) {
+            $xfer += $input->readBool($this->success);
+          } 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('GrantRevokeRoleResponse');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class Database {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+  /**
+   * @var string
+   */
+  public $locationUri = null;
+  /**
+   * @var array
+   */
+  public $parameters = null;
+  /**
+   * @var \metastore\PrincipalPrivilegeSet
+   */
+  public $privileges = null;
+  /**
+   * @var string
+   */
+  public $ownerName = null;
+  /**
+   * @var int
+   */
+  public $ownerType = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'description',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'locationUri',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'parameters',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::STRING,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        5 => array(
+          'var' => 'privileges',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\PrincipalPrivilegeSet',
+          ),
+        6 => array(
+          'var' => 'ownerName',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'ownerType',
+          'type' => TType::I32,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['description'])) {
+        $this->description = $vals['description'];
+      }
+      if (isset($vals['locationUri'])) {
+        $this->locationUri = $vals['locationUri'];
+      }
+      if (isset($vals['parameters'])) {
+        $this->parameters = $vals['parameters'];
+      }
+      if (isset($vals['privileges'])) {
+        $this->privileges = $vals['privileges'];
+      }
+      if (isset($vals['ownerName'])) {
+        $this->ownerName = $vals['ownerName'];
+      }
+      if (isset($vals['ownerType'])) {
+        $this->ownerType = $vals['ownerType'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Database';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->description);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->locationUri);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::MAP) {
+            $this->parameters = array();
+            $_size83 = 0;
+            $_ktype84 = 0;
+            $_vtype85 = 0;
+            $xfer += $input->readMapBegin($_ktype84, $_vtype85, $_size83);
+            for ($_i87 = 0; $_i87 < $_size83; ++$_i87)
+            {
+              $key88 = '';
+              $val89 = '';
+              $xfer += $input->readString($key88);
+              $xfer += $input->readString($val89);
+              $this->parameters[$key88] = $val89;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRUCT) {
+            $this->privileges = new \metastore\PrincipalPrivilegeSet();
+            $xfer += $this->privileges->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->ownerName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->ownerType);
+          } 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('Database');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->description !== null) {
+      $xfer += $output->writeFieldBegin('description', TType::STRING, 2);
+      $xfer += $output->writeString($this->description);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->locationUri !== null) {
+      $xfer += $output->writeFieldBegin('locationUri', TType::STRING, 3);
+      $xfer += $output->writeString($this->locationUri);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->parameters !== null) {
+      if (!is_array($this->parameters)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('parameters', TType::MAP, 4);
+      {
+        $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
+        {
+          foreach ($this->parameters as $kiter90 => $viter91)
+          {
+            $xfer += $output->writeString($kiter90);
+            $xfer += $output->writeString($viter91);
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->privileges !== null) {
+      if (!is_object($this->privileges)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 5);
+      $xfer += $this->privileges->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->ownerName !== null) {
+      $xfer += $output->writeFieldBegin('ownerName', TType::STRING, 6);
+      $xfe

<TRUNCATED>

[02/31] hive git commit: HIVE-16754: LLAP: Print hive version info on llap daemon startup (Prasanth Jayachandran reviewed by Siddharth Seth)

Posted by we...@apache.org.
HIVE-16754: LLAP: Print hive version info on llap daemon startup (Prasanth Jayachandran reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: ef6a90af4bf8c61b3350ee841802e1bb1e1d355a
Parents: 3fe65a3
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed May 24 15:34:56 2017 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed May 24 15:34:56 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ef6a90af/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index aae146e..cfca3f7 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hive.common.util.HiveVersionInfo;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.apache.logging.log4j.core.config.Configurator;
 import org.slf4j.Logger;
@@ -198,7 +199,8 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       ", workDirs=" + Arrays.toString(localDirs) +
       ", shufflePort=" + shufflePort +
       ", waitQueueSize= " + waitQueueSize +
-      ", enablePreemption= " + enablePreemption;
+      ", enablePreemption= " + enablePreemption +
+      ", versionInfo= (" + HiveVersionInfo.getBuildVersion() + ")";
     LOG.info(logMsg);
     final String currTSISO8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ").format(new Date());
     // Time based log retrieval may not fetch the above log line so logging to stderr for debugging purpose.


[10/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
index 6da40ac..3fcb048 100644
--- a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MConstraint.java
@@ -36,8 +36,12 @@ public class MConstraint
 
   // 0 - Primary Key
   // 1 - PK-FK relationship
+  // 2 - Unique Constraint
+  // 3 - Not Null Constraint
   public final static int PRIMARY_KEY_CONSTRAINT = 0;
   public final static int FOREIGN_KEY_CONSTRAINT = 1;
+  public final static int UNIQUE_CONSTRAINT = 2;
+  public final static int NOT_NULL_CONSTRAINT = 3;
 
   @SuppressWarnings("serial")
   public static class PK implements Serializable {

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index 6499ac6..53c381b 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -332,3 +332,37 @@ message ForeignKeys {
 
   repeated ForeignKey fks = 1;
 }
+
+message UniqueConstraints {
+  message UniqueConstraint {
+    message UniqueConstraintColumn {
+      required string column_name = 1;
+      required sint32 key_seq = 2;
+    }
+
+    required string uk_name = 1;
+    repeated UniqueConstraintColumn cols = 2;
+    optional bool enable_constraint = 3;
+    optional bool validate_constraint = 4;
+    optional bool rely_constraint = 5;
+  }
+
+  repeated UniqueConstraint uks = 1;
+}
+
+message NotNullConstraints {
+  message NotNullConstraint {
+    message NotNullConstraintColumn {
+      required string column_name = 1;
+    }
+
+    required string nn_name = 1;
+    repeated NotNullConstraintColumn cols = 2;
+    optional bool enable_constraint = 3;
+    optional bool validate_constraint = 4;
+    optional bool rely_constraint = 5;
+  }
+
+  repeated NotNullConstraint nns = 1;
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index f53944f..bd33c71 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -52,7 +52,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -847,8 +849,24 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public void createTableWithConstraints(Table tbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
   }
@@ -872,6 +890,18 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
   public Map<String, List<ColumnStatisticsObj>> getColStatsForTablePartitions(String dbName,
       String tableName) throws MetaException, NoSuchObjectException {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index e0f5cdb..94cbd52 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -53,7 +53,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -863,8 +865,24 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+    throws MetaException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public void createTableWithConstraints(Table tbl,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
     throws InvalidObjectException, MetaException {
     // TODO Auto-generated method stub
   }
@@ -887,6 +905,17 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     // TODO Auto-generated method stub
   }
 
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns)
+    throws InvalidObjectException, MetaException {
+    // TODO Auto-generated method stub
+  }
+
   @Override
   public Map<String, List<ColumnStatisticsObj>> getColStatsForTablePartitions(String dbName,
       String tableName) throws MetaException, NoSuchObjectException {

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index 0cf56e5..4aa8c34 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -56,7 +56,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -1405,7 +1407,7 @@ public class TestHBaseStore {
     List<SQLPrimaryKey> pk = Arrays.asList(
         new SQLPrimaryKey(DB, tableName, pkColNames[0], 0, pkName, true, false, true));
 
-    store.createTableWithConstraints(table, pk, null);
+    store.createTableWithConstraints(table, pk, null, null, null);
 
     pk = store.getPrimaryKeys(DB, tableName);
 
@@ -1441,7 +1443,7 @@ public class TestHBaseStore {
         new SQLForeignKey(DB, pkTable, pkColNames[0], DB, tableName, fkColNames[0], 0, 1, 2,
             fkName, pkName, true, false, false));
 
-    store.createTableWithConstraints(table, null, fk);
+    store.createTableWithConstraints(table, null, fk, null, null);
 
     fk = store.getForeignKeys(DB, pkTable, DB, tableName);
 
@@ -1760,12 +1762,142 @@ public class TestHBaseStore {
     List<SQLPrimaryKey> pk = Arrays.asList(
         new SQLPrimaryKey(DB, tableName, pkColNames[0], 0, pkName, true, false, true));
 
-    store.createTableWithConstraints(table, pk, null);
+    store.createTableWithConstraints(table, pk, null, null, null);
 
     store.addPrimaryKeys(pk);
   }
 
+  @Test
+  public void createTableWithUniqueConstraint() throws Exception {
+    String tableName = "uktable";
+    String ukName = "test_uk";
+    String ukColNames[] = { "col0" };
+    Table table = createMultiColumnTable(tableName, "int");
+
+    List<SQLUniqueConstraint> uk = Arrays.asList(
+        new SQLUniqueConstraint(DB, tableName, ukColNames[0], 0, ukName, true, false, true));
+
+    store.createTableWithConstraints(table, null, null, uk, null);
+
+    uk = store.getUniqueConstraints(DB, tableName);
+
+    Assert.assertNotNull(uk);
+    Assert.assertEquals(1, uk.size());
+    Assert.assertEquals(DB, uk.get(0).getTable_db());
+    Assert.assertEquals(tableName, uk.get(0).getTable_name());
+    Assert.assertEquals(ukColNames[0], uk.get(0).getColumn_name());
+    Assert.assertEquals(0, uk.get(0).getKey_seq());
+    Assert.assertEquals(ukName, uk.get(0).getUk_name());
+    Assert.assertTrue(uk.get(0).isEnable_cstr());
+    Assert.assertFalse(uk.get(0).isValidate_cstr());
+    Assert.assertTrue(uk.get(0).isRely_cstr());
+
+    // Drop the unique constraint
+    store.dropConstraint(DB, tableName, ukName);
+
+    uk = store.getUniqueConstraints(DB, tableName);
+    Assert.assertNull(uk);
+  }
+
+  @Test
+  public void addMultiUniqueConstraints() throws Exception {
+    String tableName = "mcuktable";
+    String ukName = "test_uk";
+    String ukName2 = "test_uk2";
+    String ukColNames[] = { "col0", "col1" };
+    Table table = createMultiColumnTable(tableName, "int", "double", "timestamp");
+
+    List<SQLUniqueConstraint> uks = Arrays.asList(
+        new SQLUniqueConstraint(DB, tableName, ukColNames[0], 0, ukName, true, false, true),
+        new SQLUniqueConstraint(DB, tableName, ukColNames[1], 0, ukName2, true, false, true)
+    );
+
+    store.createTable(table);
+    store.addUniqueConstraints(uks);
+
+    uks = store.getUniqueConstraints(DB, tableName);
+
+    Assert.assertNotNull(uks);
+    Assert.assertEquals(2, uks.size());
+    SQLUniqueConstraint[] sorted = uks.toArray(new SQLUniqueConstraint[2]);
+    Arrays.sort(sorted, new Comparator<SQLUniqueConstraint>() {
+      @Override
+      public int compare(SQLUniqueConstraint o1, SQLUniqueConstraint o2) {
+        if (o1.getUk_name().equals(o2.getUk_name())) {
+          return o1.getColumn_name().compareTo(o2.getColumn_name());
+        } else {
+          return o1.getUk_name().compareTo(o2.getUk_name());
+        }
+      }
+    });
+
+    Assert.assertEquals(DB, sorted[0].getTable_db());
+    Assert.assertEquals(tableName, sorted[0].getTable_name());
+    Assert.assertEquals(ukColNames[0], sorted[0].getColumn_name());
+    Assert.assertEquals(0, sorted[0].getKey_seq());
+    Assert.assertEquals(ukName, sorted[0].getUk_name());
+    Assert.assertTrue(sorted[0].isEnable_cstr());
+    Assert.assertFalse(sorted[0].isValidate_cstr());
+    Assert.assertTrue(sorted[0].isRely_cstr());
+
+    Assert.assertEquals(DB, sorted[1].getTable_db());
+    Assert.assertEquals(tableName, sorted[1].getTable_name());
+    Assert.assertEquals(ukColNames[1], sorted[1].getColumn_name());
+    Assert.assertEquals(0, sorted[1].getKey_seq());
+    Assert.assertEquals(ukName2, sorted[1].getUk_name());
+    Assert.assertTrue(sorted[1].isEnable_cstr());
+    Assert.assertFalse(sorted[1].isValidate_cstr());
+    Assert.assertTrue(sorted[1].isRely_cstr());
+  }
+
+  @Test
+  public void addMultiNotNullConstraints() throws Exception {
+    String tableName = "mcnntable";
+    String nnName = "test_nn";
+    String nnName2 = "test_nn2";
+    String nnColNames[] = { "col0", "col1" };
+    Table table = createMultiColumnTable(tableName, "int", "double", "timestamp");
+
+    List<SQLNotNullConstraint> nns = Arrays.asList(
+        new SQLNotNullConstraint(DB, tableName, nnColNames[0], nnName, true, false, true),
+        new SQLNotNullConstraint(DB, tableName, nnColNames[1], nnName2, true, false, true)
+    );
 
+    store.createTable(table);
+    store.addNotNullConstraints(nns);
+
+    nns = store.getNotNullConstraints(DB, tableName);
+
+    Assert.assertNotNull(nns);
+    Assert.assertEquals(2, nns.size());
+    SQLNotNullConstraint[] sorted = nns.toArray(new SQLNotNullConstraint[2]);
+    Arrays.sort(sorted, new Comparator<SQLNotNullConstraint>() {
+      @Override
+      public int compare(SQLNotNullConstraint o1, SQLNotNullConstraint o2) {
+        if (o1.getNn_name().equals(o2.getNn_name())) {
+          return o1.getColumn_name().compareTo(o2.getColumn_name());
+        } else {
+          return o1.getNn_name().compareTo(o2.getNn_name());
+        }
+      }
+    });
+
+    Assert.assertEquals(DB, sorted[0].getTable_db());
+    Assert.assertEquals(tableName, sorted[0].getTable_name());
+    Assert.assertEquals(nnColNames[0], sorted[0].getColumn_name());
+    Assert.assertEquals(nnName, sorted[0].getNn_name());
+    Assert.assertTrue(sorted[0].isEnable_cstr());
+    Assert.assertFalse(sorted[0].isValidate_cstr());
+    Assert.assertTrue(sorted[0].isRely_cstr());
+
+    Assert.assertEquals(DB, sorted[1].getTable_db());
+    Assert.assertEquals(tableName, sorted[1].getTable_name());
+    Assert.assertEquals(nnColNames[1], sorted[1].getColumn_name());
+    Assert.assertEquals(nnName2, sorted[1].getNn_name());
+    Assert.assertTrue(sorted[1].isEnable_cstr());
+    Assert.assertFalse(sorted[1].isValidate_cstr());
+    Assert.assertTrue(sorted[1].isRely_cstr());
+  }
 
   private Table createMockTableAndPartition(String partType, String partVal) throws Exception {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index d01a203..6651900 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -452,12 +452,14 @@ public enum ErrorMsg {
   CLASSPATH_ERROR(10323, "Classpath error"),
   IMPORT_SEMANTIC_ERROR(10324, "Import Semantic Analyzer Error"),
   INVALID_FK_SYNTAX(10325, "Invalid Foreign Key syntax"),
-  INVALID_PK_SYNTAX(10326, "Invalid Primary Key syntax"),
+  INVALID_CSTR_SYNTAX(10326, "Invalid Constraint syntax"),
   ACID_NOT_ENOUGH_HISTORY(10327, "Not enough history available for ({0},{1}).  " +
     "Oldest available base: {2}", true),
   INVALID_COLUMN_NAME(10328, "Invalid column name"),
   UNSUPPORTED_SET_OPERATOR(10329, "Unsupported set operator"),
   LOCK_ACQUIRE_CANCELLED(10330, "Query was cancelled while acquiring locks on the underlying objects. "),
+  NOT_RECOGNIZED_CONSTRAINT(10331, "Constraint not recognized"),
+  INVALID_CONSTRAINT(10332, "Invalid constraint definition"),
   REPLACE_VIEW_WITH_MATERIALIZED(10400, "Attempt to replace view {0} with materialized view", true),
   REPLACE_MATERIALIZED_WITH_VIEW(10401, "Attempt to replace materialized view {0} with view", true),
   UPDATE_DELETE_VIEW(10402, "You cannot update or delete records in a view"),

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 44655af..617555e 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
@@ -62,9 +62,8 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.io.HdfsUtils;
-import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.StatObjectConverter;
@@ -87,7 +86,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
@@ -136,10 +137,12 @@ import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.NotNullConstraint;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PartitionIterable;
 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.formatting.MetaDataFormatUtils;
 import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter;
 import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc;
@@ -240,8 +243,6 @@ import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.shims.HadoopShims;
-import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.tools.HadoopArchives;
@@ -389,7 +390,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         if (alterTbl.getOp() == AlterTableTypes.DROPCONSTRAINT ) {
           return dropConstraint(db, alterTbl);
         } else if (alterTbl.getOp() == AlterTableTypes.ADDCONSTRAINT) {
-          return addConstraint(db, alterTbl);
+          return addConstraints(db, alterTbl);
         } else {
           return alterTable(db, alterTbl);
         }
@@ -3415,9 +3416,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
       PrimaryKeyInfo pkInfo = null;
       ForeignKeyInfo fkInfo = null;
+      UniqueConstraint ukInfo = null;
+      NotNullConstraint nnInfo = 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());
       }
       fixDecimalColumnTypeName(cols);
       // In case the query is served by HiveServer2, don't pad it with spaces,
@@ -3425,7 +3430,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
       formatter.describeTable(outStream, colPath, tableName, tbl, part,
           cols, descTbl.isFormatted(), descTbl.isExt(),
-          descTbl.isPretty(), isOutputPadded, colStats, pkInfo, fkInfo);
+          descTbl.isPretty(), isOutputPadded, colStats,
+          pkInfo, fkInfo, ukInfo, nnInfo);
 
       LOG.debug("DDLTask: written data for " + tbl.getTableName());
 
@@ -3604,6 +3610,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       } else {
         db.alterPartitions(tbl.getTableName(), allPartitions, alterTbl.getEnvironmentContext());
       }
+      // Add constraints if necessary
+      addConstraints(db, alterTbl);
     } catch (InvalidOperationException e) {
       LOG.error("alter table: " + stringifyException(e));
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
@@ -4004,27 +4012,38 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-   private int dropConstraint(Hive db, AlterTableDesc alterTbl)
-    throws SemanticException, HiveException {
-     try {
-      db.dropConstraint(Utilities.getDatabaseName(alterTbl.getOldName()),
-        Utilities.getTableName(alterTbl.getOldName()),
-          alterTbl.getConstraintName());
-      } catch (NoSuchObjectException e) {
-        throw new HiveException(e);
-      }
-     return 0;
-   }
-
-   private int addConstraint(Hive db, AlterTableDesc alterTbl)
-    throws SemanticException, HiveException {
+  private int dropConstraint(Hive db, AlterTableDesc alterTbl)
+          throws SemanticException, HiveException {
     try {
-    // This is either an alter table add foreign key or add primary key command.
-    if (!alterTbl.getForeignKeyCols().isEmpty()) {
-       db.addForeignKey(alterTbl.getForeignKeyCols());
-     } else if (!alterTbl.getPrimaryKeyCols().isEmpty()) {
-       db.addPrimaryKey(alterTbl.getPrimaryKeyCols());
+     db.dropConstraint(Utilities.getDatabaseName(alterTbl.getOldName()),
+       Utilities.getTableName(alterTbl.getOldName()),
+         alterTbl.getConstraintName());
+     } catch (NoSuchObjectException e) {
+       throw new HiveException(e);
      }
+    return 0;
+  }
+
+  private int addConstraints(Hive db, AlterTableDesc alterTbl)
+           throws SemanticException, HiveException {
+    try {
+      // This is either an alter table add foreign key or add primary key command.
+      if (alterTbl.getForeignKeyCols() != null
+              && !alterTbl.getForeignKeyCols().isEmpty()) {
+        db.addForeignKey(alterTbl.getForeignKeyCols());
+      }
+      if (alterTbl.getPrimaryKeyCols() != null
+              && !alterTbl.getPrimaryKeyCols().isEmpty()) {
+        db.addPrimaryKey(alterTbl.getPrimaryKeyCols());
+      }
+      if (alterTbl.getUniqueConstraintCols() != null
+              && !alterTbl.getUniqueConstraintCols().isEmpty()) {
+        db.addUniqueConstraint(alterTbl.getUniqueConstraintCols());
+      }
+      if (alterTbl.getNotNullConstraintCols() != null
+              && !alterTbl.getNotNullConstraintCols().isEmpty()) {
+        db.addNotNullConstraint(alterTbl.getNotNullConstraintCols());
+      }
     } catch (NoSuchObjectException e) {
       throw new HiveException(e);
     }
@@ -4338,6 +4357,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     Table tbl = crtTbl.toTable(conf);
     List<SQLPrimaryKey> primaryKeys = crtTbl.getPrimaryKeys();
     List<SQLForeignKey> foreignKeys = crtTbl.getForeignKeys();
+    List<SQLUniqueConstraint> uniqueConstraints = crtTbl.getUniqueConstraints();
+    List<SQLNotNullConstraint> notNullConstraints = crtTbl.getNotNullConstraints();
     LOG.info("creating table " + tbl.getDbName() + "." + tbl.getTableName() + " on " +
             tbl.getDataLocation());
 
@@ -4381,8 +4402,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
     } else {
       if ((foreignKeys != null && foreignKeys.size() > 0 ) ||
-          (primaryKeys != null && primaryKeys.size() > 0)) {
-        db.createTable(tbl, crtTbl.getIfNotExists(), primaryKeys, foreignKeys);
+          (primaryKeys != null && primaryKeys.size() > 0) ||
+          (uniqueConstraints != null && uniqueConstraints.size() > 0) ||
+          (notNullConstraints != null && notNullConstraints.size() > 0)) {
+        db.createTable(tbl, crtTbl.getIfNotExists(), primaryKeys, foreignKeys,
+                uniqueConstraints, notNullConstraints);
       } else {
         db.createTable(tbl, crtTbl.getIfNotExists());
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 5b908e8..3f032c8 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
@@ -62,7 +62,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hive.common.BlobStorageUtils;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.ObjectPair;
@@ -107,6 +106,7 @@ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
@@ -115,12 +115,15 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 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.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
@@ -833,7 +836,11 @@ public class Hive {
    * @throws HiveException
    */
   public void createTable(Table tbl, boolean ifNotExists,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)  throws HiveException {
+    List<SQLPrimaryKey> primaryKeys,
+    List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints,
+    List<SQLNotNullConstraint> notNullConstraints)
+            throws HiveException {
     try {
       if (tbl.getDbName() == null || "".equals(tbl.getDbName().trim())) {
         tbl.setDbName(SessionState.get().getCurrentDatabase());
@@ -858,10 +865,12 @@ public class Hive {
           tTbl.setPrivileges(principalPrivs);
         }
       }
-      if (primaryKeys == null && foreignKeys == null) {
+      if (primaryKeys == null && foreignKeys == null
+              && uniqueConstraints == null && notNullConstraints == null) {
         getMSC().createTable(tTbl);
       } else {
-        getMSC().createTableWithConstraints(tTbl, primaryKeys, foreignKeys);
+        getMSC().createTableWithConstraints(tTbl, primaryKeys, foreignKeys,
+            uniqueConstraints, notNullConstraints);
       }
 
     } catch (AlreadyExistsException e) {
@@ -874,7 +883,7 @@ public class Hive {
   }
 
   public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
-   createTable(tbl, ifNotExists, null, null);
+   createTable(tbl, ifNotExists, null, null, null, null);
  }
 
   public static List<FieldSchema> getFieldsFromDeserializerForMsStorage(
@@ -4114,6 +4123,42 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  /**
+   * Get all unique constraints associated with the table.
+   *
+   * @param dbName Database Name
+   * @param tblName Table Name
+   * @return Unique constraints associated with the table.
+   * @throws HiveException
+   */
+  public UniqueConstraint getUniqueConstraints(String dbName, String tblName) throws HiveException {
+    try {
+      List<SQLUniqueConstraint> uniqueConstraints = getMSC().getUniqueConstraints(
+              new UniqueConstraintsRequest(dbName, tblName));
+      return new UniqueConstraint(uniqueConstraints, tblName, dbName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
+   * Get all not null constraints associated with the table.
+   *
+   * @param dbName Database Name
+   * @param tblName Table Name
+   * @return Not null constraints associated with the table.
+   * @throws HiveException
+   */
+  public NotNullConstraint getNotNullConstraints(String dbName, String tblName) throws HiveException {
+    try {
+      List<SQLNotNullConstraint> notNullConstraints = getMSC().getNotNullConstraints(
+              new NotNullConstraintsRequest(dbName, tblName));
+      return new NotNullConstraint(notNullConstraints, tblName, dbName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
   public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols)
     throws HiveException, NoSuchObjectException {
     try {
@@ -4131,4 +4176,22 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e);
     }
   }
+
+  public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols)
+    throws HiveException, NoSuchObjectException {
+    try {
+      getMSC().addUniqueConstraint(uniqueConstraintCols);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols)
+    throws HiveException, NoSuchObjectException {
+    try {
+      getMSC().addNotNullConstraint(notNullConstraintCols);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
 };

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/java/org/apache/hadoop/hive/ql/metadata/NotNullConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/NotNullConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/NotNullConstraint.java
new file mode 100644
index 0000000..915eb30
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/NotNullConstraint.java
@@ -0,0 +1,86 @@
+/**
+ * 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.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+
+/**
+ * NotNullConstraintInfo is a metadata structure containing the not null constraints
+ * associated with a table.
+ */
+@SuppressWarnings("serial")
+public class NotNullConstraint implements Serializable {
+
+  // Mapping from constraint name to list of not null columns
+  Map<String, String> notNullConstraints;
+  String databaseName;
+  String tableName;
+
+  public NotNullConstraint() {}
+
+  public NotNullConstraint(List<SQLNotNullConstraint> nns, String tableName, String databaseName) {
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.notNullConstraints = new TreeMap<String, String>();
+    if (nns ==null) {
+      return;
+    }
+    for (SQLNotNullConstraint pk : nns) {
+      if (pk.getTable_db().equalsIgnoreCase(databaseName) &&
+          pk.getTable_name().equalsIgnoreCase(tableName)) {
+        notNullConstraints.put(pk.getNn_name(), pk.getColumn_name());
+      }
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public Map<String, String> getNotNullConstraints() {
+    return notNullConstraints;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Not Null Constraints for " + databaseName + "." + tableName + ":");
+    sb.append("[");
+    if (notNullConstraints != null && notNullConstraints.size() > 0) {
+      for (Map.Entry<String, String> me : notNullConstraints.entrySet()) {
+        sb.append(" {Constraint Name: " + me.getKey());
+        sb.append(", Column Name: " + me.getValue());
+        sb.append("},");
+      }
+      sb.setLength(sb.length()-1);
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/java/org/apache/hadoop/hive/ql/metadata/UniqueConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/UniqueConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/UniqueConstraint.java
new file mode 100644
index 0000000..5022a19
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/UniqueConstraint.java
@@ -0,0 +1,111 @@
+/**
+ * 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.SQLUniqueConstraint;
+
+/**
+ * UniqueConstraintInfo is a metadata structure containing the unique constraints
+ * associated with a table.
+ */
+@SuppressWarnings("serial")
+public class UniqueConstraint implements Serializable {
+
+  public class UniqueConstraintCol {
+    public String colName;
+    public Integer position;
+
+    public UniqueConstraintCol(String colName, Integer position) {
+      this.colName = colName;
+      this.position = position;
+    }
+  }
+
+  // Mapping from constraint name to list of unique constraints
+  Map<String, List<UniqueConstraintCol>> uniqueConstraints;
+  String tableName;
+  String databaseName;
+
+  public UniqueConstraint() {}
+
+  public UniqueConstraint(List<SQLUniqueConstraint> uks, String tableName, String databaseName) {
+    this.tableName = tableName;
+    this.databaseName = databaseName;
+    uniqueConstraints = new TreeMap<String, List<UniqueConstraintCol>>();
+    if (uks == null) {
+      return;
+    }
+    for (SQLUniqueConstraint uk : uks) {
+      if (uk.getTable_db().equalsIgnoreCase(databaseName) &&
+          uk.getTable_name().equalsIgnoreCase(tableName)) {
+        UniqueConstraintCol currCol = new UniqueConstraintCol(
+                uk.getColumn_name(), uk.getKey_seq());
+        String constraintName = uk.getUk_name();
+        if (uniqueConstraints.containsKey(constraintName)) {
+          uniqueConstraints.get(constraintName).add(currCol);
+        } else {
+          List<UniqueConstraintCol> currList = new ArrayList<UniqueConstraintCol>();
+          currList.add(currCol);
+          uniqueConstraints.put(constraintName, currList);
+        }
+      }
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public Map<String, List<UniqueConstraintCol>> getUniqueConstraints() {
+    return uniqueConstraints;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Unique Constraints for " + databaseName + "." + tableName + ":");
+    sb.append("[");
+    if (uniqueConstraints != null && uniqueConstraints.size() > 0) {
+      for (Map.Entry<String, List<UniqueConstraintCol>> me : uniqueConstraints.entrySet()) {
+        sb.append(" {Constraint Name: " + me.getKey() + ",");
+        List<UniqueConstraintCol> currCol = me.getValue();
+        if (currCol != null && currCol.size() > 0) {
+          for (UniqueConstraintCol ukc : currCol) {
+            sb.append (" (Column Name: " + ukc.colName + ", Key Sequence: " + ukc.position+ "),");
+          }
+          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/696be9f5/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 3315806..d795a19 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
@@ -41,9 +41,11 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 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.codehaus.jackson.map.ObjectMapper;
 
 /**
@@ -104,7 +106,9 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
   public void describeTable(DataOutputStream out, String colPath,
       String tableName, Table tbl, Partition part, List<FieldSchema> cols,
       boolean isFormatted, boolean isExt, boolean isPretty,
-      boolean isOutputPadded, List<ColumnStatisticsObj> colStats, PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo) throws HiveException {
+      boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
+      PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo) throws HiveException {
     MapBuilder builder = MapBuilder.create();
     builder.put("columns", makeColsUnformatted(cols));
 
@@ -121,6 +125,12 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
       if (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) {
         builder.put("foreignKeyInfo", fkInfo);
       }
+      if (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) {
+        builder.put("uniqueConstraintInfo", ukInfo);
+      }
+      if (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty()) {
+        builder.put("notNullConstraintInfo", nnInfo);
+      }
     }
 
     asJson(out, builder.build());

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 f73c610..aa77234 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
@@ -42,7 +42,10 @@ import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 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.ql.plan.ShowIndexesDesc;
@@ -275,7 +278,8 @@ public final class MetaDataFormatUtils {
     return indexInfo.toString();
   }
 
-  public static String getConstraintsInformation(PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo) {
+  public static String getConstraintsInformation(PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
+          UniqueConstraint ukInfo, NotNullConstraint nnInfo) {
     StringBuilder constraintsInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 
     constraintsInfo.append(LINE_DELIM).append("# Constraints").append(LINE_DELIM);
@@ -287,6 +291,14 @@ public final class MetaDataFormatUtils {
       constraintsInfo.append(LINE_DELIM).append("# Foreign Keys").append(LINE_DELIM);
       getForeignKeysInformation(constraintsInfo, fkInfo);
     }
+    if (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) {
+      constraintsInfo.append(LINE_DELIM).append("# Unique Constraints").append(LINE_DELIM);
+      getUniqueConstraintsInformation(constraintsInfo, ukInfo);
+    }
+    if (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty()) {
+      constraintsInfo.append(LINE_DELIM).append("# Not Null Constraints").append(LINE_DELIM);
+      getNotNullConstraintsInformation(constraintsInfo, nnInfo);
+    }
     return constraintsInfo.toString();
   }
 
@@ -338,6 +350,55 @@ public final class MetaDataFormatUtils {
     }
   }
 
+  private static void getUniqueConstraintColInformation(StringBuilder constraintsInfo,
+      UniqueConstraintCol ukCol) {
+    String[] fkcFields = new String[2];
+    fkcFields[0] = "Column Name:" + ukCol.colName;
+    fkcFields[1] = "Key Sequence:" + ukCol.position;
+    formatOutput(fkcFields, constraintsInfo);
+  }
+
+  private static void getUniqueConstraintRelInformation(
+      StringBuilder constraintsInfo,
+      String constraintName,
+      List<UniqueConstraintCol> ukRel) {
+    formatOutput("Constraint Name:", constraintName, constraintsInfo);
+    if (ukRel != null && ukRel.size() > 0) {
+      for (UniqueConstraintCol ukc : ukRel) {
+        getUniqueConstraintColInformation(constraintsInfo, ukc);
+      }
+    }
+    constraintsInfo.append(LINE_DELIM);
+  }
+
+  private static void getUniqueConstraintsInformation(StringBuilder constraintsInfo,
+      UniqueConstraint ukInfo) {
+    formatOutput("Table:",
+                 ukInfo.getDatabaseName() + "." + ukInfo.getTableName(),
+                 constraintsInfo);
+    Map<String, List<UniqueConstraintCol>> uniqueConstraints = ukInfo.getUniqueConstraints();
+    if (uniqueConstraints != null && uniqueConstraints.size() > 0) {
+      for (Map.Entry<String, List<UniqueConstraintCol>> me : uniqueConstraints.entrySet()) {
+        getUniqueConstraintRelInformation(constraintsInfo, me.getKey(), me.getValue());
+      }
+    }
+  }
+
+  private static void getNotNullConstraintsInformation(StringBuilder constraintsInfo,
+      NotNullConstraint nnInfo) {
+    formatOutput("Table:",
+                 nnInfo.getDatabaseName() + "." + nnInfo.getTableName(),
+                 constraintsInfo);
+    Map<String, String> notNullConstraints = nnInfo.getNotNullConstraints();
+    if (notNullConstraints != null && notNullConstraints.size() > 0) {
+      for (Map.Entry<String, String> me : notNullConstraints.entrySet()) {
+        formatOutput("Constraint Name:", me.getKey(), constraintsInfo);
+        formatOutput("Column Name:", me.getValue(), constraintsInfo);
+        constraintsInfo.append(LINE_DELIM);
+      }
+    }
+  }
+
   public static String getPartitionInformation(Partition part) {
     StringBuilder tableInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 71b7ebf..72ee440 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,9 +30,11 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 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;
 
 /**
  * Interface to format table and index information.  We can format it
@@ -75,12 +77,16 @@ public interface MetaDataFormatter {
    * @param colStats
    * @param fkInfo  foreign keys information
    * @param pkInfo  primary key information
+   * @param ukInfo  unique constraint information
+   * @param nnInfo  not null constraint information
    * @throws HiveException
    */
   public void describeTable(DataOutputStream out, String colPath,
       String tableName, Table tbl, Partition part, List<FieldSchema> cols,
       boolean isFormatted, boolean isExt, boolean isPretty,
-      boolean isOutputPadded, List<ColumnStatisticsObj> colStats, PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo)
+      boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
+      PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo)
           throws HiveException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 39a327d..dcecb0e 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
@@ -44,9 +44,11 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 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.session.SessionState;
 
 /**
@@ -122,7 +124,9 @@ class TextMetaDataFormatter implements MetaDataFormatter {
   public void describeTable(DataOutputStream outStream,  String colPath,
       String tableName, Table tbl, Partition part, List<FieldSchema> cols,
       boolean isFormatted, boolean isExt, boolean isPretty,
-      boolean isOutputPadded, List<ColumnStatisticsObj> colStats, PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo) throws HiveException {
+      boolean isOutputPadded, List<ColumnStatisticsObj> colStats,
+      PrimaryKeyInfo pkInfo, ForeignKeyInfo fkInfo,
+      UniqueConstraint ukInfo, NotNullConstraint nnInfo) throws HiveException {
     try {
       String output;
       if (colPath.equals(tableName)) {
@@ -155,8 +159,10 @@ class TextMetaDataFormatter implements MetaDataFormatter {
           outStream.write(output.getBytes("UTF-8"));
 
           if ((pkInfo != null && !pkInfo.getColNames().isEmpty()) ||
-              (fkInfo != null && !fkInfo.getForeignKeys().isEmpty())) {
-            output = MetaDataFormatUtils.getConstraintsInformation(pkInfo, fkInfo);
+              (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) ||
+              (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) ||
+              (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty())) {
+            output = MetaDataFormatUtils.getConstraintsInformation(pkInfo, fkInfo, ukInfo, nnInfo);
             outStream.write(output.getBytes("UTF-8"));
           }
         }
@@ -182,17 +188,27 @@ class TextMetaDataFormatter implements MetaDataFormatter {
             outStream.write(terminator);
           }
           if ((pkInfo != null && !pkInfo.getColNames().isEmpty()) ||
-              (fkInfo != null && !fkInfo.getForeignKeys().isEmpty())) {
-              outStream.write(("Constraints").getBytes("UTF-8"));
-              outStream.write(separator);
-              if (pkInfo != null && !pkInfo.getColNames().isEmpty()) {
-                outStream.write(pkInfo.toString().getBytes("UTF-8"));
-                outStream.write(terminator);
-              }
-              if (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) {
-                outStream.write(fkInfo.toString().getBytes("UTF-8"));
-                outStream.write(terminator);
-              }
+              (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) ||
+              (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) ||
+              (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty())) {
+            outStream.write(("Constraints").getBytes("UTF-8"));
+            outStream.write(separator);
+            if (pkInfo != null && !pkInfo.getColNames().isEmpty()) {
+              outStream.write(pkInfo.toString().getBytes("UTF-8"));
+              outStream.write(terminator);
+            }
+            if (fkInfo != null && !fkInfo.getForeignKeys().isEmpty()) {
+              outStream.write(fkInfo.toString().getBytes("UTF-8"));
+              outStream.write(terminator);
+            }
+            if (ukInfo != null && !ukInfo.getUniqueConstraints().isEmpty()) {
+              outStream.write(ukInfo.toString().getBytes("UTF-8"));
+              outStream.write(terminator);
+            }
+            if (nnInfo != null && !nnInfo.getNotNullConstraints().isEmpty()) {
+              outStream.write(nnInfo.toString().getBytes("UTF-8"));
+              outStream.write(terminator);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 41245c8..136e951 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
@@ -46,7 +46,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -88,6 +90,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 
 /**
  * BaseSemanticAnalyzer.
@@ -627,127 +630,175 @@ public abstract class BaseSemanticAnalyzer {
    * Get the list of FieldSchema out of the ASTNode.
    */
   public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase) throws SemanticException {
-    return getColumns(ast, lowerCase, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>());
+    return getColumns(ast, lowerCase, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
+            new ArrayList<SQLUniqueConstraint>(), new ArrayList<SQLNotNullConstraint>());
   }
 
-  static class PKInfo {
-   public String colName;
-   public String constraintName;
-   public boolean rely;
+  private static class ConstraintInfo {
+    final String colName;
+    final String constraintName;
+    final boolean enable;
+    final boolean validate;
+    final boolean rely;
 
-   public PKInfo(String colName, String constraintName, boolean rely) {
-     this.colName = colName;
-     this.constraintName = constraintName;
-     this.rely = rely;
-   }
+    ConstraintInfo(String colName, String constraintName,
+        boolean enable, boolean validate, boolean rely) {
+      this.colName = colName;
+      this.constraintName = constraintName;
+      this.enable = enable;
+      this.validate = validate;
+      this.rely = rely;
+    }
   }
 
   /**
-   * Get the primary keys from the AST and populate the pkInfos with the required
-   * information.
-   * @param child  The node with primary key token
-   * @param pkInfos Primary Key information structure
-   * @throws SemanticException
+   * Process the primary keys from the ast node and populate the SQLPrimaryKey list.
    */
-  private static void processPrimaryKeyInfos(
-    ASTNode child, List<PKInfo> pkInfos) throws SemanticException {
-    if (child.getChildCount() < 4) {
-      throw new SemanticException(ErrorMsg.INVALID_PK_SYNTAX.getMsg());
+  protected static void processPrimaryKeys(String databaseName, String tableName,
+      ASTNode child, List<SQLPrimaryKey> primaryKeys) throws SemanticException {
+    List<ConstraintInfo> primaryKeyInfos = new ArrayList<ConstraintInfo>();
+    generateConstraintInfos(child, primaryKeyInfos);
+    constraintInfosToPrimaryKeys(databaseName, tableName, primaryKeyInfos, primaryKeys);
+  }
+
+  protected static void processPrimaryKeys(String databaseName, String tableName,
+      ASTNode child, List<String> columnNames, List<SQLPrimaryKey> primaryKeys)
+          throws SemanticException {
+    List<ConstraintInfo> primaryKeyInfos = new ArrayList<ConstraintInfo>();
+    generateConstraintInfos(child, columnNames, primaryKeyInfos);
+    constraintInfosToPrimaryKeys(databaseName, tableName, primaryKeyInfos, primaryKeys);
+  }
+
+  private static void constraintInfosToPrimaryKeys(String databaseName, String tableName,
+          List<ConstraintInfo> primaryKeyInfos, List<SQLPrimaryKey> primaryKeys) {
+    int i = 1;
+    for (ConstraintInfo primaryKeyInfo : primaryKeyInfos) {
+      primaryKeys.add(new SQLPrimaryKey(databaseName, tableName, primaryKeyInfo.colName,
+              i++, primaryKeyInfo.constraintName, primaryKeyInfo.enable,
+              primaryKeyInfo.validate, primaryKeyInfo.rely));
     }
-    // The ANTLR grammar looks like :
-    // 1. KW_CONSTRAINT idfr=identifier KW_PRIMARY KW_KEY pkCols=columnParenthesesList
-    //  enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    // -> ^(TOK_PRIMARY_KEY $pkCols $idfr $relySpec $enableSpec $validateSpec)
-    // when the user specifies the constraint name (i.e. child.getChildCount() == 5)
-    // 2.  KW_PRIMARY KW_KEY columnParenthesesList
-    // enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
-    // -> ^(TOK_PRIMARY_KEY columnParenthesesList $relySpec $enableSpec $validateSpec)
-    // when the user does not specify the constraint name (i.e. child.getChildCount() == 4)
-    boolean userSpecifiedConstraintName = child.getChildCount() == 5;
-    int relyIndex =  child.getChildCount() == 5 ? 2 : 1;
-    for (int j = 0; j < child.getChild(0).getChildCount(); j++) {
-     Tree grandChild = child.getChild(0).getChild(j);
-     boolean rely = child.getChild(relyIndex).getType() == HiveParser.TOK_VALIDATE;
-     boolean enable =  child.getChild(relyIndex+1).getType() == HiveParser.TOK_ENABLE;
-     boolean validate =  child.getChild(relyIndex+2).getType() == HiveParser.TOK_VALIDATE;
-     if (enable) {
-       throw new SemanticException(
-         ErrorMsg.INVALID_PK_SYNTAX.getMsg(" ENABLE feature not supported yet"));
-     }
-     if (validate) {
-       throw new SemanticException(
-         ErrorMsg.INVALID_PK_SYNTAX.getMsg(" VALIDATE feature not supported yet"));
-     }
-      checkColumnName(grandChild.getText());
-     pkInfos.add(
-       new PKInfo(
-         unescapeIdentifier(grandChild.getText().toLowerCase()),
-         (userSpecifiedConstraintName ?
-         unescapeIdentifier(child.getChild(1).getText().toLowerCase()) : null),
-         rely));
+  }
+
+  /**
+   * Process the unique constraints from the ast node and populate the SQLUniqueConstraint list.
+   */
+  protected static void processUniqueConstraints(String databaseName, String tableName,
+      ASTNode child, List<SQLUniqueConstraint> uniqueConstraints) throws SemanticException {
+    List<ConstraintInfo> uniqueInfos = new ArrayList<ConstraintInfo>();
+    generateConstraintInfos(child, uniqueInfos);
+    constraintInfosToUniqueConstraints(databaseName, tableName, uniqueInfos, uniqueConstraints);
+  }
+
+  protected static void processUniqueConstraints(String databaseName, String tableName,
+      ASTNode child, List<String> columnNames, List<SQLUniqueConstraint> uniqueConstraints)
+          throws SemanticException {
+    List<ConstraintInfo> uniqueInfos = new ArrayList<ConstraintInfo>();
+    generateConstraintInfos(child, columnNames, uniqueInfos);
+    constraintInfosToUniqueConstraints(databaseName, tableName, uniqueInfos, uniqueConstraints);
+  }
+
+  private static void constraintInfosToUniqueConstraints(String databaseName, String tableName,
+          List<ConstraintInfo> uniqueInfos, List<SQLUniqueConstraint> uniqueConstraints) {
+    int i = 1;
+    for (ConstraintInfo uniqueInfo : uniqueInfos) {
+      uniqueConstraints.add(new SQLUniqueConstraint(databaseName, tableName, uniqueInfo.colName,
+              i++, uniqueInfo.constraintName, uniqueInfo.enable, uniqueInfo.validate, uniqueInfo.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);
+    constraintInfosToNotNullConstraints(databaseName, tableName, notNullInfos, notNullConstraints);
+  }
+
+  private static void constraintInfosToNotNullConstraints(String databaseName, String tableName,
+          List<ConstraintInfo> notNullInfos, List<SQLNotNullConstraint> notNullConstraints) {
+    for (ConstraintInfo notNullInfo : notNullInfos) {
+      notNullConstraints.add(new SQLNotNullConstraint(databaseName, tableName, notNullInfo.colName,
+              notNullInfo.constraintName, notNullInfo.enable, notNullInfo.validate, notNullInfo.rely));
     }
   }
 
   /**
-   * Process the primary keys from the pkInfos structure and populate the SQLPrimaryKey list
-   * @param parent Parent of the primary key token node
-   * @param pkInfos primary key information
-   * @param primaryKeys SQLPrimaryKey list
-   * @param nametoFS Mapping from column name to field schema for the current table
+   * Get the constraint from the AST and populate the cstrInfos with the required
+   * information.
+   * @param child  The node with the constraint token
+   * @param cstrInfos Constraint information
    * @throws SemanticException
    */
-  private static void processPrimaryKeys(ASTNode parent, List<PKInfo> pkInfos,
-    List<SQLPrimaryKey> primaryKeys, Map<String, FieldSchema> nametoFS) throws SemanticException {
-    int cnt = 1;
-    String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
-
-    for (int i = 0; i < pkInfos.size(); i++) {
-      String pk = pkInfos.get(i).colName;
-      if (nametoFS.containsKey(pk)) {
-        SQLPrimaryKey currPrimaryKey = new SQLPrimaryKey(
-          qualifiedTabName[0], qualifiedTabName[1], pk, cnt++, pkInfos.get(i).constraintName,
-          false, false, pkInfos.get(i).rely);
-        primaryKeys.add(currPrimaryKey);
-      } else {
-        throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(pk));
-      }
+  private static void generateConstraintInfos(ASTNode child,
+      List<ConstraintInfo> cstrInfos) throws SemanticException {
+    ImmutableList.Builder<String> columnNames = ImmutableList.builder();
+    for (int j = 0; j < child.getChild(0).getChildCount(); j++) {
+      Tree columnName = child.getChild(0).getChild(j);
+      checkColumnName(columnName.getText());
+      columnNames.add(unescapeIdentifier(columnName.getText().toLowerCase()));
     }
+    generateConstraintInfos(child, columnNames.build(), cstrInfos);
   }
 
   /**
-   * Process the primary keys from the ast nodes and populate the SQLPrimaryKey list.
-   * As of now, this is used by 'alter table add constraint' command. We expect constraint
-   * name to be user specified.
-   * @param parent Parent of the primary key token node
-   * @param child Child of the primary key token node containing the primary key columns details
-   * @param primaryKeys SQLPrimaryKey list to be populated by this function
+   * 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
    * @throws SemanticException
    */
-  protected static void processPrimaryKeys(ASTNode parent, ASTNode child, List<SQLPrimaryKey> primaryKeys)
-    throws SemanticException {
-    int relyIndex = 2;
-    int cnt = 1;
-    String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
-    for (int j = 0; j < child.getChild(0).getChildCount(); j++) {
-     Tree grandChild = child.getChild(0).getChild(j);
-     boolean rely = child.getChild(relyIndex).getType() == HiveParser.TOK_VALIDATE;
-     boolean enable =  child.getChild(relyIndex+1).getType() == HiveParser.TOK_ENABLE;
-     boolean validate =  child.getChild(relyIndex+2).getType() == HiveParser.TOK_VALIDATE;
-     if (enable) {
-       throw new SemanticException(
-         ErrorMsg.INVALID_PK_SYNTAX.getMsg(" ENABLE feature not supported yet"));
-     }
-     if (validate) {
-       throw new SemanticException(
-         ErrorMsg.INVALID_PK_SYNTAX.getMsg(" VALIDATE feature not supported yet"));
-     }
-     primaryKeys.add(
-       new SQLPrimaryKey(
-         qualifiedTabName[0], qualifiedTabName[1],
-         unescapeIdentifier(grandChild.getText().toLowerCase()),
-         cnt++,
-         unescapeIdentifier(child.getChild(1).getText().toLowerCase()), false, false,
-         rely));
+  private static void generateConstraintInfos(ASTNode child, List<String> columnNames,
+      List<ConstraintInfo> cstrInfos) throws SemanticException {
+    // The ANTLR grammar looks like :
+    // 1. KW_CONSTRAINT idfr=identifier KW_PRIMARY KW_KEY pkCols=columnParenthesesList
+    //  constraintOptsCreate?
+    // -> ^(TOK_PRIMARY_KEY $pkCols $idfr constraintOptsCreate?)
+    // when the user specifies the constraint name.
+    // 2.  KW_PRIMARY KW_KEY columnParenthesesList
+    // constraintOptsCreate?
+    // -> ^(TOK_PRIMARY_KEY columnParenthesesList constraintOptsCreate?)
+    // when the user does not specify the constraint name.
+    // Default values
+    String constraintName = null;
+    boolean enable = true;
+    boolean validate = true;
+    boolean rely = false;
+    for (int i = 0; i < child.getChildCount(); i++) {
+      ASTNode grandChild = (ASTNode) child.getChild(i);
+      int type = grandChild.getToken().getType();
+      if (type == HiveParser.TOK_CONSTRAINT_NAME) {
+        constraintName = unescapeIdentifier(grandChild.getChild(0).getText().toLowerCase());
+      } else if (type == HiveParser.TOK_ENABLE) {
+        enable = true;
+        // validate is true by default if we enable the constraint
+        validate = true;
+      } else if (type == HiveParser.TOK_DISABLE) {
+        enable = false;
+        // validate is false by default if we disable the constraint
+        validate = false;
+      } else if (type == HiveParser.TOK_VALIDATE) {
+        validate = true;
+      } else if (type == HiveParser.TOK_NOVALIDATE) {
+        validate = false;
+      } else if (type == HiveParser.TOK_RELY) {
+        rely = true;
+      }
+    }
+    if (enable) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("ENABLE feature not supported yet. "
+              + "Please use DISABLE instead."));
+    }
+    if (validate) {
+      throw new SemanticException(
+        ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("VALIDATE feature not supported yet. "
+              + "Please use NOVALIDATE instead."));
+    }
+
+    for (String columnName : columnNames) {
+      cstrInfos.add(new ConstraintInfo(columnName, constraintName,
+          enable, validate, rely));
     }
   }
 
@@ -758,9 +809,8 @@ public abstract class BaseSemanticAnalyzer {
    * @param foreignKeys SQLForeignKey list
    * @throws SemanticException
    */
-  protected static void processForeignKeys(
-    ASTNode parent, ASTNode child, List<SQLForeignKey> foreignKeys) throws SemanticException {
-    String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+  protected static void processForeignKeys(String databaseName, String tableName,
+      ASTNode child, List<SQLForeignKey> foreignKeys) throws SemanticException {
     // The ANTLR grammar looks like :
     // 1.  KW_CONSTRAINT idfr=identifier KW_FOREIGN KW_KEY fkCols=columnParenthesesList
     // KW_REFERENCES tabName=tableName parCols=columnParenthesesList
@@ -772,50 +822,69 @@ public abstract class BaseSemanticAnalyzer {
     // enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
     // -> ^(TOK_FOREIGN_KEY $fkCols  $tabName $parCols $relySpec $enableSpec $validateSpec)
     // when the user does not specify the constraint name (i.e. child.getChildCount() == 6)
-    boolean userSpecifiedConstraintName = child.getChildCount() == 7;
-    int fkIndex = userSpecifiedConstraintName ? 1 : 0;
-    int ptIndex = fkIndex + 1;
-    int pkIndex = ptIndex + 1;
-    int relyIndex = pkIndex + 1;
-
-    if (child.getChildCount() <= fkIndex ||child.getChildCount() <= pkIndex ||
-      child.getChildCount() <= ptIndex) {
-      throw new SemanticException(ErrorMsg.INVALID_FK_SYNTAX.getMsg());
+    String constraintName = null;
+    boolean enable = true;
+    boolean validate = true;
+    boolean rely = false;
+    int fkIndex = -1;
+    for (int i = 0; i < child.getChildCount(); i++) {
+      ASTNode grandChild = (ASTNode) child.getChild(i);
+      int type = grandChild.getToken().getType();
+      if (type == HiveParser.TOK_CONSTRAINT_NAME) {
+        constraintName = unescapeIdentifier(grandChild.getChild(0).getText().toLowerCase());
+      } else if (type == HiveParser.TOK_ENABLE) {
+        enable = true;
+        // validate is true by default if we enable the constraint
+        validate = true;
+      } else if (type == HiveParser.TOK_DISABLE) {
+        enable = false;
+        // validate is false by default if we disable the constraint
+        validate = false;
+      } else if (type == HiveParser.TOK_VALIDATE) {
+        validate = true;
+      } else if (type == HiveParser.TOK_NOVALIDATE) {
+        validate = false;
+      } else if (type == HiveParser.TOK_RELY) {
+        rely = true;
+      } else if (type == HiveParser.TOK_TABCOLNAME && fkIndex == -1) {
+        fkIndex = i;
+      }
+    }
+    if (enable) {
+      throw new SemanticException(
+          ErrorMsg.INVALID_FK_SYNTAX.getMsg("ENABLE feature not supported yet. "
+              + "Please use DISABLE instead."));
+    }
+    if (validate) {
+      throw new SemanticException(
+        ErrorMsg.INVALID_FK_SYNTAX.getMsg("VALIDATE feature not supported yet. "
+              + "Please use NOVALIDATE instead."));
     }
 
-    String[] parentDBTbl = getQualifiedTableName((ASTNode) child.getChild(ptIndex));
-
+    int ptIndex = fkIndex + 1;
+    int pkIndex = ptIndex + 1;
     if (child.getChild(fkIndex).getChildCount() != child.getChild(pkIndex).getChildCount()) {
       throw new SemanticException(ErrorMsg.INVALID_FK_SYNTAX.getMsg(
         " The number of foreign key columns should be same as number of parent key columns "));
     }
+
+    String[] parentDBTbl = getQualifiedTableName((ASTNode) child.getChild(ptIndex));
     for (int j = 0; j < child.getChild(fkIndex).getChildCount(); j++) {
       SQLForeignKey sqlForeignKey = new SQLForeignKey();
+      sqlForeignKey.setFktable_db(databaseName);
+      sqlForeignKey.setFktable_name(tableName);
       Tree fkgrandChild = child.getChild(fkIndex).getChild(j);
       checkColumnName(fkgrandChild.getText());
-      boolean rely = child.getChild(relyIndex).getType() == HiveParser.TOK_VALIDATE;
-      boolean enable =  child.getChild(relyIndex+1).getType() == HiveParser.TOK_ENABLE;
-      boolean validate =  child.getChild(relyIndex+2).getType() == HiveParser.TOK_VALIDATE;
-      if (enable) {
-        throw new SemanticException(
-          ErrorMsg.INVALID_FK_SYNTAX.getMsg(" ENABLE feature not supported yet"));
-      }
-      if (validate) {
-        throw new SemanticException(
-          ErrorMsg.INVALID_FK_SYNTAX.getMsg(" VALIDATE feature not supported yet"));
-      }
-      sqlForeignKey.setRely_cstr(rely);
+      sqlForeignKey.setFkcolumn_name(unescapeIdentifier(fkgrandChild.getText().toLowerCase()));
       sqlForeignKey.setPktable_db(parentDBTbl[0]);
       sqlForeignKey.setPktable_name(parentDBTbl[1]);
-      sqlForeignKey.setFktable_db(qualifiedTabName[0]);
-      sqlForeignKey.setFktable_name(qualifiedTabName[1]);
-      sqlForeignKey.setFkcolumn_name(unescapeIdentifier(fkgrandChild.getText().toLowerCase()));
       Tree pkgrandChild = child.getChild(pkIndex).getChild(j);
       sqlForeignKey.setPkcolumn_name(unescapeIdentifier(pkgrandChild.getText().toLowerCase()));
       sqlForeignKey.setKey_seq(j+1);
-      if (userSpecifiedConstraintName) {
-        sqlForeignKey.setFk_name(unescapeIdentifier(child.getChild(0).getText().toLowerCase()));
-      }
+      sqlForeignKey.setFk_name(constraintName);
+      sqlForeignKey.setEnable_cstr(enable);
+      sqlForeignKey.setValidate_cstr(validate);
+      sqlForeignKey.setRely_cstr(rely);
       foreignKeys.add(sqlForeignKey);
     }
   }
@@ -831,47 +900,95 @@ public abstract class BaseSemanticAnalyzer {
    * Additionally, populate the primaryKeys and foreignKeys if any.
    */
   public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) throws SemanticException {
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints)
+        throws SemanticException {
     List<FieldSchema> colList = new ArrayList<FieldSchema>();
-    int numCh = ast.getChildCount();
-    List<PKInfo> pkInfos = new ArrayList<PKInfo>();
-    Map<String, FieldSchema> nametoFS = new HashMap<String, FieldSchema>();
     Tree parent = ast.getParent();
 
-    for (int i = 0; i < numCh; i++) {
+    for (int i = 0; i < ast.getChildCount(); i++) {
       FieldSchema col = new FieldSchema();
       ASTNode child = (ASTNode) ast.getChild(i);
-      if (child.getToken().getType() == HiveParser.TOK_PRIMARY_KEY) {
-        processPrimaryKeyInfos(child, pkInfos);
-      } else if (child.getToken().getType() == HiveParser.TOK_FOREIGN_KEY) {
-        processForeignKeys((ASTNode)parent, child, foreignKeys);
-      }
-      else {
-        Tree grandChild = child.getChild(0);
-        if(grandChild != null) {
-          String name = grandChild.getText();
-          if(lowerCase) {
-            name = name.toLowerCase();
+      switch (child.getToken().getType()) {
+        case HiveParser.TOK_UNIQUE: {
+            String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+            processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1], child, uniqueConstraints);
           }
-          checkColumnName(name);
-          // child 0 is the name of the column
-          col.setName(unescapeIdentifier(name));
-          // child 1 is the type of the column
-          ASTNode typeChild = (ASTNode) (child.getChild(1));
-          col.setType(getTypeStringFromAST(typeChild));
-
-          // child 2 is the optional comment of the column
-          if (child.getChildCount() == 3) {
-            col.setComment(unescapeSQLString(child.getChild(2).getText()));
+          break;
+        case HiveParser.TOK_PRIMARY_KEY: {
+            if (!primaryKeys.isEmpty()) {
+              throw new SemanticException(ErrorMsg.INVALID_CONSTRAINT.getMsg(
+                  "Cannot exist more than one primary key definition for the same table"));
+            }
+            String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+            processPrimaryKeys(qualifiedTabName[0], qualifiedTabName[1], child, primaryKeys);
           }
-        }
-        nametoFS.put(col.getName(), col);
-        colList.add(col);
+          break;
+        case HiveParser.TOK_FOREIGN_KEY: {
+            String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+            processForeignKeys(qualifiedTabName[0], qualifiedTabName[1], child, foreignKeys);
+          }
+          break;
+        default:
+          Tree grandChild = child.getChild(0);
+          if(grandChild != null) {
+            String name = grandChild.getText();
+            if(lowerCase) {
+              name = name.toLowerCase();
+            }
+            checkColumnName(name);
+            // child 0 is the name of the column
+            col.setName(unescapeIdentifier(name));
+            // child 1 is the type of the column
+            ASTNode typeChild = (ASTNode) (child.getChild(1));
+            col.setType(getTypeStringFromAST(typeChild));
+
+            // child 2 is the optional comment of the column
+            // child 3 is the optional constraint
+            ASTNode constraintChild = null;
+            if (child.getChildCount() == 4) {
+              col.setComment(unescapeSQLString(child.getChild(2).getText()));
+              constraintChild = (ASTNode) child.getChild(3);
+            } else if (child.getChildCount() == 3
+                && ((ASTNode) child.getChild(2)).getToken().getType() == HiveParser.StringLiteral) {
+              col.setComment(unescapeSQLString(child.getChild(2).getText()));
+            } else if (child.getChildCount() == 3) {
+              constraintChild = (ASTNode) child.getChild(2);
+            }
+            if (constraintChild != null) {
+              String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+              // Process column constraint
+              switch (constraintChild.getToken().getType()) {
+                case HiveParser.TOK_NOT_NULL:
+                  processNotNullConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                          ImmutableList.of(col.getName()), notNullConstraints);
+                  break;
+                case HiveParser.TOK_UNIQUE:
+                  processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                          ImmutableList.of(col.getName()), uniqueConstraints);
+                  break;
+                case HiveParser.TOK_PRIMARY_KEY:
+                  if (!primaryKeys.isEmpty()) {
+                    throw new SemanticException(ErrorMsg.INVALID_CONSTRAINT.getMsg(
+                        "Cannot exist more than one primary key definition for the same table"));
+                  }
+                  processPrimaryKeys(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                          ImmutableList.of(col.getName()), primaryKeys);
+                  break;
+                case HiveParser.TOK_FOREIGN_KEY:
+                  processForeignKeys(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                          foreignKeys);
+                  break;
+                default:
+                  throw new SemanticException(ErrorMsg.NOT_RECOGNIZED_CONSTRAINT.getMsg(
+                      constraintChild.getToken().getText()));
+              }
+            }
+          }
+          colList.add(col);
+          break;
       }
     }
-    if (!pkInfos.isEmpty()) {
-      processPrimaryKeys((ASTNode) parent, pkInfos, primaryKeys, nametoFS);
-    }
     return colList;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 7601267..dee6a10 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import org.antlr.runtime.tree.CommonTree;
@@ -41,7 +42,9 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -1782,16 +1785,31 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   private void analyzeAlterTableAddConstraint(ASTNode ast, String tableName)
     throws SemanticException {
     ASTNode parent = (ASTNode) ast.getParent();
+    String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
     ASTNode child = (ASTNode) ast.getChild(0);
-    List<SQLPrimaryKey> primaryKeys = new ArrayList<SQLPrimaryKey>();
-    List<SQLForeignKey> foreignKeys = new ArrayList<SQLForeignKey>();
-
-    if (child.getToken().getType() == HiveParser.TOK_PRIMARY_KEY) {
-      BaseSemanticAnalyzer.processPrimaryKeys(parent, child, primaryKeys);
-    } else if (child.getToken().getType() == HiveParser.TOK_FOREIGN_KEY) {
-      BaseSemanticAnalyzer.processForeignKeys(parent, child, foreignKeys);
+    List<SQLPrimaryKey> primaryKeys = new ArrayList<>();
+    List<SQLForeignKey> foreignKeys = new ArrayList<>();
+    List<SQLUniqueConstraint> uniqueConstraints = new ArrayList<>();
+
+    switch (child.getToken().getType()) {
+      case HiveParser.TOK_UNIQUE:
+        BaseSemanticAnalyzer.processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1],
+                child, uniqueConstraints);        
+        break;
+      case HiveParser.TOK_PRIMARY_KEY:
+        BaseSemanticAnalyzer.processPrimaryKeys(qualifiedTabName[0], qualifiedTabName[1],
+                child, primaryKeys);
+        break;
+      case HiveParser.TOK_FOREIGN_KEY:
+        BaseSemanticAnalyzer.processForeignKeys(qualifiedTabName[0], qualifiedTabName[1],
+                child, foreignKeys);
+        break;
+      default:
+        throw new SemanticException(ErrorMsg.NOT_RECOGNIZED_CONSTRAINT.getMsg(
+                child.getToken().getText()));
     }
-    AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, primaryKeys, foreignKeys);
+    AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, primaryKeys, foreignKeys,
+            uniqueConstraints);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         alterTblDesc), conf));
@@ -2620,6 +2638,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     String oldColName = ast.getChild(0).getText();
     String newColName = ast.getChild(1).getText();
     String newType = getTypeStringFromAST((ASTNode) ast.getChild(2));
+    ASTNode constraintChild = null;
     int childCount = ast.getChildCount();
     for (int i = 3; i < childCount; i++) {
       ASTNode child = (ASTNode)ast.getChild(i);
@@ -2639,8 +2658,39 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         case HiveParser.TOK_RESTRICT:
           break;
         default:
-          throw new SemanticException("Unsupported token: " + child.getToken()
-              + " for alter table");
+          constraintChild = (ASTNode) child;
+      }
+    }
+    List<SQLPrimaryKey> primaryKeys = null;
+    List<SQLForeignKey> foreignKeys = null;
+    List<SQLUniqueConstraint> uniqueConstraints = null;
+    List<SQLNotNullConstraint> notNullConstraints = 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()));
       }
     }
 
@@ -2656,9 +2706,18 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     String tblName = getDotName(qualified);
-    AlterTableDesc alterTblDesc = new AlterTableDesc(tblName, partSpec,
-        unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
-        newType, newComment, first, flagCol, isCascade);
+    AlterTableDesc alterTblDesc;
+    if (primaryKeys == null && foreignKeys == null
+            && uniqueConstraints == null && notNullConstraints == null) {
+      alterTblDesc = new AlterTableDesc(tblName, partSpec,
+          unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
+          newType, newComment, first, flagCol, isCascade);
+    } else {
+      alterTblDesc = new AlterTableDesc(tblName, partSpec,
+          unescapeIdentifier(oldColName), unescapeIdentifier(newColName),
+          newType, newComment, first, flagCol, isCascade,
+          primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints);
+    }
     addInputsOutputsAlterTable(tblName, partSpec, alterTblDesc);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index cebe441..393df65 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -336,6 +336,7 @@ KW_VALIDATE: 'VALIDATE';
 KW_NOVALIDATE: 'NOVALIDATE';
 KW_RELY: 'RELY';
 KW_NORELY: 'NORELY';
+KW_UNIQUE: 'UNIQUE';
 KW_KEY: 'KEY';
 KW_ABORT: 'ABORT';
 KW_EXTRACT: 'EXTRACT';


[04/31] hive git commit: HIVE-16675: Fix ConcurrentModificationException in SparkClientImpl#startDriver (Zhang Liyun, reviewed by Ferdinand Xu)

Posted by we...@apache.org.
HIVE-16675: Fix ConcurrentModificationException in SparkClientImpl#startDriver (Zhang Liyun, reviewed by Ferdinand Xu)


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

Branch: refs/heads/hive-14535
Commit: cc78d037987b5683ff731c4e733bfc0d3380566f
Parents: 8a093a7
Author: Ferdinand Xu <ch...@intel.com>
Authored: Thu May 25 13:56:41 2017 +0800
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Thu May 25 13:56:41 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hive/spark/client/SparkClientImpl.java   | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cc78d037/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
index d4b63f0..c4495a9 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
@@ -46,6 +46,7 @@ import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -481,7 +482,7 @@ class SparkClientImpl implements SparkClient {
 
       final Process child = pb.start();
       int childId = childIdGenerator.incrementAndGet();
-      final List<String> childErrorLog = new ArrayList<String>();
+      final List<String> childErrorLog = Collections.synchronizedList(new ArrayList<String>());
       redirect("stdout-redir-" + childId, new Redirector(child.getInputStream()));
       redirect("stderr-redir-" + childId, new Redirector(child.getErrorStream(), childErrorLog));
 
@@ -492,9 +493,12 @@ class SparkClientImpl implements SparkClient {
             int exitCode = child.waitFor();
             if (exitCode != 0) {
               StringBuilder errStr = new StringBuilder();
-              for (String s : childErrorLog) {
-                errStr.append(s);
-                errStr.append('\n');
+              synchronized(childErrorLog) {
+                Iterator iter = childErrorLog.iterator();
+                while(iter.hasNext()){
+                  errStr.append(iter.next());
+                  errStr.append('\n');
+                }
               }
 
               rpcServer.cancelClient(clientId,


[27/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 696be9f52dfc6fb59c24de19726b4460100fc9ba
Parents: 823f01c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed May 3 10:09:49 2017 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu May 25 11:26:22 2017 +0100

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    28 +-
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |     2 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |     2 +
 metastore/if/hive_metastore.thrift              |    61 +-
 .../upgrade/derby/042-HIVE-16575.derby.sql      |     4 +
 .../upgrade/derby/hive-schema-3.0.0.derby.sql   |     4 +-
 .../derby/upgrade-2.3.0-to-3.0.0.derby.sql      |     2 +-
 .../upgrade/hive/hive-schema-3.0.0.hive.sql     |    78 +-
 .../upgrade/mssql/027-HIVE-16575.mssql.sql      |     1 +
 .../upgrade/mssql/hive-schema-3.0.0.mssql.sql   |     2 +
 .../mssql/upgrade-2.3.0-to-3.0.0.mssql.sql      |     1 +
 .../upgrade/mysql/042-HIVE-16575.mysql.sql      |     1 +
 .../upgrade/mysql/hive-schema-3.0.0.mysql.sql   |     2 +
 .../mysql/upgrade-2.3.0-to-3.0.0.mysql.sql      |     1 +
 .../upgrade/oracle/042-HIVE-16575.oracle.sql    |     1 +
 .../upgrade/oracle/hive-schema-3.0.0.oracle.sql |     2 +
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |     1 +
 .../postgres/041-HIVE-16575.postgres.sql        |     1 +
 .../postgres/hive-schema-3.0.0.postgres.sql     |     2 +
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |     1 +
 .../metastore/hbase/HbaseMetastoreProto.java    |  4700 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  5850 ++-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |   584 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    22 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  7378 +--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   436 +
 .../hive/metastore/api/AbortTxnsRequest.java    |    32 +-
 .../metastore/api/AddDynamicPartitions.java     |    32 +-
 .../metastore/api/AddForeignKeyRequest.java     |    36 +-
 .../api/AddNotNullConstraintRequest.java        |   443 +
 .../metastore/api/AddPartitionsRequest.java     |    36 +-
 .../hive/metastore/api/AddPartitionsResult.java |    36 +-
 .../metastore/api/AddPrimaryKeyRequest.java     |    36 +-
 .../api/AddUniqueConstraintRequest.java         |   443 +
 .../metastore/api/ClearFileMetadataRequest.java |    32 +-
 .../hive/metastore/api/ClientCapabilities.java  |    32 +-
 .../hive/metastore/api/CompactionRequest.java   |    44 +-
 .../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 +-
 .../api/HeartbeatTxnRangeResponse.java          |    64 +-
 .../metastore/api/InsertEventRequestData.java   |    64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |    36 +-
 .../api/NotNullConstraintsRequest.java          |   490 +
 .../api/NotNullConstraintsResponse.java         |   443 +
 .../api/NotificationEventResponse.java          |    36 +-
 .../hive/metastore/api/OpenTxnsResponse.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/SQLNotNullConstraint.java     |  1005 +
 .../hive/metastore/api/SQLUniqueConstraint.java |  1103 +
 .../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/ThriftHiveMetastore.java | 43424 +++++++++--------
 .../metastore/api/UniqueConstraintsRequest.java |   490 +
 .../api/UniqueConstraintsResponse.java          |   443 +
 .../gen-php/metastore/ThriftHiveMetastore.php   |  2554 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  9511 ++--
 .../hive_metastore/ThriftHiveMetastore-remote   |    36 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  2096 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  2207 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   166 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   274 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   128 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    37 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    21 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   111 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   324 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    15 +-
 .../hive/metastore/cache/CachedStore.java       |    37 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |    76 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   103 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |   131 +-
 .../hive/metastore/model/MConstraint.java       |     4 +
 .../metastore/hbase/hbase_metastore_proto.proto |    34 +
 .../DummyRawStoreControlledCommit.java          |    32 +-
 .../DummyRawStoreForJdoConnection.java          |    31 +-
 .../hive/metastore/hbase/TestHBaseStore.java    |   138 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |     4 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    78 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    73 +-
 .../hive/ql/metadata/NotNullConstraint.java     |    86 +
 .../hive/ql/metadata/UniqueConstraint.java      |   111 +
 .../formatting/JsonMetaDataFormatter.java       |    12 +-
 .../formatting/MetaDataFormatUtils.java         |    63 +-
 .../metadata/formatting/MetaDataFormatter.java  |     8 +-
 .../formatting/TextMetaDataFormatter.java       |    44 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   447 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    85 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |     1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   158 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    12 +-
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |    44 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |    41 +-
 .../hadoop/hive/ql/plan/ImportTableDesc.java    |     6 +-
 .../hive/ql/parse/TestHiveDecimalParse.java     |     2 +-
 .../TestSQL11ReservedKeyWordsNegative.java      |    13 +
 .../alter_table_constraint_duplicate_pk.q       |     2 +-
 .../alter_table_constraint_invalid_fk_col1.q    |     4 +-
 .../alter_table_constraint_invalid_fk_col2.q    |     4 +-
 .../alter_table_constraint_invalid_fk_tbl1.q    |     4 +-
 .../alter_table_constraint_invalid_fk_tbl2.q    |     4 +-
 .../alter_table_constraint_invalid_pk_tbl.q     |     2 +-
 .../create_with_constraints_duplicate_name.q    |     4 +-
 .../create_with_constraints_enable.q            |     2 +-
 .../clientnegative/create_with_fk_constraint.q  |     2 +
 .../create_with_multi_pk_constraint.q           |     1 +
 .../clientnegative/drop_invalid_constraint1.q   |     2 +-
 .../clientnegative/drop_invalid_constraint2.q   |     2 +-
 .../clientnegative/drop_invalid_constraint3.q   |     2 +-
 .../clientnegative/drop_invalid_constraint4.q   |     4 +-
 .../clientpositive/create_with_constraints.q    |    84 +-
 .../alter_table_constraint_duplicate_pk.q.out   |     4 +-
 ...alter_table_constraint_invalid_fk_col1.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_col2.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_tbl1.q.out |     8 +-
 ...alter_table_constraint_invalid_fk_tbl2.q.out |     8 +-
 .../alter_table_constraint_invalid_pk_tbl.q.out |     4 +-
 ...create_with_constraints_duplicate_name.q.out |     6 +-
 .../create_with_constraints_enable.q.out        |     2 +-
 .../create_with_constraints_validate.q.out      |     2 +-
 .../create_with_fk_constraint.q.out             |    13 +
 .../create_with_multi_pk_constraint.q.out       |     1 +
 .../drop_invalid_constraint1.q.out              |     4 +-
 .../drop_invalid_constraint2.q.out              |     4 +-
 .../drop_invalid_constraint3.q.out              |     4 +-
 .../drop_invalid_constraint4.q.out              |     8 +-
 .../create_with_constraints.q.out               |  1241 +-
 .../results/clientpositive/llap/sysdb.q.out     |   152 +-
 143 files changed, 57491 insertions(+), 32342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 3dc63bd..b016920 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
@@ -55,7 +55,9 @@ 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.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
 import org.apache.hadoop.hive.metastore.api.Type;
@@ -893,8 +895,22 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    return null;
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+      throws MetaException {
+    return null;
+  }
+
+  @Override
   public void createTableWithConstraints(Table tbl,
-                                         List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)
+                                         List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+                                         List<SQLUniqueConstraint> uniqueConstraints,
+                                         List<SQLNotNullConstraint> notNullConstraints)
       throws InvalidObjectException, MetaException {
   }
 
@@ -914,6 +930,16 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public void addUniqueConstraints(List<SQLUniqueConstraint> uks)
+      throws InvalidObjectException, MetaException {
+  }
+
+  @Override
+  public void addNotNullConstraints(List<SQLNotNullConstraint> nns)
+      throws InvalidObjectException, MetaException {
+  }
+
+  @Override
   public Map<String, List<ColumnStatisticsObj>> getColStatsForTablePartitions(String dbName,
       String tableName) throws MetaException, NoSuchObjectException {
     return objectStore.getColStatsForTablePartitions(dbName, tableName);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index 6e9223a..1108934 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -141,7 +141,7 @@ public class TestJdbcDriver2 {
         + " (under_col int comment 'the under column', value string) comment '" + tableComment
         + "'");
     stmt.execute("create table " + tableNameWithPk
-        + " (a STRING, b STRING, primary key (a) disable novalidate) ");
+        + " (a STRING, b STRING, primary key (a) disable) ");
     // load data
     stmt.execute("load data local inpath '" + dataFilePath.toString() + "' into table " + tableName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 111cc11..cd6c41c 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
@@ -1696,6 +1696,8 @@ public class QTestUtil {
       ".*.hive-staging.*",
       "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]*",
       ".*at com\\.sun\\.proxy.*",
       ".*at com\\.jolbox.*",
       ".*at com\\.zaxxer.*",

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 53e5f29..0573f0c 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -69,6 +69,27 @@ struct SQLForeignKey {
   14: bool rely_cstr       // Rely/No Rely
 }
 
+struct SQLUniqueConstraint {
+  1: string table_db,    // table schema
+  2: string table_name,  // table name
+  3: string column_name, // column name
+  4: i32 key_seq,        // sequence number within unique constraint
+  5: string uk_name,     // unique key name
+  6: bool enable_cstr,   // Enable/Disable
+  7: bool validate_cstr, // Validate/No validate
+  8: bool rely_cstr      // Rely/No Rely
+}
+
+struct SQLNotNullConstraint {
+  1: string table_db,    // table schema
+  2: string table_name,  // table name
+  3: string column_name, // column name
+  4: string nn_name,     // not null name
+  5: bool enable_cstr,   // Enable/Disable
+  6: bool validate_cstr, // Validate/No validate
+  7: 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)
@@ -497,6 +518,24 @@ struct ForeignKeysResponse {
   1: required list<SQLForeignKey> foreignKeys
 }
 
+struct UniqueConstraintsRequest {
+  1: required string db_name,
+  2: required string tbl_name
+}
+
+struct UniqueConstraintsResponse {
+  1: required list<SQLUniqueConstraint> uniqueConstraints
+}
+
+struct NotNullConstraintsRequest {
+  1: required string db_name,
+  2: required string tbl_name
+}
+
+struct NotNullConstraintsResponse {
+  1: required list<SQLNotNullConstraint> notNullConstraints
+}
+
 struct DropConstraintRequest {
   1: required string dbname, 
   2: required string tablename,
@@ -511,6 +550,14 @@ struct AddForeignKeyRequest {
   1: required list<SQLForeignKey> foreignKeyCols
 }
 
+struct AddUniqueConstraintRequest {
+  1: required list<SQLUniqueConstraint> uniqueConstraintCols
+}
+
+struct AddNotNullConstraintRequest {
+  1: required list<SQLNotNullConstraint> notNullConstraintCols
+}
+
 // Return type for get_partitions_by_expr
 struct PartitionsByExprResult {
   1: required list<Partition> partitions,
@@ -1055,7 +1102,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
       throws (1:AlreadyExistsException o1,
               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)
+  void create_table_with_constraints(1:Table tbl, 2: list<SQLPrimaryKey> primaryKeys, 3: list<SQLForeignKey> foreignKeys,
+  4: list<SQLUniqueConstraint> uniqueConstraints, 5: list<SQLNotNullConstraint> notNullConstraints)
       throws (1:AlreadyExistsException o1,
               2:InvalidObjectException o2, 3:MetaException o3,
               4:NoSuchObjectException o4)
@@ -1065,6 +1113,10 @@ service ThriftHiveMetastore extends fb303.FacebookService
       throws(1:NoSuchObjectException o1, 2:MetaException o2)
   void add_foreign_key(1:AddForeignKeyRequest req)
       throws(1:NoSuchObjectException o1, 2:MetaException o2)  
+  void add_unique_constraint(1:AddUniqueConstraintRequest req)
+      throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  void add_not_null_constraint(1:AddNotNullConstraintRequest 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
@@ -1313,11 +1365,16 @@ service ThriftHiveMetastore extends fb303.FacebookService
   list<string> get_index_names(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1)
                        throws(1:MetaException o2)
 
- //primary keys and foreign keys
+  //primary keys and foreign keys
   PrimaryKeysResponse get_primary_keys(1:PrimaryKeysRequest request)
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
   ForeignKeysResponse get_foreign_keys(1:ForeignKeysRequest request)
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  // other constraints
+  UniqueConstraintsResponse get_unique_constraints(1:UniqueConstraintsRequest request)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  NotNullConstraintsResponse get_not_null_constraints(1:NotNullConstraintsRequest request)
+                       throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
   // column statistics interfaces
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/derby/042-HIVE-16575.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/042-HIVE-16575.derby.sql b/metastore/scripts/upgrade/derby/042-HIVE-16575.derby.sql
new file mode 100644
index 0000000..c614f3e
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/042-HIVE-16575.derby.sql
@@ -0,0 +1,4 @@
+-- Remove the NOT NULL constraint from the CHILD_INTEGER_IDX column
+ALTER TABLE "APP"."KEY_CONSTRAINTS" ALTER COLUMN "CHILD_INTEGER_IDX" NULL;
+
+CREATE INDEX "APP"."CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "APP"."KEY_CONSTRAINTS"("CONSTRAINT_TYPE");

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 0226d6f..a9a5329 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
@@ -106,7 +106,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 NOT NULL, "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);
 
 CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ("PROPERTY_KEY" VARCHAR(255) NOT NULL, "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, "DESCRIPTION" VARCHAR(1000));
 
@@ -150,6 +150,8 @@ CREATE INDEX "APP"."FUNC_RU_N49" ON "APP"."FUNC_RU" ("FUNC_ID");
 
 CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID");
 
+CREATE INDEX "APP"."CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "APP"."KEY_CONSTRAINTS"("CONSTRAINT_TYPE");
+
 -- ----------------------------------------------
 -- DDL Statements for keys
 -- ----------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 455651f..a31fc5e 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -1,5 +1,5 @@
 -- Upgrade MetaStore schema from 2.3.0 to 3.0.0
-
 RUN '041-HIVE-16556.derby.sql';
+RUN '042-HIVE-16575.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/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 a70884c..70559cb 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
@@ -8,7 +8,7 @@ CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
   `SD_ID` bigint,
   `BUCKET_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -24,7 +24,7 @@ FROM
 
 CREATE TABLE IF NOT EXISTS `CDS` (
   `CD_ID` bigint,
-  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -42,7 +42,7 @@ CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
   `COLUMN_NAME` string,
   `TYPE_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -62,7 +62,7 @@ CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
   `DB_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -82,7 +82,7 @@ CREATE TABLE IF NOT EXISTS `DBS` (
   `NAME` string,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -108,7 +108,7 @@ CREATE TABLE IF NOT EXISTS `DB_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `DB_PRIV` string,
-  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -137,7 +137,7 @@ CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `USER_PRIV` string,
-  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -166,7 +166,7 @@ CREATE TABLE IF NOT EXISTS `IDXS` (
   `LAST_ACCESS_TIME` int,
   `ORIG_TBL_ID` bigint,
   `SD_ID` bigint,
-  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -190,7 +190,7 @@ CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` (
   `INDEX_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -211,7 +211,7 @@ CREATE TABLE IF NOT EXISTS `PARTITIONS` (
   `PART_NAME` string,
   `SD_ID` bigint,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -234,7 +234,7 @@ CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` (
   `PKEY_NAME` string,
   `PKEY_TYPE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -254,7 +254,7 @@ CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` (
   `PART_ID` bigint,
   `PART_KEY_VAL` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -272,7 +272,7 @@ CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` (
   `PART_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -297,7 +297,7 @@ CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_COL_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -328,7 +328,7 @@ CREATE TABLE IF NOT EXISTS `PART_PRIVS` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `PART_PRIV` string,
-  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -353,7 +353,7 @@ CREATE TABLE IF NOT EXISTS `ROLES` (
   `CREATE_TIME` int,
   `OWNER_NAME` string,
   `ROLE_NAME` string,
-  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -377,7 +377,7 @@ CREATE TABLE IF NOT EXISTS `ROLE_MAP` (
   `PRINCIPAL_NAME` string,
   `PRINCIPAL_TYPE` string,
   `ROLE_ID` bigint,
-  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -406,7 +406,7 @@ CREATE TABLE IF NOT EXISTS `SDS` (
   `NUM_BUCKETS` int,
   `OUTPUT_FORMAT` string,
   `SERDE_ID` bigint,
-  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -430,7 +430,7 @@ CREATE TABLE IF NOT EXISTS `SD_PARAMS` (
   `SD_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -447,7 +447,7 @@ FROM
 CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` (
   `SEQUENCE_NAME` string,
   `NEXT_VAL` bigint,
-  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -464,7 +464,7 @@ CREATE TABLE IF NOT EXISTS `SERDES` (
   `SERDE_ID` bigint,
   `NAME` string,
   `SLIB` string,
-  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -482,7 +482,7 @@ CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` (
   `SERDE_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -500,7 +500,7 @@ CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` (
   `SD_ID` bigint,
   `SKEWED_COL_NAME` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -518,7 +518,7 @@ CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` (
   `SD_ID` bigint,
   `STRING_LIST_ID_KID` bigint,
   `LOCATION` string,
-  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -534,7 +534,7 @@ FROM
 
 CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` (
   `STRING_LIST_ID` bigint,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -550,7 +550,7 @@ CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` (
   `STRING_LIST_ID` bigint,
   `STRING_LIST_VALUE` string,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -568,7 +568,7 @@ CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` (
   `SD_ID_OID` bigint,
   `STRING_LIST_ID_EID` bigint,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -587,7 +587,7 @@ CREATE TABLE IF NOT EXISTS `SORT_COLS` (
   `COLUMN_NAME` string,
   `ORDER` int,
   `INTEGER_IDX` int,
-  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -606,7 +606,7 @@ CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
   `TBL_ID` bigint,
   `PARAM_KEY` string,
   `PARAM_VALUE` string,
-  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -633,7 +633,7 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `VIEW_EXPANDED_TEXT` string,
   `VIEW_ORIGINAL_TEXT` string,
   `IS_REWRITE_ENABLED` boolean,
-  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -666,7 +666,7 @@ CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_COL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -697,7 +697,7 @@ CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
   `PRINCIPAL_TYPE` string,
   `TBL_PRIV` string,
   `TBL_ID` bigint,
-  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -737,7 +737,7 @@ CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -788,7 +788,7 @@ CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
  `NUM_TRUES` bigint,
  `NUM_FALSES` bigint,
  `LAST_ANALYZED` bigint,
-  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -823,7 +823,7 @@ CREATE TABLE IF NOT EXISTS `VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 );
 
 INSERT INTO `VERSION` VALUES (1, '3.0.0', 'Hive release version 3.0.0');
@@ -832,7 +832,7 @@ CREATE TABLE IF NOT EXISTS `DB_VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
   `VERSION_COMMENT` string,
-  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -855,7 +855,7 @@ CREATE TABLE IF NOT EXISTS `FUNCS` (
   `FUNC_TYPE` int,
   `OWNER_NAME` string,
   `OWNER_TYPE` string,
-  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (
@@ -879,7 +879,7 @@ FROM
 --   `RESOURCE_TYPE` int,
 --   `RESOURCE_URI` string,
 --   `INTEGER_IDX` int,
---   CONSTRAINT `SYS_PK_FUNCS_RU` PRIMARY KEY (`FUNC_ID`, `INTEGER_IDX`) DISABLE NOVALIDATE
+--   CONSTRAINT `SYS_PK_FUNCS_RU` PRIMARY KEY (`FUNC_ID`, `INTEGER_IDX`) DISABLE
 -- )
 -- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 -- TBLPROPERTIES (
@@ -901,7 +901,7 @@ CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
   `UPDATE_RULE` string,
   `DELETE_RULE` string,
   `ENABLE_VALIDATE_RELY` int,
-  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE
+  CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE
 )
 STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
 TBLPROPERTIES (

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mssql/027-HIVE-16575.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/027-HIVE-16575.mssql.sql b/metastore/scripts/upgrade/mssql/027-HIVE-16575.mssql.sql
new file mode 100644
index 0000000..213e649
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/027-HIVE-16575.mssql.sql
@@ -0,0 +1 @@
+CREATE INDEX CONSTRAINTS_CONSTRAINT_TYPE_INDEX ON KEY_CONSTRAINTS(CONSTRAINT_TYPE);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
index d7bd3f8..1cfe2d1 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
@@ -1007,6 +1007,8 @@ ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAIN
 
 CREATE INDEX CONSTRAINTS_PARENT_TBL_ID__INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
 
+CREATE INDEX CONSTRAINTS_CONSTRAINT_TYPE_INDEX ON KEY_CONSTRAINTS(CONSTRAINT_TYPE);
+
 CREATE TABLE WRITE_SET (
   WS_DATABASE nvarchar(128) NOT NULL,
   WS_TABLE nvarchar(128) NOT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index e579dac..f8bf581 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;
 
 :r 026-HIVE-16556.mssql.sql
+:r 027-HIVE-16575.mssql.sql
 
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mysql/042-HIVE-16575.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/042-HIVE-16575.mysql.sql b/metastore/scripts/upgrade/mysql/042-HIVE-16575.mysql.sql
new file mode 100644
index 0000000..b97c571
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/042-HIVE-16575.mysql.sql
@@ -0,0 +1 @@
+CREATE INDEX `CONSTRAINTS_CONSTRAINT_TYPE_INDEX` ON KEY_CONSTRAINTS (`CONSTRAINT_TYPE`) USING BTREE;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
index ff874bb..232b097 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
@@ -829,6 +829,8 @@ CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS`
 
 CREATE INDEX `CONSTRAINTS_PARENT_TABLE_ID_INDEX` ON KEY_CONSTRAINTS (`PARENT_TBL_ID`) USING BTREE;
 
+CREATE INDEX `CONSTRAINTS_CONSTRAINT_TYPE_INDEX` ON KEY_CONSTRAINTS (`CONSTRAINT_TYPE`) USING BTREE;
+
 -- -----------------------------
 -- Metastore DB Properties table
 -- -----------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 2884387..9371f35 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';
 
 SOURCE 041-HIVE-16556.mysql.sql;
+SOURCE 042-HIVE-16575.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/oracle/042-HIVE-16575.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/042-HIVE-16575.oracle.sql b/metastore/scripts/upgrade/oracle/042-HIVE-16575.oracle.sql
new file mode 100644
index 0000000..77adc9d
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/042-HIVE-16575.oracle.sql
@@ -0,0 +1 @@
+CREATE INDEX CONSTRAINTS_CT_INDEX ON KEY_CONSTRAINTS(CONSTRAINT_TYPE);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
index 0ca6143..8fdb552 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
@@ -799,6 +799,8 @@ ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY (CONSTRAIN
 
 CREATE INDEX CONSTRAINTS_PT_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
 
+CREATE INDEX CONSTRAINTS_CT_INDEX ON KEY_CONSTRAINTS(CONSTRAINT_TYPE);
+
 -- Table for METASTORE_DB_PROPERTIES and its constraints
 CREATE TABLE METASTORE_DB_PROPERTIES
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index a15f288..33791d5 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;
 
 @041-HIVE-16556.oracle.sql;
+@042-HIVE-16575.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/postgres/041-HIVE-16575.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/041-HIVE-16575.postgres.sql b/metastore/scripts/upgrade/postgres/041-HIVE-16575.postgres.sql
new file mode 100644
index 0000000..b956c8f
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/041-HIVE-16575.postgres.sql
@@ -0,0 +1 @@
+CREATE INDEX "CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "KEY_CONSTRAINTS" USING BTREE ("CONSTRAINT_TYPE");

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
index 91a71f5..1cdeb6b 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
@@ -1209,6 +1209,8 @@ CREATE INDEX "FUNC_RU_N49" ON "FUNC_RU" ("FUNC_ID");
 
 CREATE INDEX "CONSTRAINTS_PARENT_TBLID_INDEX" ON "KEY_CONSTRAINTS" USING BTREE ("PARENT_TBL_ID");
 
+CREATE INDEX "CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "KEY_CONSTRAINTS" USING BTREE ("CONSTRAINT_TYPE");
+
 ALTER TABLE ONLY "SKEWED_STRING_LIST_VALUES"
     ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_fkey" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "SKEWED_STRING_LIST"("STRING_LIST_ID") DEFERRABLE;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index 4d8f0cd..314198c 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.3.0 to 3.0.0';
 
 \i 040-HIVE-16556.postgres.sql;
+\i 041-HIVE-16575.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.0.0', "VERSION_COMMENT"='Hive release version 3.0.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0';


[05/31] hive git commit: HIVE-16706 : Bootstrap REPL DUMP shouldn't fail when a partition is dropped/renamed when dump in progress (Sankar Hariappan via Thejas Nair)

Posted by we...@apache.org.
HIVE-16706 : Bootstrap REPL DUMP shouldn't fail when a partition is dropped/renamed when dump in progress (Sankar Hariappan via Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: 3c1b354eadb2033681efe3c06a99920ba4912a13
Parents: cc78d03
Author: Sankar Hariappan <ma...@gmail.com>
Authored: Wed May 24 23:26:23 2017 -0700
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Wed May 24 23:26:34 2017 -0700

----------------------------------------------------------------------
 .../hive/ql/parse/TestReplicationScenarios.java | 171 ++++++++++++++-----
 .../InjectableBehaviourObjectStore.java         |  20 +++
 2 files changed, 144 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3c1b354e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 0aee012..21f09ae 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -386,7 +386,6 @@ public class TestReplicationScenarios {
     run("LOAD DATA LOCAL INPATH '" + ptn_locn_2 + "' OVERWRITE INTO TABLE " + dbName + ".ptned PARTITION(b=2)");
     verifySetup("SELECT a from " + dbName + ".ptned WHERE b=2", ptn_data_2);
 
-
     advanceDumpDir();
 
     BehaviourInjection<Table,Table> ptnedTableNuller = new BehaviourInjection<Table,Table>(){
@@ -414,12 +413,14 @@ public class TestReplicationScenarios {
     LOG.info("Bootstrap-Dump: Dumped to {} with id {}", replDumpLocn, replDumpId);
     run("REPL LOAD " + dbName + "_dupe FROM '" + replDumpLocn + "'");
 
-    // The ptned table should miss in target as the table was marked cisrtually as dropped
+    // The ptned table should miss in target as the table was marked virtually as dropped
     verifyRun("SELECT * from " + dbName + "_dupe.unptned", unptn_data);
     verifyFail("SELECT a from " + dbName + "_dupe.ptned WHERE b=1");
+    verifyIfTableNotExist(dbName + "_dupe", "ptned");
 
     // Verify if Drop table on a non-existing table is idempotent
     run("DROP TABLE " + dbName + ".ptned");
+    verifyIfTableNotExist(dbName, "ptned");
 
     advanceDumpDir();
     run("REPL DUMP " + dbName + " FROM " + replDumpId);
@@ -429,10 +430,82 @@ public class TestReplicationScenarios {
     assert(run("REPL LOAD " + dbName + "_dupe FROM '" + postDropReplDumpLocn + "'", true));
 
     verifyRun("SELECT * from " + dbName + "_dupe.unptned", unptn_data);
+    verifyIfTableNotExist(dbName + "_dupe", "ptned");
     verifyFail("SELECT a from " + dbName + "_dupe.ptned WHERE b=1");
   }
 
   @Test
+  public void testBootstrapWithConcurrentDropPartition() throws IOException {
+    String name = testName.getMethodName();
+    String dbName = createDB(name);
+    run("CREATE TABLE " + dbName + ".ptned(a string) partitioned by (b int) STORED AS TEXTFILE");
+
+    String[] ptn_data_1 = new String[]{ "thirteen", "fourteen", "fifteen"};
+    String[] ptn_data_2 = new String[]{ "fifteen", "sixteen", "seventeen"};
+    String[] empty = new String[]{};
+
+    String ptn_locn_1 = new Path(TEST_PATH, name + "_ptn1").toUri().getPath();
+    String ptn_locn_2 = new Path(TEST_PATH, name + "_ptn2").toUri().getPath();
+
+    createTestDataFile(ptn_locn_1, ptn_data_1);
+    createTestDataFile(ptn_locn_2, ptn_data_2);
+
+    run("LOAD DATA LOCAL INPATH '" + ptn_locn_1 + "' OVERWRITE INTO TABLE " + dbName + ".ptned PARTITION(b=1)");
+    verifySetup("SELECT a from " + dbName + ".ptned WHERE b=1", ptn_data_1);
+    run("LOAD DATA LOCAL INPATH '" + ptn_locn_2 + "' OVERWRITE INTO TABLE " + dbName + ".ptned PARTITION(b=2)");
+    verifySetup("SELECT a from " + dbName + ".ptned WHERE b=2", ptn_data_2);
+
+    advanceDumpDir();
+
+    BehaviourInjection<List<String>, List<String>> listPartitionNamesNuller
+            = new BehaviourInjection<List<String>, List<String>>(){
+      @Nullable
+      @Override
+      public List<String> apply(@Nullable List<String> partitions) {
+        injectionPathCalled = true;
+        return new ArrayList<String>();
+      }
+    };
+    InjectableBehaviourObjectStore.setListPartitionNamesBehaviour(listPartitionNamesNuller);
+
+    // None of the partitions will be dumped as the partitions list was empty
+    run("REPL DUMP " + dbName);
+    listPartitionNamesNuller.assertInjectionsPerformed(true, false);
+    InjectableBehaviourObjectStore.resetListPartitionNamesBehaviour(); // reset the behaviour
+
+    String replDumpLocn = getResult(0, 0);
+    String replDumpId = getResult(0, 1, true);
+    LOG.info("Bootstrap-Dump: Dumped to {} with id {}", replDumpLocn, replDumpId);
+    run("REPL LOAD " + dbName + "_dupe FROM '" + replDumpLocn + "'");
+
+    // All partitions should miss in target as it was marked virtually as dropped
+    verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b=1", empty);
+    verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b=2", empty);
+    verifyIfPartitionNotExist(dbName + "_dupe", "ptned", new ArrayList<>(Arrays.asList("1")));
+    verifyIfPartitionNotExist(dbName + "_dupe", "ptned", new ArrayList<>(Arrays.asList("2")));
+
+    // Verify if drop partition on a non-existing partition is idempotent and just a noop.
+    run("ALTER TABLE " + dbName + ".ptned DROP PARTITION (b=1)");
+    run("ALTER TABLE " + dbName + ".ptned DROP PARTITION (b=2)");
+    verifyIfPartitionNotExist(dbName, "ptned", new ArrayList<>(Arrays.asList("1")));
+    verifyIfPartitionNotExist(dbName, "ptned", new ArrayList<>(Arrays.asList("2")));
+    verifySetup("SELECT a from " + dbName + ".ptned WHERE b=1", empty);
+    verifySetup("SELECT a from " + dbName + ".ptned WHERE b=2", empty);
+
+    advanceDumpDir();
+    run("REPL DUMP " + dbName + " FROM " + replDumpId);
+    String postDropReplDumpLocn = getResult(0,0);
+    String postDropReplDumpId = getResult(0,1,true);
+    LOG.info("Dumped to {} with id {}->{}", postDropReplDumpLocn, replDumpId, postDropReplDumpId);
+    assert(run("REPL LOAD " + dbName + "_dupe FROM '" + postDropReplDumpLocn + "'", true));
+
+    verifyIfPartitionNotExist(dbName + "_dupe", "ptned", new ArrayList<>(Arrays.asList("1")));
+    verifyIfPartitionNotExist(dbName + "_dupe", "ptned", new ArrayList<>(Arrays.asList("2")));
+    verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b=1", empty);
+    verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b=2", empty);
+  }
+
+  @Test
   public void testIncrementalAdds() throws IOException {
     String name = testName.getMethodName();
     String dbName = createDB(name);
@@ -599,30 +672,14 @@ public class TestReplicationScenarios {
     // not existing, and thus, throwing a NoSuchObjectException, or returning nulls
     // or select * returning empty, depending on what we're testing.
 
-    Exception e = null;
-    try {
-      Table tbl = metaStoreClient.getTable(dbName + "_dupe", "unptned");
-      assertNull(tbl);
-    } catch (TException te) {
-      e = te;
-    }
-    assertNotNull(e);
-    assertEquals(NoSuchObjectException.class, e.getClass());
+    verifyIfTableNotExist(dbName + "_dupe", "unptned");
 
     verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b='2'", empty);
     verifyRun("SELECT a from " + dbName + "_dupe.ptned", ptn_data_1);
     verifyRun("SELECT a from " + dbName + "_dupe.ptned3 WHERE b=1", empty);
     verifyRun("SELECT a from " + dbName + "_dupe.ptned3", ptn_data_2);
 
-    Exception e2 = null;
-    try {
-      Table tbl = metaStoreClient.getTable(dbName+"_dupe","ptned2");
-      assertNull(tbl);
-    } catch (TException te) {
-      e2 = te;
-    }
-    assertNotNull(e2);
-    assertEquals(NoSuchObjectException.class, e.getClass());
+    verifyIfTableNotExist(dbName + "_dupe", "ptned2");
   }
 
   @Test
@@ -732,15 +789,7 @@ public class TestReplicationScenarios {
     run("SELECT a from " + dbName + "_dupe.ptned");
     verifyResults(ptn_data_1);
 
-    Exception e2 = null;
-    try {
-      Table tbl = metaStoreClient.getTable(dbName+"_dupe","ptned2");
-      assertNull(tbl);
-    } catch (TException te) {
-      e2 = te;
-    }
-    assertNotNull(e2);
-    assertEquals(NoSuchObjectException.class, e.getClass());
+    verifyIfTableNotExist(dbName +"_dupe", "ptned2");
 
     run("SELECT a from " + dbName + "_dupe.unptned_copy");
     verifyResults(unptn_data);
@@ -874,15 +923,7 @@ public class TestReplicationScenarios {
     // Replication done, we now do the following verifications:
 
     // verify that unpartitioned table rename succeeded.
-    Exception e = null;
-    try {
-      Table tbl = metaStoreClient.getTable(dbName + "_dupe" , "unptned");
-      assertNull(tbl);
-    } catch (TException te) {
-      e = te;
-    }
-    assertNotNull(e);
-    assertEquals(NoSuchObjectException.class, e.getClass());
+    verifyIfTableNotExist(dbName + "_dupe", "unptned");
     verifyRun("SELECT * from " + dbName + "_dupe.unptned_rn", unptn_data);
 
     // verify that partition rename succeded.
@@ -898,15 +939,7 @@ public class TestReplicationScenarios {
     verifyRun("SELECT a from " + dbName + "_dupe.ptned WHERE b=22", ptn_data_2);
 
     // verify that ptned table rename succeded.
-    Exception e2 = null;
-    try {
-      Table tbl = metaStoreClient.getTable(dbName + "_dupe" , "ptned2");
-      assertNull(tbl);
-    } catch (TException te) {
-      e2 = te;
-    }
-    assertNotNull(e2);
-    assertEquals(NoSuchObjectException.class, e.getClass());
+    verifyIfTableNotExist(dbName + "_dupe", "ptned2");
     verifyRun("SELECT a from " + dbName + "_dupe.ptned2_rn WHERE b=2", ptn_data_2);
 
     // verify that ptned table property set worked
@@ -1970,6 +2003,50 @@ public class TestReplicationScenarios {
     }
   }
 
+  private void verifyIfTableNotExist(String dbName, String tableName){
+    Exception e = null;
+    try {
+      Table tbl = metaStoreClient.getTable(dbName, tableName);
+      assertNull(tbl);
+    } catch (TException te) {
+      e = te;
+    }
+    assertNotNull(e);
+    assertEquals(NoSuchObjectException.class, e.getClass());
+  }
+
+  private void verifyIfTableExist(String dbName, String tableName){
+    Exception e = null;
+    try {
+      Table tbl = metaStoreClient.getTable(dbName, tableName);
+      assertNotNull(tbl);
+    } catch (TException te) {
+      assert(false);
+    }
+  }
+
+  private void verifyIfPartitionNotExist(String dbName, String tableName, List<String> partValues){
+    Exception e = null;
+    try {
+      Partition ptn = metaStoreClient.getPartition(dbName, tableName, partValues);
+      assertNull(ptn);
+    } catch (TException te) {
+      e = te;
+    }
+    assertNotNull(e);
+    assertEquals(NoSuchObjectException.class, e.getClass());
+  }
+
+  private void verifyIfPartitionExist(String dbName, String tableName, List<String> partValues){
+    Exception e = null;
+    try {
+      Partition ptn = metaStoreClient.getPartition(dbName, tableName, partValues);
+      assertNotNull(ptn);
+    } catch (TException te) {
+      assert(false);
+    }
+  }
+
   private void verifySetup(String cmd, String[] data) throws  IOException {
     if (VERIFY_SETUP_STEPS){
       run(cmd);

http://git-wip-us.apache.org/repos/asf/hive/blob/3c1b354e/metastore/src/test/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
index a832c78..ed6d4be 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import java.util.List;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 
@@ -50,6 +51,8 @@ public class InjectableBehaviourObjectStore extends ObjectStore {
 
   private static com.google.common.base.Function<Table,Table> getTableModifier =
       com.google.common.base.Functions.identity();
+  private static com.google.common.base.Function<List<String>, List<String>> listPartitionNamesModifier =
+          com.google.common.base.Functions.identity();
 
   public static void setGetTableBehaviour(com.google.common.base.Function<Table,Table> modifier){
     getTableModifier = (modifier == null)? com.google.common.base.Functions.identity() : modifier;
@@ -63,8 +66,25 @@ public class InjectableBehaviourObjectStore extends ObjectStore {
     return getTableModifier;
   }
 
+  public static void setListPartitionNamesBehaviour(com.google.common.base.Function<List<String>, List<String>> modifier){
+    listPartitionNamesModifier = (modifier == null)? com.google.common.base.Functions.identity() : modifier;
+  }
+
+  public static void resetListPartitionNamesBehaviour(){
+    setListPartitionNamesBehaviour(null);
+  }
+
+  public static com.google.common.base.Function<List<String>, List<String>> getListPartitionNamesBehaviour() {
+    return listPartitionNamesModifier;
+  }
+
   @Override
   public Table getTable(String dbName, String tableName) throws MetaException {
     return getTableModifier.apply(super.getTable(dbName, tableName));
   }
+
+  @Override
+  public List<String> listPartitionNames(String dbName, String tableName, short max) throws MetaException {
+    return listPartitionNamesModifier.apply(super.listPartitionNames(dbName, tableName, max));
+  }
 }


[03/31] hive git commit: HIVE-16613: SaslClientHandler.sendHello is eating exceptions (Rui reviewed by Xuefu)

Posted by we...@apache.org.
HIVE-16613: SaslClientHandler.sendHello is eating exceptions (Rui reviewed by Xuefu)


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

Branch: refs/heads/hive-14535
Commit: 8a093a7d24705e317a84f0af3fa1e4e5ad941b98
Parents: ef6a90a
Author: Rui Li <li...@apache.org>
Authored: Thu May 25 11:00:46 2017 +0800
Committer: Rui Li <li...@apache.org>
Committed: Thu May 25 11:00:46 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hive/spark/client/rpc/Rpc.java   | 7 ++++++-
 .../java/org/apache/hive/spark/client/rpc/SaslHandler.java    | 2 +-
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8a093a7d/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java
index 680e6b8..cbbfb1c 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java
@@ -491,7 +491,12 @@ public class Rpc implements Closeable {
     void sendHello(Channel c) throws Exception {
       byte[] hello = client.hasInitialResponse() ?
         client.evaluateChallenge(new byte[0]) : new byte[0];
-      c.writeAndFlush(new SaslMessage(clientId, hello));
+      c.writeAndFlush(new SaslMessage(clientId, hello)).addListener(future -> {
+        if (!future.isSuccess()) {
+          LOG.error("Failed to send hello to server", future.cause());
+          onError(future.cause());
+        }
+      });
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/8a093a7d/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java
index 26edc63..8a42773 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java
@@ -36,7 +36,7 @@ abstract class SaslHandler extends SimpleChannelInboundHandler<Rpc.SaslMessage>
 
   // LOG is not static to make debugging easier (being able to identify which sub-class
   // generated the log message).
-  private final Logger LOG;
+  protected final Logger LOG;
   private final boolean requiresEncryption;
   private KryoMessageCodec kryo;
   private boolean hasAuthResponse = false;


[23/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index e3725a5..1e46ac4 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -990,30 +990,43 @@ void SQLForeignKey::printTo(std::ostream& out) const {
 }
 
 
-Type::~Type() throw() {
+SQLUniqueConstraint::~SQLUniqueConstraint() throw() {
 }
 
 
-void Type::__set_name(const std::string& val) {
-  this->name = val;
+void SQLUniqueConstraint::__set_table_db(const std::string& val) {
+  this->table_db = val;
 }
 
-void Type::__set_type1(const std::string& val) {
-  this->type1 = val;
-__isset.type1 = true;
+void SQLUniqueConstraint::__set_table_name(const std::string& val) {
+  this->table_name = val;
 }
 
-void Type::__set_type2(const std::string& val) {
-  this->type2 = val;
-__isset.type2 = true;
+void SQLUniqueConstraint::__set_column_name(const std::string& val) {
+  this->column_name = val;
 }
 
-void Type::__set_fields(const std::vector<FieldSchema> & val) {
-  this->fields = val;
-__isset.fields = true;
+void SQLUniqueConstraint::__set_key_seq(const int32_t val) {
+  this->key_seq = val;
 }
 
-uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
+void SQLUniqueConstraint::__set_uk_name(const std::string& val) {
+  this->uk_name = val;
+}
+
+void SQLUniqueConstraint::__set_enable_cstr(const bool val) {
+  this->enable_cstr = val;
+}
+
+void SQLUniqueConstraint::__set_validate_cstr(const bool val) {
+  this->validate_cstr = val;
+}
+
+void SQLUniqueConstraint::__set_rely_cstr(const bool val) {
+  this->rely_cstr = val;
+}
+
+uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1036,44 +1049,64 @@ uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
+          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->type1);
-          this->__isset.type1 = true;
+          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->type2);
-          this->__isset.type2 = true;
+          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_LIST) {
-          {
-            this->fields.clear();
-            uint32_t _size8;
-            ::apache::thrift::protocol::TType _etype11;
-            xfer += iprot->readListBegin(_etype11, _size8);
-            this->fields.resize(_size8);
-            uint32_t _i12;
-            for (_i12 = 0; _i12 < _size8; ++_i12)
-            {
-              xfer += this->fields[_i12].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.fields = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->key_seq);
+          this->__isset.key_seq = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->uk_name);
+          this->__isset.uk_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);
         }
@@ -1090,103 +1123,132 @@ uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t SQLUniqueConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("Type");
+  xfer += oprot->writeStructBegin("SQLUniqueConstraint");
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->name);
+  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("key_seq", ::apache::thrift::protocol::T_I32, 4);
+  xfer += oprot->writeI32(this->key_seq);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("uk_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->uk_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();
 
-  if (this->__isset.type1) {
-    xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2);
-    xfer += oprot->writeString(this->type1);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.type2) {
-    xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3);
-    xfer += oprot->writeString(this->type2);
-    xfer += oprot->writeFieldEnd();
-  }
-  if (this->__isset.fields) {
-    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 _iter13;
-      for (_iter13 = this->fields.begin(); _iter13 != this->fields.end(); ++_iter13)
-      {
-        xfer += (*_iter13).write(oprot);
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(Type &a, Type &b) {
+void swap(SQLUniqueConstraint &a, SQLUniqueConstraint &b) {
   using ::std::swap;
-  swap(a.name, b.name);
-  swap(a.type1, b.type1);
-  swap(a.type2, b.type2);
-  swap(a.fields, b.fields);
+  swap(a.table_db, b.table_db);
+  swap(a.table_name, b.table_name);
+  swap(a.column_name, b.column_name);
+  swap(a.key_seq, b.key_seq);
+  swap(a.uk_name, b.uk_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);
 }
 
-Type::Type(const Type& other14) {
-  name = other14.name;
-  type1 = other14.type1;
-  type2 = other14.type2;
-  fields = other14.fields;
-  __isset = other14.__isset;
-}
-Type& Type::operator=(const Type& other15) {
-  name = other15.name;
-  type1 = other15.type1;
-  type2 = other15.type2;
-  fields = other15.fields;
-  __isset = other15.__isset;
+SQLUniqueConstraint::SQLUniqueConstraint(const SQLUniqueConstraint& other8) {
+  table_db = other8.table_db;
+  table_name = other8.table_name;
+  column_name = other8.column_name;
+  key_seq = other8.key_seq;
+  uk_name = other8.uk_name;
+  enable_cstr = other8.enable_cstr;
+  validate_cstr = other8.validate_cstr;
+  rely_cstr = other8.rely_cstr;
+  __isset = other8.__isset;
+}
+SQLUniqueConstraint& SQLUniqueConstraint::operator=(const SQLUniqueConstraint& other9) {
+  table_db = other9.table_db;
+  table_name = other9.table_name;
+  column_name = other9.column_name;
+  key_seq = other9.key_seq;
+  uk_name = other9.uk_name;
+  enable_cstr = other9.enable_cstr;
+  validate_cstr = other9.validate_cstr;
+  rely_cstr = other9.rely_cstr;
+  __isset = other9.__isset;
   return *this;
 }
-void Type::printTo(std::ostream& out) const {
+void SQLUniqueConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "Type(";
-  out << "name=" << to_string(name);
-  out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "<null>"));
-  out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "<null>"));
-  out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "<null>"));
+  out << "SQLUniqueConstraint(";
+  out << "table_db=" << to_string(table_db);
+  out << ", " << "table_name=" << to_string(table_name);
+  out << ", " << "column_name=" << to_string(column_name);
+  out << ", " << "key_seq=" << to_string(key_seq);
+  out << ", " << "uk_name=" << to_string(uk_name);
+  out << ", " << "enable_cstr=" << to_string(enable_cstr);
+  out << ", " << "validate_cstr=" << to_string(validate_cstr);
+  out << ", " << "rely_cstr=" << to_string(rely_cstr);
   out << ")";
 }
 
 
-HiveObjectRef::~HiveObjectRef() throw() {
+SQLNotNullConstraint::~SQLNotNullConstraint() throw() {
 }
 
 
-void HiveObjectRef::__set_objectType(const HiveObjectType::type val) {
-  this->objectType = val;
+void SQLNotNullConstraint::__set_table_db(const std::string& val) {
+  this->table_db = val;
 }
 
-void HiveObjectRef::__set_dbName(const std::string& val) {
-  this->dbName = val;
+void SQLNotNullConstraint::__set_table_name(const std::string& val) {
+  this->table_name = val;
 }
 
-void HiveObjectRef::__set_objectName(const std::string& val) {
-  this->objectName = val;
+void SQLNotNullConstraint::__set_column_name(const std::string& val) {
+  this->column_name = val;
 }
 
-void HiveObjectRef::__set_partValues(const std::vector<std::string> & val) {
-  this->partValues = val;
+void SQLNotNullConstraint::__set_nn_name(const std::string& val) {
+  this->nn_name = val;
 }
 
-void HiveObjectRef::__set_columnName(const std::string& val) {
-  this->columnName = val;
+void SQLNotNullConstraint::__set_enable_cstr(const bool val) {
+  this->enable_cstr = val;
 }
 
-uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
+void SQLNotNullConstraint::__set_validate_cstr(const bool val) {
+  this->validate_cstr = val;
+}
+
+void SQLNotNullConstraint::__set_rely_cstr(const bool val) {
+  this->rely_cstr = val;
+}
+
+uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1208,55 +1270,57 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast16;
-          xfer += iprot->readI32(ecast16);
-          this->objectType = (HiveObjectType::type)ecast16;
-          this->__isset.objectType = true;
+        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->dbName);
-          this->__isset.dbName = true;
+          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->objectName);
-          this->__isset.objectName = true;
+          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_LIST) {
-          {
-            this->partValues.clear();
-            uint32_t _size17;
-            ::apache::thrift::protocol::TType _etype20;
-            xfer += iprot->readListBegin(_etype20, _size17);
-            this->partValues.resize(_size17);
-            uint32_t _i21;
-            for (_i21 = 0; _i21 < _size17; ++_i21)
-            {
-              xfer += iprot->readString(this->partValues[_i21]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.partValues = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->nn_name);
+          this->__isset.nn_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 5:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->columnName);
-          this->__isset.columnName = true;
+        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 6:
+        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 7:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rely_cstr);
+          this->__isset.rely_cstr = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1273,37 +1337,37 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t SQLNotNullConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveObjectRef");
+  xfer += oprot->writeStructBegin("SQLNotNullConstraint");
 
-  xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32((int32_t)this->objectType);
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->table_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->table_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->objectName);
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->column_name);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter22;
-    for (_iter22 = this->partValues.begin(); _iter22 != this->partValues.end(); ++_iter22)
-    {
-      xfer += oprot->writeString((*_iter22));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("nn_name", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeString(this->nn_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5);
-  xfer += oprot->writeString(this->columnName);
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 5);
+  xfer += oprot->writeBool(this->enable_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeBool(this->validate_cstr);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1311,70 +1375,77 @@ uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) cons
   return xfer;
 }
 
-void swap(HiveObjectRef &a, HiveObjectRef &b) {
+void swap(SQLNotNullConstraint &a, SQLNotNullConstraint &b) {
   using ::std::swap;
-  swap(a.objectType, b.objectType);
-  swap(a.dbName, b.dbName);
-  swap(a.objectName, b.objectName);
-  swap(a.partValues, b.partValues);
-  swap(a.columnName, b.columnName);
+  swap(a.table_db, b.table_db);
+  swap(a.table_name, b.table_name);
+  swap(a.column_name, b.column_name);
+  swap(a.nn_name, b.nn_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);
 }
 
-HiveObjectRef::HiveObjectRef(const HiveObjectRef& other23) {
-  objectType = other23.objectType;
-  dbName = other23.dbName;
-  objectName = other23.objectName;
-  partValues = other23.partValues;
-  columnName = other23.columnName;
-  __isset = other23.__isset;
-}
-HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other24) {
-  objectType = other24.objectType;
-  dbName = other24.dbName;
-  objectName = other24.objectName;
-  partValues = other24.partValues;
-  columnName = other24.columnName;
-  __isset = other24.__isset;
+SQLNotNullConstraint::SQLNotNullConstraint(const SQLNotNullConstraint& other10) {
+  table_db = other10.table_db;
+  table_name = other10.table_name;
+  column_name = other10.column_name;
+  nn_name = other10.nn_name;
+  enable_cstr = other10.enable_cstr;
+  validate_cstr = other10.validate_cstr;
+  rely_cstr = other10.rely_cstr;
+  __isset = other10.__isset;
+}
+SQLNotNullConstraint& SQLNotNullConstraint::operator=(const SQLNotNullConstraint& other11) {
+  table_db = other11.table_db;
+  table_name = other11.table_name;
+  column_name = other11.column_name;
+  nn_name = other11.nn_name;
+  enable_cstr = other11.enable_cstr;
+  validate_cstr = other11.validate_cstr;
+  rely_cstr = other11.rely_cstr;
+  __isset = other11.__isset;
   return *this;
 }
-void HiveObjectRef::printTo(std::ostream& out) const {
+void SQLNotNullConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "HiveObjectRef(";
-  out << "objectType=" << to_string(objectType);
-  out << ", " << "dbName=" << to_string(dbName);
-  out << ", " << "objectName=" << to_string(objectName);
-  out << ", " << "partValues=" << to_string(partValues);
-  out << ", " << "columnName=" << to_string(columnName);
+  out << "SQLNotNullConstraint(";
+  out << "table_db=" << to_string(table_db);
+  out << ", " << "table_name=" << to_string(table_name);
+  out << ", " << "column_name=" << to_string(column_name);
+  out << ", " << "nn_name=" << to_string(nn_name);
+  out << ", " << "enable_cstr=" << to_string(enable_cstr);
+  out << ", " << "validate_cstr=" << to_string(validate_cstr);
+  out << ", " << "rely_cstr=" << to_string(rely_cstr);
   out << ")";
 }
 
 
-PrivilegeGrantInfo::~PrivilegeGrantInfo() throw() {
+Type::~Type() throw() {
 }
 
 
-void PrivilegeGrantInfo::__set_privilege(const std::string& val) {
-  this->privilege = val;
-}
-
-void PrivilegeGrantInfo::__set_createTime(const int32_t val) {
-  this->createTime = val;
+void Type::__set_name(const std::string& val) {
+  this->name = val;
 }
 
-void PrivilegeGrantInfo::__set_grantor(const std::string& val) {
-  this->grantor = val;
+void Type::__set_type1(const std::string& val) {
+  this->type1 = val;
+__isset.type1 = true;
 }
 
-void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) {
-  this->grantorType = val;
+void Type::__set_type2(const std::string& val) {
+  this->type2 = val;
+__isset.type2 = true;
 }
 
-void PrivilegeGrantInfo::__set_grantOption(const bool val) {
-  this->grantOption = val;
+void Type::__set_fields(const std::vector<FieldSchema> & val) {
+  this->fields = val;
+__isset.fields = true;
 }
 
-uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t Type::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1397,42 +1468,44 @@ uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot)
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->privilege);
-          this->__isset.privilege = true;
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->createTime);
-          this->__isset.createTime = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->type1);
+          this->__isset.type1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->grantor);
-          this->__isset.grantor = true;
+          xfer += iprot->readString(this->type2);
+          this->__isset.type2 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast25;
-          xfer += iprot->readI32(ecast25);
-          this->grantorType = (PrincipalType::type)ecast25;
-          this->__isset.grantorType = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 5:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->grantOption);
-          this->__isset.grantOption = true;
+        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)
+            {
+              xfer += this->fields[_i16].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.fields = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1449,96 +1522,103 @@ uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot)
   return xfer;
 }
 
-uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrivilegeGrantInfo");
-
-  xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->privilege);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2);
-  xfer += oprot->writeI32(this->createTime);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->grantor);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4);
-  xfer += oprot->writeI32((int32_t)this->grantorType);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("Type");
 
-  xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5);
-  xfer += oprot->writeBool(this->grantOption);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.type1) {
+    xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2);
+    xfer += oprot->writeString(this->type1);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.type2) {
+    xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3);
+    xfer += oprot->writeString(this->type2);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.fields) {
+    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)
+      {
+        xfer += (*_iter17).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) {
+void swap(Type &a, Type &b) {
   using ::std::swap;
-  swap(a.privilege, b.privilege);
-  swap(a.createTime, b.createTime);
-  swap(a.grantor, b.grantor);
-  swap(a.grantorType, b.grantorType);
-  swap(a.grantOption, b.grantOption);
+  swap(a.name, b.name);
+  swap(a.type1, b.type1);
+  swap(a.type2, b.type2);
+  swap(a.fields, b.fields);
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other26) {
-  privilege = other26.privilege;
-  createTime = other26.createTime;
-  grantor = other26.grantor;
-  grantorType = other26.grantorType;
-  grantOption = other26.grantOption;
-  __isset = other26.__isset;
-}
-PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other27) {
-  privilege = other27.privilege;
-  createTime = other27.createTime;
-  grantor = other27.grantor;
-  grantorType = other27.grantorType;
-  grantOption = other27.grantOption;
-  __isset = other27.__isset;
+Type::Type(const Type& other18) {
+  name = other18.name;
+  type1 = other18.type1;
+  type2 = other18.type2;
+  fields = other18.fields;
+  __isset = other18.__isset;
+}
+Type& Type::operator=(const Type& other19) {
+  name = other19.name;
+  type1 = other19.type1;
+  type2 = other19.type2;
+  fields = other19.fields;
+  __isset = other19.__isset;
   return *this;
 }
-void PrivilegeGrantInfo::printTo(std::ostream& out) const {
+void Type::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrivilegeGrantInfo(";
-  out << "privilege=" << to_string(privilege);
-  out << ", " << "createTime=" << to_string(createTime);
-  out << ", " << "grantor=" << to_string(grantor);
-  out << ", " << "grantorType=" << to_string(grantorType);
-  out << ", " << "grantOption=" << to_string(grantOption);
+  out << "Type(";
+  out << "name=" << to_string(name);
+  out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "<null>"));
+  out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "<null>"));
+  out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "<null>"));
   out << ")";
 }
 
 
-HiveObjectPrivilege::~HiveObjectPrivilege() throw() {
+HiveObjectRef::~HiveObjectRef() throw() {
 }
 
 
-void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) {
-  this->hiveObject = val;
+void HiveObjectRef::__set_objectType(const HiveObjectType::type val) {
+  this->objectType = val;
 }
 
-void HiveObjectPrivilege::__set_principalName(const std::string& val) {
-  this->principalName = val;
+void HiveObjectRef::__set_dbName(const std::string& val) {
+  this->dbName = val;
 }
 
-void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) {
-  this->principalType = val;
+void HiveObjectRef::__set_objectName(const std::string& val) {
+  this->objectName = val;
 }
 
-void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) {
-  this->grantInfo = val;
+void HiveObjectRef::__set_partValues(const std::vector<std::string> & val) {
+  this->partValues = val;
 }
 
-uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot) {
+void HiveObjectRef::__set_columnName(const std::string& val) {
+  this->columnName = val;
+}
+
+uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1560,35 +1640,55 @@ uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot)
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->hiveObject.read(iprot);
-          this->__isset.hiveObject = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast20;
+          xfer += iprot->readI32(ecast20);
+          this->objectType = (HiveObjectType::type)ecast20;
+          this->__isset.objectType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->principalName);
-          this->__isset.principalName = true;
+          xfer += iprot->readString(this->dbName);
+          this->__isset.dbName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast28;
-          xfer += iprot->readI32(ecast28);
-          this->principalType = (PrincipalType::type)ecast28;
-          this->__isset.principalType = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->objectName);
+          this->__isset.objectName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->grantInfo.read(iprot);
-          this->__isset.grantInfo = true;
+        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)
+            {
+              xfer += iprot->readString(this->partValues[_i25]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.partValues = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->columnName);
+          this->__isset.columnName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1605,25 +1705,37 @@ uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot)
   return xfer;
 }
 
-uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("HiveObjectPrivilege");
+  xfer += oprot->writeStructBegin("HiveObjectRef");
 
-  xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->hiveObject.write(oprot);
+  xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1);
+  xfer += oprot->writeI32((int32_t)this->objectType);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->principalName);
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->dbName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3);
-  xfer += oprot->writeI32((int32_t)this->principalType);
+  xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->objectName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += this->grantInfo.write(oprot);
+  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)
+    {
+      xfer += oprot->writeString((*_iter26));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeString(this->columnName);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1631,50 +1743,70 @@ uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot
   return xfer;
 }
 
-void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) {
+void swap(HiveObjectRef &a, HiveObjectRef &b) {
   using ::std::swap;
-  swap(a.hiveObject, b.hiveObject);
-  swap(a.principalName, b.principalName);
-  swap(a.principalType, b.principalType);
-  swap(a.grantInfo, b.grantInfo);
+  swap(a.objectType, b.objectType);
+  swap(a.dbName, b.dbName);
+  swap(a.objectName, b.objectName);
+  swap(a.partValues, b.partValues);
+  swap(a.columnName, b.columnName);
   swap(a.__isset, b.__isset);
 }
 
-HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other29) {
-  hiveObject = other29.hiveObject;
-  principalName = other29.principalName;
-  principalType = other29.principalType;
-  grantInfo = other29.grantInfo;
-  __isset = other29.__isset;
+HiveObjectRef::HiveObjectRef(const HiveObjectRef& other27) {
+  objectType = other27.objectType;
+  dbName = other27.dbName;
+  objectName = other27.objectName;
+  partValues = other27.partValues;
+  columnName = other27.columnName;
+  __isset = other27.__isset;
 }
-HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other30) {
-  hiveObject = other30.hiveObject;
-  principalName = other30.principalName;
-  principalType = other30.principalType;
-  grantInfo = other30.grantInfo;
-  __isset = other30.__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;
   return *this;
 }
-void HiveObjectPrivilege::printTo(std::ostream& out) const {
+void HiveObjectRef::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "HiveObjectPrivilege(";
-  out << "hiveObject=" << to_string(hiveObject);
-  out << ", " << "principalName=" << to_string(principalName);
-  out << ", " << "principalType=" << to_string(principalType);
-  out << ", " << "grantInfo=" << to_string(grantInfo);
+  out << "HiveObjectRef(";
+  out << "objectType=" << to_string(objectType);
+  out << ", " << "dbName=" << to_string(dbName);
+  out << ", " << "objectName=" << to_string(objectName);
+  out << ", " << "partValues=" << to_string(partValues);
+  out << ", " << "columnName=" << to_string(columnName);
   out << ")";
 }
 
 
-PrivilegeBag::~PrivilegeBag() throw() {
+PrivilegeGrantInfo::~PrivilegeGrantInfo() throw() {
 }
 
 
-void PrivilegeBag::__set_privileges(const std::vector<HiveObjectPrivilege> & val) {
-  this->privileges = val;
+void PrivilegeGrantInfo::__set_privilege(const std::string& val) {
+  this->privilege = val;
 }
 
-uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
+void PrivilegeGrantInfo::__set_createTime(const int32_t val) {
+  this->createTime = val;
+}
+
+void PrivilegeGrantInfo::__set_grantor(const std::string& val) {
+  this->grantor = val;
+}
+
+void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) {
+  this->grantorType = val;
+}
+
+void PrivilegeGrantInfo::__set_grantOption(const bool val) {
+  this->grantOption = val;
+}
+
+uint32_t PrivilegeGrantInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1696,21 +1828,43 @@ uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->privileges.clear();
-            uint32_t _size31;
-            ::apache::thrift::protocol::TType _etype34;
-            xfer += iprot->readListBegin(_etype34, _size31);
-            this->privileges.resize(_size31);
-            uint32_t _i35;
-            for (_i35 = 0; _i35 < _size31; ++_i35)
-            {
-              xfer += this->privileges[_i35].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.privileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->privilege);
+          this->__isset.privilege = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->createTime);
+          this->__isset.createTime = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->grantor);
+          this->__isset.grantor = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast29;
+          xfer += iprot->readI32(ecast29);
+          this->grantorType = (PrincipalType::type)ecast29;
+          this->__isset.grantorType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->grantOption);
+          this->__isset.grantOption = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1727,21 +1881,29 @@ uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
   return xfer;
 }
 
-uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrivilegeBag");
+  xfer += oprot->writeStructBegin("PrivilegeGrantInfo");
 
-  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 _iter36;
-    for (_iter36 = this->privileges.begin(); _iter36 != this->privileges.end(); ++_iter36)
-    {
-      xfer += (*_iter36).write(oprot);
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->privilege);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2);
+  xfer += oprot->writeI32(this->createTime);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->grantor);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4);
+  xfer += oprot->writeI32((int32_t)this->grantorType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5);
+  xfer += oprot->writeBool(this->grantOption);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1749,46 +1911,66 @@ uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const
   return xfer;
 }
 
-void swap(PrivilegeBag &a, PrivilegeBag &b) {
+void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) {
   using ::std::swap;
-  swap(a.privileges, b.privileges);
+  swap(a.privilege, b.privilege);
+  swap(a.createTime, b.createTime);
+  swap(a.grantor, b.grantor);
+  swap(a.grantorType, b.grantorType);
+  swap(a.grantOption, b.grantOption);
   swap(a.__isset, b.__isset);
 }
 
-PrivilegeBag::PrivilegeBag(const PrivilegeBag& other37) {
-  privileges = other37.privileges;
-  __isset = other37.__isset;
+PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other30) {
+  privilege = other30.privilege;
+  createTime = other30.createTime;
+  grantor = other30.grantor;
+  grantorType = other30.grantorType;
+  grantOption = other30.grantOption;
+  __isset = other30.__isset;
 }
-PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other38) {
-  privileges = other38.privileges;
-  __isset = other38.__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;
   return *this;
 }
-void PrivilegeBag::printTo(std::ostream& out) const {
+void PrivilegeGrantInfo::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrivilegeBag(";
-  out << "privileges=" << to_string(privileges);
+  out << "PrivilegeGrantInfo(";
+  out << "privilege=" << to_string(privilege);
+  out << ", " << "createTime=" << to_string(createTime);
+  out << ", " << "grantor=" << to_string(grantor);
+  out << ", " << "grantorType=" << to_string(grantorType);
+  out << ", " << "grantOption=" << to_string(grantOption);
   out << ")";
 }
 
 
-PrincipalPrivilegeSet::~PrincipalPrivilegeSet() throw() {
+HiveObjectPrivilege::~HiveObjectPrivilege() throw() {
 }
 
 
-void PrincipalPrivilegeSet::__set_userPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->userPrivileges = val;
+void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) {
+  this->hiveObject = val;
 }
 
-void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->groupPrivileges = val;
+void HiveObjectPrivilege::__set_principalName(const std::string& val) {
+  this->principalName = val;
 }
 
-void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
-  this->rolePrivileges = val;
+void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) {
+  this->principalType = val;
 }
 
-uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* iprot) {
+void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) {
+  this->grantInfo = val;
+}
+
+uint32_t HiveObjectPrivilege::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1810,106 +1992,35 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->userPrivileges.clear();
-            uint32_t _size39;
-            ::apache::thrift::protocol::TType _ktype40;
-            ::apache::thrift::protocol::TType _vtype41;
-            xfer += iprot->readMapBegin(_ktype40, _vtype41, _size39);
-            uint32_t _i43;
-            for (_i43 = 0; _i43 < _size39; ++_i43)
-            {
-              std::string _key44;
-              xfer += iprot->readString(_key44);
-              std::vector<PrivilegeGrantInfo> & _val45 = this->userPrivileges[_key44];
-              {
-                _val45.clear();
-                uint32_t _size46;
-                ::apache::thrift::protocol::TType _etype49;
-                xfer += iprot->readListBegin(_etype49, _size46);
-                _val45.resize(_size46);
-                uint32_t _i50;
-                for (_i50 = 0; _i50 < _size46; ++_i50)
-                {
-                  xfer += _val45[_i50].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.userPrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->hiveObject.read(iprot);
+          this->__isset.hiveObject = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->groupPrivileges.clear();
-            uint32_t _size51;
-            ::apache::thrift::protocol::TType _ktype52;
-            ::apache::thrift::protocol::TType _vtype53;
-            xfer += iprot->readMapBegin(_ktype52, _vtype53, _size51);
-            uint32_t _i55;
-            for (_i55 = 0; _i55 < _size51; ++_i55)
-            {
-              std::string _key56;
-              xfer += iprot->readString(_key56);
-              std::vector<PrivilegeGrantInfo> & _val57 = this->groupPrivileges[_key56];
-              {
-                _val57.clear();
-                uint32_t _size58;
-                ::apache::thrift::protocol::TType _etype61;
-                xfer += iprot->readListBegin(_etype61, _size58);
-                _val57.resize(_size58);
-                uint32_t _i62;
-                for (_i62 = 0; _i62 < _size58; ++_i62)
-                {
-                  xfer += _val57[_i62].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.groupPrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->principalName);
+          this->__isset.principalName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->rolePrivileges.clear();
-            uint32_t _size63;
-            ::apache::thrift::protocol::TType _ktype64;
-            ::apache::thrift::protocol::TType _vtype65;
-            xfer += iprot->readMapBegin(_ktype64, _vtype65, _size63);
-            uint32_t _i67;
-            for (_i67 = 0; _i67 < _size63; ++_i67)
-            {
-              std::string _key68;
-              xfer += iprot->readString(_key68);
-              std::vector<PrivilegeGrantInfo> & _val69 = this->rolePrivileges[_key68];
-              {
-                _val69.clear();
-                uint32_t _size70;
-                ::apache::thrift::protocol::TType _etype73;
-                xfer += iprot->readListBegin(_etype73, _size70);
-                _val69.resize(_size70);
-                uint32_t _i74;
-                for (_i74 = 0; _i74 < _size70; ++_i74)
-                {
-                  xfer += _val69[_i74].read(iprot);
-                }
-                xfer += iprot->readListEnd();
-              }
-            }
-            xfer += iprot->readMapEnd();
-          }
-          this->__isset.rolePrivileges = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast32;
+          xfer += iprot->readI32(ecast32);
+          this->principalType = (PrincipalType::type)ecast32;
+          this->__isset.principalType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->grantInfo.read(iprot);
+          this->__isset.grantInfo = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1926,72 +2037,25 @@ uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* ipro
   return xfer;
 }
 
-uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("PrincipalPrivilegeSet");
+  xfer += oprot->writeStructBegin("HiveObjectPrivilege");
 
-  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 _iter75;
-    for (_iter75 = this->userPrivileges.begin(); _iter75 != this->userPrivileges.end(); ++_iter75)
-    {
-      xfer += oprot->writeString(_iter75->first);
-      {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter75->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter76;
-        for (_iter76 = _iter75->second.begin(); _iter76 != _iter75->second.end(); ++_iter76)
-        {
-          xfer += (*_iter76).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->hiveObject.write(oprot);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter77;
-    for (_iter77 = this->groupPrivileges.begin(); _iter77 != this->groupPrivileges.end(); ++_iter77)
-    {
-      xfer += oprot->writeString(_iter77->first);
-      {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter77->second.size()));
-        std::vector<PrivilegeGrantInfo> ::const_iterator _iter78;
-        for (_iter78 = _iter77->second.begin(); _iter78 != _iter77->second.end(); ++_iter78)
-        {
-          xfer += (*_iter78).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->principalName);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter79;
-    for (_iter79 = this->rolePrivileges.begin(); _iter79 != this->rolePrivileges.end(); ++_iter79)
-    {
-      xfer += oprot->writeString(_iter79->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 += (*_iter80).write(oprot);
-        }
-        xfer += oprot->writeListEnd();
-      }
-    }
-    xfer += oprot->writeMapEnd();
-  }
+  xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3);
+  xfer += oprot->writeI32((int32_t)this->principalType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += this->grantInfo.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1999,55 +2063,50 @@ uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* opr
   return xfer;
 }
 
-void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) {
+void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) {
   using ::std::swap;
-  swap(a.userPrivileges, b.userPrivileges);
-  swap(a.groupPrivileges, b.groupPrivileges);
-  swap(a.rolePrivileges, b.rolePrivileges);
+  swap(a.hiveObject, b.hiveObject);
+  swap(a.principalName, b.principalName);
+  swap(a.principalType, b.principalType);
+  swap(a.grantInfo, b.grantInfo);
   swap(a.__isset, b.__isset);
 }
 
-PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other81) {
-  userPrivileges = other81.userPrivileges;
-  groupPrivileges = other81.groupPrivileges;
-  rolePrivileges = other81.rolePrivileges;
-  __isset = other81.__isset;
+HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other33) {
+  hiveObject = other33.hiveObject;
+  principalName = other33.principalName;
+  principalType = other33.principalType;
+  grantInfo = other33.grantInfo;
+  __isset = other33.__isset;
 }
-PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other82) {
-  userPrivileges = other82.userPrivileges;
-  groupPrivileges = other82.groupPrivileges;
-  rolePrivileges = other82.rolePrivileges;
-  __isset = other82.__isset;
+HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other34) {
+  hiveObject = other34.hiveObject;
+  principalName = other34.principalName;
+  principalType = other34.principalType;
+  grantInfo = other34.grantInfo;
+  __isset = other34.__isset;
   return *this;
 }
-void PrincipalPrivilegeSet::printTo(std::ostream& out) const {
+void HiveObjectPrivilege::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "PrincipalPrivilegeSet(";
-  out << "userPrivileges=" << to_string(userPrivileges);
-  out << ", " << "groupPrivileges=" << to_string(groupPrivileges);
-  out << ", " << "rolePrivileges=" << to_string(rolePrivileges);
+  out << "HiveObjectPrivilege(";
+  out << "hiveObject=" << to_string(hiveObject);
+  out << ", " << "principalName=" << to_string(principalName);
+  out << ", " << "principalType=" << to_string(principalType);
+  out << ", " << "grantInfo=" << to_string(grantInfo);
   out << ")";
 }
 
 
-GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() throw() {
+PrivilegeBag::~PrivilegeBag() throw() {
 }
 
 
-void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) {
-  this->requestType = val;
-}
-
-void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) {
+void PrivilegeBag::__set_privileges(const std::vector<HiveObjectPrivilege> & val) {
   this->privileges = val;
 }
 
-void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) {
-  this->revokeGrantOption = val;
-__isset.revokeGrantOption = true;
-}
-
-uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t PrivilegeBag::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2069,31 +2128,25 @@ uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast83;
-          xfer += iprot->readI32(ecast83);
-          this->requestType = (GrantRevokeType::type)ecast83;
-          this->__isset.requestType = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->privileges.read(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)
+            {
+              xfer += this->privileges[_i39].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
           this->__isset.privileges = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->revokeGrantOption);
-          this->__isset.revokeGrantOption = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2106,70 +2159,68 @@ uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol
   return xfer;
 }
 
-uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest");
-
-  xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1);
-  xfer += oprot->writeI32((int32_t)this->requestType);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("PrivilegeBag");
 
-  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2);
-  xfer += this->privileges.write(oprot);
+  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)
+    {
+      xfer += (*_iter40).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.revokeGrantOption) {
-    xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3);
-    xfer += oprot->writeBool(this->revokeGrantOption);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) {
+void swap(PrivilegeBag &a, PrivilegeBag &b) {
   using ::std::swap;
-  swap(a.requestType, b.requestType);
   swap(a.privileges, b.privileges);
-  swap(a.revokeGrantOption, b.revokeGrantOption);
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other84) {
-  requestType = other84.requestType;
-  privileges = other84.privileges;
-  revokeGrantOption = other84.revokeGrantOption;
-  __isset = other84.__isset;
+PrivilegeBag::PrivilegeBag(const PrivilegeBag& other41) {
+  privileges = other41.privileges;
+  __isset = other41.__isset;
 }
-GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other85) {
-  requestType = other85.requestType;
-  privileges = other85.privileges;
-  revokeGrantOption = other85.revokeGrantOption;
-  __isset = other85.__isset;
+PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other42) {
+  privileges = other42.privileges;
+  __isset = other42.__isset;
   return *this;
 }
-void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const {
+void PrivilegeBag::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "GrantRevokePrivilegeRequest(";
-  out << "requestType=" << to_string(requestType);
-  out << ", " << "privileges=" << to_string(privileges);
-  out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "<null>"));
+  out << "PrivilegeBag(";
+  out << "privileges=" << to_string(privileges);
   out << ")";
 }
 
 
-GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() throw() {
+PrincipalPrivilegeSet::~PrincipalPrivilegeSet() throw() {
 }
 
 
-void GrantRevokePrivilegeResponse::__set_success(const bool val) {
-  this->success = val;
-__isset.success = true;
+void PrincipalPrivilegeSet::__set_userPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->userPrivileges = val;
 }
 
-uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->groupPrivileges = val;
+}
+
+void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map<std::string, std::vector<PrivilegeGrantInfo> > & val) {
+  this->rolePrivileges = val;
+}
+
+uint32_t PrincipalPrivilegeSet::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2191,9 +2242,106 @@ uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtoco
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->success);
-          this->__isset.success = true;
+        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)
+            {
+              std::string _key48;
+              xfer += iprot->readString(_key48);
+              std::vector<PrivilegeGrantInfo> & _val49 = this->userPrivileges[_key48];
+              {
+                _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)
+                {
+                  xfer += _val49[_i54].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.userPrivileges = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        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)
+            {
+              std::string _key60;
+              xfer += iprot->readString(_key60);
+              std::vector<PrivilegeGrantInfo> & _val61 = this->groupPrivileges[_key60];
+              {
+                _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)
+                {
+                  xfer += _val61[_i66].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.groupPrivileges = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        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)
+            {
+              std::string _key72;
+              xfer += iprot->readString(_key72);
+              std::vector<PrivilegeGrantInfo> & _val73 = this->rolePrivileges[_key72];
+              {
+                _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)
+                {
+                  xfer += _val73[_i78].read(iprot);
+                }
+                xfer += iprot->readListEnd();
+              }
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.rolePrivileges = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2210,61 +2358,128 @@ uint32_t GrantRevokePrivilegeResponse::read(::apache::thrift::protocol::TProtoco
   return xfer;
 }
 
-uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse");
+  xfer += oprot->writeStructBegin("PrincipalPrivilegeSet");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
-    xfer += oprot->writeBool(this->success);
-    xfer += oprot->writeFieldEnd();
+  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)
+    {
+      xfer += oprot->writeString(_iter79->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 += (*_iter80).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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)
+    {
+      xfer += oprot->writeString(_iter81->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 += (*_iter82).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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)
+    {
+      xfer += oprot->writeString(_iter83->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 += (*_iter84).write(oprot);
+        }
+        xfer += oprot->writeListEnd();
+      }
+    }
+    xfer += oprot->writeMapEnd();
   }
+  xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) {
+void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) {
   using ::std::swap;
-  swap(a.success, b.success);
+  swap(a.userPrivileges, b.userPrivileges);
+  swap(a.groupPrivileges, b.groupPrivileges);
+  swap(a.rolePrivileges, b.rolePrivileges);
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other86) {
-  success = other86.success;
-  __isset = other86.__isset;
+PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other85) {
+  userPrivileges = other85.userPrivileges;
+  groupPrivileges = other85.groupPrivileges;
+  rolePrivileges = other85.rolePrivileges;
+  __isset = other85.__isset;
 }
-GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other87) {
-  success = other87.success;
-  __isset = other87.__isset;
+PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other86) {
+  userPrivileges = other86.userPrivileges;
+  groupPrivileges = other86.groupPrivileges;
+  rolePrivileges = other86.rolePrivileges;
+  __isset = other86.__isset;
   return *this;
 }
-void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const {
+void PrincipalPrivilegeSet::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "GrantRevokePrivilegeResponse(";
-  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << "PrincipalPrivilegeSet(";
+  out << "userPrivileges=" << to_string(userPrivileges);
+  out << ", " << "groupPrivileges=" << to_string(groupPrivileges);
+  out << ", " << "rolePrivileges=" << to_string(rolePrivileges);
   out << ")";
 }
 
 
-Role::~Role() throw() {
+GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() throw() {
 }
 
 
-void Role::__set_roleName(const std::string& val) {
-  this->roleName = val;
+void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) {
+  this->requestType = val;
 }
 
-void Role::__set_createTime(const int32_t val) {
-  this->createTime = val;
+void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) {
+  this->privileges = val;
 }
 
-void Role::__set_ownerName(const std::string& val) {
-  this->ownerName = val;
+void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) {
+  this->revokeGrantOption = val;
+__isset.revokeGrantOption = true;
 }
 
-uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t GrantRevokePrivilegeRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2286,15 +2501,232 @@ uint32_t Role::read(::apache::thrift::protocol::TProtocol* iprot) {
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->roleName);
-          this->__isset.roleName = true;
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast87;
+          xfer += iprot->readI32(ecast87);
+          this->requestType = (GrantRevokeType::type)ecast87;
+          this->__isset.requestType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->privileges.read(iprot);
+          this->__isset.privileges = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->revokeGrantOption);
+          this->__isset.revokeGrantOption = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest");
+
+  xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1);
+  xfer += oprot->writeI32((int32_t)this->requestType);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += this->privileges.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.revokeGrantOption) {
+    xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3);
+    xfer += oprot->writeBool(this->revokeGrantOption);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) {
+  using ::std::swap;
+  swap(a.requestType, b.requestType);
+  swap(a.privileges, b.privileges);
+  swap(a.revokeGrantOption, b.revokeGrantOption);
+  swap(a.__isset, b.__isset);
+}
+
+GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other88) {
+  requestType = other88.requestType;
+  privileges = other88.privileges;
+  revokeGrantOption = other88.revokeGrantOption;
+  __isset = other88.__isset;
+}
+GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other89) {
+  requestType = other89.requestType;
+  privileges = other89.privileges;
+  revokeGrantOption = other89.revokeGrantOption;
+  __isset = other89.__isset;
+  return *this;
+}
+void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GrantRevokePrivilegeRequest(";
+  out << "requestType=" << to_string(requestType);
+  out << ", " << "privileges=" << to_string(privileges);
+  out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "<null>"));
+  out << ")";
+}
+
+
+GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() throw() {
+}
+
+
+void GrantRevokePrivilegeResponse::__set_success(const bool val) {
+  this->success = val;
+__isset.success = true;
+}
+
+uint32_t GrantRevokePrivilegeResponse::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_BOOL) {
+          xfer += iprot->readBool(this->success);
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
+    xfer += oprot->writeBool(this->success);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) {
+  using ::std::swap;
+  swap(a.success, b.success);
+  swap(a.__isset, b.__isset);
+}
+
+GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other90) {
+  success = other90.success;
+  __isset = other90.__isset;
+}
+GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other91) {
+  success = other91.success;
+  __isset = other91.__isset;
+  return *this;
+}
+void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GrantRevokePrivilegeResponse(";
+  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << ")";
+}
+
+
+Role::~Role() throw() {
+}
+
+
+void Role::__set_roleName(const std::string& val) {
+  this->roleName = val;
+}
+
+void Role::__set_createTime(const int32_t val) {
+  this->createTime = val;
+}
+
+void Role::__set_ownerName(const std::string& val) {
+  this->ownerName = val;
+}
+
+uint32_t Role::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->roleName);
+          this->__isset.roleName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
           xfer += iprot->readI32(this->createTime);
           this->__isset.createTime = true;
         } else {
@@ -2351,17 +2783,17 @@ void swap(Role &a, Role &b) {
   swap(a.__isset, b.__isset);
 }
 
-Role::Role(const Role& other88) {
-  roleName = other88.roleName;
-  createTime = other88.createTime;
-  ownerName = other88.ownerName;
-  __isset = other88.__isset;
+Role::Role(const Role& other92) {
+  roleName = other92.roleName;
+  createTime = other92.createTime;
+  ownerName = other92.ownerName;
+  __isset = other92.__isset;
 }
-Role& Role::operator=(const Role& other89) {
-  roleName = other89.roleName;
-  createTime = other89.createTime;
-  ownerName = other89.ownerName;
-  __isset = other89.__isset;
+Role& Role::operator=(const Role& other93) {
+  roleName = other93.roleName;
+  createTime = other93.createTime;
+  ownerName = other93.ownerName;
+  __isset = other93.__isset;
   return *this;
 }
 void Role::printTo(std::ostream& out) const {
@@ -2445,9 +2877,9 @@ uint32_t RolePrincipalGrant::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast90;
-          xfer += iprot->readI32(ecast90);
-          this->principalType = (PrincipalType::type)ecast90;
+          int32_t ecast94;
+          xfer += iprot->readI32(ecast94);
+          this->principalType = (PrincipalType::type)ecast94;
           this->__isset.principalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2479,9 +2911,9 @@ uint32_t RolePrincipalGrant::read(::apache::thrift::protocol::TProtocol* iprot)
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast91;
-          xfer += iprot->readI32(ecast91);
-          this->grantorPrincipalType = (PrincipalType::type)ecast91;
+          int32_t ecast95;
+          xfer += iprot->readI32(ecast95);
+          this->grantorPrincipalType = (PrincipalType::type)ecast95;
           this->__isset.grantorPrincipalType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2549,25 +2981,25 @@ void swap(RolePrincipalGrant &a, RolePrincipalGrant &b) {
   swap(a.__isset, b.__isset);
 }
 
-RolePrincipalGrant::RolePrincipalGrant(const RolePrincipalGrant& other92) {
-  roleName = other92.roleName;
-  principalName = other92.principalName;
-  principalType = other92.principalType;
-  grantOption = other92.grantOption;
-  grantTime = other92.grantTime;
-  grantorName = other92.grantorName;
-  grantorPrincipalType = other92.grantorPrincipalType;
-  __isset = other92.__isset;
-}
-RolePrincipalGrant& RolePrincipalGrant::operator=(const RolePrincipalGrant& other93) {
-  roleName = other93.roleName;
-  principalName = other93.principalName;
-  principalType = other93.principalType;
-  grantOption = other93.grantOption;
-  grantTime = other93.grantTime;
-  grantorName = other93.grantorName;
-  grantorPrincipalType = other93.grantorPrincipalType;
-  __isset = other93.__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;
   return *this;
 }
 void RolePrincipalGrant::printTo(std::ostream& out) const {
@@ -2629,9 +3061,9 @@ uint32_t GetRoleGrantsForPrincipalRequest::read(::apache::thrift::protocol::TPro
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast94;
-          xfer += iprot->readI32(ecast94);
-          this->principal_type = (PrincipalType::type)ecast94;
+          int32_t ecast98;
+          xfer += iprot->readI32(ecast98);
+          this->principal_type = (PrincipalType::type)ecast98;
           isset_principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2677,13 +3109,13 @@ void swap(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest
   swap(a.principal_type, b.principal_type);
 }
 
-GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other95) {
-  principal_name = other95.principal_name;
-  principal_type = other95.principal_type;
+GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other99) {
+  principal_name = other99.principal_name;
+  principal_type = other99.principal_type;
 }
-GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other96) {
-  principal_name = other96.principal_name;
-  principal_type = other96.principal_type;
+GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other100) {
+  principal_name = other100.principal_name;
+  principal_type = other100.principal_type;
   return *this;
 }
 void GetRoleGrantsForPrincipalRequest::printTo(std::ostream& out) const {
@@ -2729,14 +3161,14 @@ uint32_t GetRoleGrantsForPrincipalResponse::read(::apache::thrift::protocol::TPr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->principalGrants.clear();
-            uint32_t _size97;
-            ::apache::thrift::protocol::TType _etype100;
-            xfer += iprot->readListBegin(_etype100, _size97);
-            this->principalGrants.resize(_size97);
-            uint32_t _i101;
-            for (_i101 = 0; _i101 < _size97; ++_i101)
+            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)
             {
-              xfer += this->principalGrants[_i101].read(iprot);
+              xfer += this->principalGrants[_i105].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2767,10 +3199,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 _iter102;
-    for (_iter102 = this->principalGrants.begin(); _iter102 != this->principalGrants.end(); ++_iter102)
+    std::vector<RolePrincipalGrant> ::const_iterator _iter106;
+    for (_iter106 = this->principalGrants.begin(); _iter106 != this->principalGrants.end(); ++_iter1

<TRUNCATED>

[25/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 273d37e..0d4fe5a 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/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 _size817;
-            ::apache::thrift::protocol::TType _etype820;
-            xfer += iprot->readListBegin(_etype820, _size817);
-            this->success.resize(_size817);
-            uint32_t _i821;
-            for (_i821 = 0; _i821 < _size817; ++_i821)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            this->success.resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += iprot->readString(this->success[_i821]);
+              xfer += iprot->readString(this->success[_i861]);
             }
             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 _iter822;
-      for (_iter822 = this->success.begin(); _iter822 != this->success.end(); ++_iter822)
+      std::vector<std::string> ::const_iterator _iter862;
+      for (_iter862 = this->success.begin(); _iter862 != this->success.end(); ++_iter862)
       {
-        xfer += oprot->writeString((*_iter822));
+        xfer += oprot->writeString((*_iter862));
       }
       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 _size823;
-            ::apache::thrift::protocol::TType _etype826;
-            xfer += iprot->readListBegin(_etype826, _size823);
-            (*(this->success)).resize(_size823);
-            uint32_t _i827;
-            for (_i827 = 0; _i827 < _size823; ++_i827)
+            uint32_t _size863;
+            ::apache::thrift::protocol::TType _etype866;
+            xfer += iprot->readListBegin(_etype866, _size863);
+            (*(this->success)).resize(_size863);
+            uint32_t _i867;
+            for (_i867 = 0; _i867 < _size863; ++_i867)
             {
-              xfer += iprot->readString((*(this->success))[_i827]);
+              xfer += iprot->readString((*(this->success))[_i867]);
             }
             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 _size828;
-            ::apache::thrift::protocol::TType _etype831;
-            xfer += iprot->readListBegin(_etype831, _size828);
-            this->success.resize(_size828);
-            uint32_t _i832;
-            for (_i832 = 0; _i832 < _size828; ++_i832)
+            uint32_t _size868;
+            ::apache::thrift::protocol::TType _etype871;
+            xfer += iprot->readListBegin(_etype871, _size868);
+            this->success.resize(_size868);
+            uint32_t _i872;
+            for (_i872 = 0; _i872 < _size868; ++_i872)
             {
-              xfer += iprot->readString(this->success[_i832]);
+              xfer += iprot->readString(this->success[_i872]);
             }
             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 _iter833;
-      for (_iter833 = this->success.begin(); _iter833 != this->success.end(); ++_iter833)
+      std::vector<std::string> ::const_iterator _iter873;
+      for (_iter873 = this->success.begin(); _iter873 != this->success.end(); ++_iter873)
       {
-        xfer += oprot->writeString((*_iter833));
+        xfer += oprot->writeString((*_iter873));
       }
       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 _size834;
-            ::apache::thrift::protocol::TType _etype837;
-            xfer += iprot->readListBegin(_etype837, _size834);
-            (*(this->success)).resize(_size834);
-            uint32_t _i838;
-            for (_i838 = 0; _i838 < _size834; ++_i838)
+            uint32_t _size874;
+            ::apache::thrift::protocol::TType _etype877;
+            xfer += iprot->readListBegin(_etype877, _size874);
+            (*(this->success)).resize(_size874);
+            uint32_t _i878;
+            for (_i878 = 0; _i878 < _size874; ++_i878)
             {
-              xfer += iprot->readString((*(this->success))[_i838]);
+              xfer += iprot->readString((*(this->success))[_i878]);
             }
             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 _size839;
-            ::apache::thrift::protocol::TType _ktype840;
-            ::apache::thrift::protocol::TType _vtype841;
-            xfer += iprot->readMapBegin(_ktype840, _vtype841, _size839);
-            uint32_t _i843;
-            for (_i843 = 0; _i843 < _size839; ++_i843)
+            uint32_t _size879;
+            ::apache::thrift::protocol::TType _ktype880;
+            ::apache::thrift::protocol::TType _vtype881;
+            xfer += iprot->readMapBegin(_ktype880, _vtype881, _size879);
+            uint32_t _i883;
+            for (_i883 = 0; _i883 < _size879; ++_i883)
             {
-              std::string _key844;
-              xfer += iprot->readString(_key844);
-              Type& _val845 = this->success[_key844];
-              xfer += _val845.read(iprot);
+              std::string _key884;
+              xfer += iprot->readString(_key884);
+              Type& _val885 = this->success[_key884];
+              xfer += _val885.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 _iter846;
-      for (_iter846 = this->success.begin(); _iter846 != this->success.end(); ++_iter846)
+      std::map<std::string, Type> ::const_iterator _iter886;
+      for (_iter886 = this->success.begin(); _iter886 != this->success.end(); ++_iter886)
       {
-        xfer += oprot->writeString(_iter846->first);
-        xfer += _iter846->second.write(oprot);
+        xfer += oprot->writeString(_iter886->first);
+        xfer += _iter886->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 _size847;
-            ::apache::thrift::protocol::TType _ktype848;
-            ::apache::thrift::protocol::TType _vtype849;
-            xfer += iprot->readMapBegin(_ktype848, _vtype849, _size847);
-            uint32_t _i851;
-            for (_i851 = 0; _i851 < _size847; ++_i851)
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _ktype888;
+            ::apache::thrift::protocol::TType _vtype889;
+            xfer += iprot->readMapBegin(_ktype888, _vtype889, _size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
             {
-              std::string _key852;
-              xfer += iprot->readString(_key852);
-              Type& _val853 = (*(this->success))[_key852];
-              xfer += _val853.read(iprot);
+              std::string _key892;
+              xfer += iprot->readString(_key892);
+              Type& _val893 = (*(this->success))[_key892];
+              xfer += _val893.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 _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            this->success.resize(_size854);
-            uint32_t _i858;
-            for (_i858 = 0; _i858 < _size854; ++_i858)
+            uint32_t _size894;
+            ::apache::thrift::protocol::TType _etype897;
+            xfer += iprot->readListBegin(_etype897, _size894);
+            this->success.resize(_size894);
+            uint32_t _i898;
+            for (_i898 = 0; _i898 < _size894; ++_i898)
             {
-              xfer += this->success[_i858].read(iprot);
+              xfer += this->success[_i898].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 _iter859;
-      for (_iter859 = this->success.begin(); _iter859 != this->success.end(); ++_iter859)
+      std::vector<FieldSchema> ::const_iterator _iter899;
+      for (_iter899 = this->success.begin(); _iter899 != this->success.end(); ++_iter899)
       {
-        xfer += (*_iter859).write(oprot);
+        xfer += (*_iter899).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 _size860;
-            ::apache::thrift::protocol::TType _etype863;
-            xfer += iprot->readListBegin(_etype863, _size860);
-            (*(this->success)).resize(_size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size900;
+            ::apache::thrift::protocol::TType _etype903;
+            xfer += iprot->readListBegin(_etype903, _size900);
+            (*(this->success)).resize(_size900);
+            uint32_t _i904;
+            for (_i904 = 0; _i904 < _size900; ++_i904)
             {
-              xfer += (*(this->success))[_i864].read(iprot);
+              xfer += (*(this->success))[_i904].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 _size865;
-            ::apache::thrift::protocol::TType _etype868;
-            xfer += iprot->readListBegin(_etype868, _size865);
-            this->success.resize(_size865);
-            uint32_t _i869;
-            for (_i869 = 0; _i869 < _size865; ++_i869)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            this->success.resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += this->success[_i869].read(iprot);
+              xfer += this->success[_i909].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 _iter870;
-      for (_iter870 = this->success.begin(); _iter870 != this->success.end(); ++_iter870)
+      std::vector<FieldSchema> ::const_iterator _iter910;
+      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
       {
-        xfer += (*_iter870).write(oprot);
+        xfer += (*_iter910).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 _size871;
-            ::apache::thrift::protocol::TType _etype874;
-            xfer += iprot->readListBegin(_etype874, _size871);
-            (*(this->success)).resize(_size871);
-            uint32_t _i875;
-            for (_i875 = 0; _i875 < _size871; ++_i875)
+            uint32_t _size911;
+            ::apache::thrift::protocol::TType _etype914;
+            xfer += iprot->readListBegin(_etype914, _size911);
+            (*(this->success)).resize(_size911);
+            uint32_t _i915;
+            for (_i915 = 0; _i915 < _size911; ++_i915)
             {
-              xfer += (*(this->success))[_i875].read(iprot);
+              xfer += (*(this->success))[_i915].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 _size876;
-            ::apache::thrift::protocol::TType _etype879;
-            xfer += iprot->readListBegin(_etype879, _size876);
-            this->success.resize(_size876);
-            uint32_t _i880;
-            for (_i880 = 0; _i880 < _size876; ++_i880)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->success.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += this->success[_i880].read(iprot);
+              xfer += this->success[_i920].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 _iter881;
-      for (_iter881 = this->success.begin(); _iter881 != this->success.end(); ++_iter881)
+      std::vector<FieldSchema> ::const_iterator _iter921;
+      for (_iter921 = this->success.begin(); _iter921 != this->success.end(); ++_iter921)
       {
-        xfer += (*_iter881).write(oprot);
+        xfer += (*_iter921).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 _size882;
-            ::apache::thrift::protocol::TType _etype885;
-            xfer += iprot->readListBegin(_etype885, _size882);
-            (*(this->success)).resize(_size882);
-            uint32_t _i886;
-            for (_i886 = 0; _i886 < _size882; ++_i886)
+            uint32_t _size922;
+            ::apache::thrift::protocol::TType _etype925;
+            xfer += iprot->readListBegin(_etype925, _size922);
+            (*(this->success)).resize(_size922);
+            uint32_t _i926;
+            for (_i926 = 0; _i926 < _size922; ++_i926)
             {
-              xfer += (*(this->success))[_i886].read(iprot);
+              xfer += (*(this->success))[_i926].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 _size887;
-            ::apache::thrift::protocol::TType _etype890;
-            xfer += iprot->readListBegin(_etype890, _size887);
-            this->success.resize(_size887);
-            uint32_t _i891;
-            for (_i891 = 0; _i891 < _size887; ++_i891)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            this->success.resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += this->success[_i891].read(iprot);
+              xfer += this->success[_i931].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 _iter892;
-      for (_iter892 = this->success.begin(); _iter892 != this->success.end(); ++_iter892)
+      std::vector<FieldSchema> ::const_iterator _iter932;
+      for (_iter932 = this->success.begin(); _iter932 != this->success.end(); ++_iter932)
       {
-        xfer += (*_iter892).write(oprot);
+        xfer += (*_iter932).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 _size893;
-            ::apache::thrift::protocol::TType _etype896;
-            xfer += iprot->readListBegin(_etype896, _size893);
-            (*(this->success)).resize(_size893);
-            uint32_t _i897;
-            for (_i897 = 0; _i897 < _size893; ++_i897)
+            uint32_t _size933;
+            ::apache::thrift::protocol::TType _etype936;
+            xfer += iprot->readListBegin(_etype936, _size933);
+            (*(this->success)).resize(_size933);
+            uint32_t _i937;
+            for (_i937 = 0; _i937 < _size933; ++_i937)
             {
-              xfer += (*(this->success))[_i897].read(iprot);
+              xfer += (*(this->success))[_i937].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 _size898;
-            ::apache::thrift::protocol::TType _etype901;
-            xfer += iprot->readListBegin(_etype901, _size898);
-            this->primaryKeys.resize(_size898);
-            uint32_t _i902;
-            for (_i902 = 0; _i902 < _size898; ++_i902)
+            uint32_t _size938;
+            ::apache::thrift::protocol::TType _etype941;
+            xfer += iprot->readListBegin(_etype941, _size938);
+            this->primaryKeys.resize(_size938);
+            uint32_t _i942;
+            for (_i942 = 0; _i942 < _size938; ++_i942)
             {
-              xfer += this->primaryKeys[_i902].read(iprot);
+              xfer += this->primaryKeys[_i942].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 _size903;
-            ::apache::thrift::protocol::TType _etype906;
-            xfer += iprot->readListBegin(_etype906, _size903);
-            this->foreignKeys.resize(_size903);
-            uint32_t _i907;
-            for (_i907 = 0; _i907 < _size903; ++_i907)
+            uint32_t _size943;
+            ::apache::thrift::protocol::TType _etype946;
+            xfer += iprot->readListBegin(_etype946, _size943);
+            this->foreignKeys.resize(_size943);
+            uint32_t _i947;
+            for (_i947 = 0; _i947 < _size943; ++_i947)
             {
-              xfer += this->foreignKeys[_i907].read(iprot);
+              xfer += this->foreignKeys[_i947].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4554,6 +4554,46 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->uniqueConstraints.clear();
+            uint32_t _size948;
+            ::apache::thrift::protocol::TType _etype951;
+            xfer += iprot->readListBegin(_etype951, _size948);
+            this->uniqueConstraints.resize(_size948);
+            uint32_t _i952;
+            for (_i952 = 0; _i952 < _size948; ++_i952)
+            {
+              xfer += this->uniqueConstraints[_i952].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.uniqueConstraints = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->notNullConstraints.clear();
+            uint32_t _size953;
+            ::apache::thrift::protocol::TType _etype956;
+            xfer += iprot->readListBegin(_etype956, _size953);
+            this->notNullConstraints.resize(_size953);
+            uint32_t _i957;
+            for (_i957 = 0; _i957 < _size953; ++_i957)
+            {
+              xfer += this->notNullConstraints[_i957].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.notNullConstraints = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4578,10 +4618,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter908;
-    for (_iter908 = this->primaryKeys.begin(); _iter908 != this->primaryKeys.end(); ++_iter908)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter958;
+    for (_iter958 = this->primaryKeys.begin(); _iter958 != this->primaryKeys.end(); ++_iter958)
     {
-      xfer += (*_iter908).write(oprot);
+      xfer += (*_iter958).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4590,10 +4630,34 @@ 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 _iter909;
-    for (_iter909 = this->foreignKeys.begin(); _iter909 != this->foreignKeys.end(); ++_iter909)
+    std::vector<SQLForeignKey> ::const_iterator _iter959;
+    for (_iter959 = this->foreignKeys.begin(); _iter959 != this->foreignKeys.end(); ++_iter959)
     {
-      xfer += (*_iter909).write(oprot);
+      xfer += (*_iter959).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter960;
+    for (_iter960 = this->uniqueConstraints.begin(); _iter960 != this->uniqueConstraints.end(); ++_iter960)
+    {
+      xfer += (*_iter960).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter961;
+    for (_iter961 = this->notNullConstraints.begin(); _iter961 != this->notNullConstraints.end(); ++_iter961)
+    {
+      xfer += (*_iter961).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4621,10 +4685,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter910;
-    for (_iter910 = (*(this->primaryKeys)).begin(); _iter910 != (*(this->primaryKeys)).end(); ++_iter910)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter962;
+    for (_iter962 = (*(this->primaryKeys)).begin(); _iter962 != (*(this->primaryKeys)).end(); ++_iter962)
     {
-      xfer += (*_iter910).write(oprot);
+      xfer += (*_iter962).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4633,10 +4697,34 @@ 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 _iter911;
-    for (_iter911 = (*(this->foreignKeys)).begin(); _iter911 != (*(this->foreignKeys)).end(); ++_iter911)
+    std::vector<SQLForeignKey> ::const_iterator _iter963;
+    for (_iter963 = (*(this->foreignKeys)).begin(); _iter963 != (*(this->foreignKeys)).end(); ++_iter963)
+    {
+      xfer += (*_iter963).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter964;
+    for (_iter964 = (*(this->uniqueConstraints)).begin(); _iter964 != (*(this->uniqueConstraints)).end(); ++_iter964)
+    {
+      xfer += (*_iter964).write(oprot);
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter965;
+    for (_iter965 = (*(this->notNullConstraints)).begin(); _iter965 != (*(this->notNullConstraints)).end(); ++_iter965)
     {
-      xfer += (*_iter911).write(oprot);
+      xfer += (*_iter965).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5437,11 +5525,218 @@ uint32_t ThriftHiveMetastore_add_foreign_key_presult::read(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
+ThriftHiveMetastore_add_unique_constraint_args::~ThriftHiveMetastore_add_unique_constraint_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_unique_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_unique_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_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_unique_constraint_pargs::~ThriftHiveMetastore_add_unique_constraint_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_unique_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_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_unique_constraint_result::~ThriftHiveMetastore_add_unique_constraint_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_unique_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_unique_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_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_unique_constraint_presult::~ThriftHiveMetastore_add_unique_constraint_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_unique_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_add_not_null_constraint_args::~ThriftHiveMetastore_add_not_null_constraint_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_add_not_null_constraint_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5463,25 +5758,9 @@ uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::T
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbname);
-          this->__isset.dbname = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->deleteData);
-          this->__isset.deleteData = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -5498,21 +5777,13 @@ uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_not_null_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args");
-
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->dbname);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->name);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_args");
 
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool(this->deleteData);
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -5521,25 +5792,17 @@ uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() throw() {
+ThriftHiveMetastore_add_not_null_constraint_pargs::~ThriftHiveMetastore_add_not_null_constraint_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_not_null_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs");
-
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->dbname)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->name)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_pargs");
 
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool((*(this->deleteData)));
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -5548,11 +5811,11 @@ uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() throw() {
+ThriftHiveMetastore_add_not_null_constraint_result::~ThriftHiveMetastore_add_not_null_constraint_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_not_null_constraint_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5583,8 +5846,8 @@ uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol:
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -5601,19 +5864,19 @@ uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_not_null_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_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.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o3.write(oprot);
+  } 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();
@@ -5622,11 +5885,11 @@ uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() throw() {
+ThriftHiveMetastore_add_not_null_constraint_presult::~ThriftHiveMetastore_add_not_null_constraint_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_not_null_constraint_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5657,8 +5920,8 @@ uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -5676,11 +5939,11 @@ uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() throw() {
+ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5725,14 +5988,6 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::ap
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->environment_context.read(iprot);
-          this->__isset.environment_context = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -5745,10 +6000,10 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::ap
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args");
 
   xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->dbname);
@@ -5762,24 +6017,20 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::a
   xfer += oprot->writeBool(this->deleteData);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += this->environment_context.write(oprot);
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() throw() {
+ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs");
 
   xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString((*(this->dbname)));
@@ -5793,21 +6044,17 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::
   xfer += oprot->writeBool((*(this->deleteData)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
-  xfer += (*(this->environment_context)).write(oprot);
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() throw() {
+ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5856,11 +6103,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result");
 
   if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
@@ -5877,11 +6124,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(:
 }
 
 
-ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() throw() {
+ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5931,11 +6178,11 @@ uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::read(:
 }
 
 
-ThriftHiveMetastore_truncate_table_args::~ThriftHiveMetastore_truncate_table_args() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -5958,36 +6205,32 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbName);
-          this->__isset.dbName = true;
+          xfer += iprot->readString(this->dbname);
+          this->__isset.dbname = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->tableName);
-          this->__isset.tableName = true;
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->partNames.clear();
-            uint32_t _size912;
-            ::apache::thrift::protocol::TType _etype915;
-            xfer += iprot->readListBegin(_etype915, _size912);
-            this->partNames.resize(_size912);
-            uint32_t _i916;
-            for (_i916 = 0; _i916 < _size912; ++_i916)
-            {
-              xfer += iprot->readString(this->partNames[_i916]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.partNames = true;
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->deleteData);
+          this->__isset.deleteData = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->environment_context.read(iprot);
+          this->__isset.environment_context = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -6004,29 +6247,25 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args");
 
-  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbname);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->tableName);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->name);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter917;
-    for (_iter917 = this->partNames.begin(); _iter917 != this->partNames.end(); ++_iter917)
-    {
-      xfer += oprot->writeString((*_iter917));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool(this->deleteData);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += this->environment_context.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -6035,108 +6274,42 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_truncate_table_pargs::~ThriftHiveMetastore_truncate_table_pargs() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs");
 
-  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->dbName)));
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->dbname)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->tableName)));
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->name)));
   xfer += oprot->writeFieldEnd();
 
-  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 _iter918;
-    for (_iter918 = (*(this->partNames)).begin(); _iter918 != (*(this->partNames)).end(); ++_iter918)
-    {
-      xfer += oprot->writeString((*_iter918));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool((*(this->deleteData)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_truncate_table_result::~ThriftHiveMetastore_truncate_table_result() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_truncate_table_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;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_truncate_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_result");
+  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += (*(this->environment_context)).write(oprot);
+  xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_truncate_table_presult::~ThriftHiveMetastore_truncate_table_presult() throw() {
+ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_truncate_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6165,6 +6338,14 @@ uint32_t ThriftHiveMetastore_truncate_table_presult::read(::apache::thrift::prot
           xfer += iprot->skip(ftype);
         }
         break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -6177,12 +6358,86 @@ uint32_t ThriftHiveMetastore_truncate_table_presult::read(::apache::thrift::prot
   return xfer;
 }
 
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
-ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() throw() {
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_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.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_drop_table_with_environment_context_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->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_truncate_table_args::~ThriftHiveMetastore_truncate_table_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6205,16 +6460,36 @@ uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::T
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
+          xfer += iprot->readString(this->dbName);
+          this->__isset.dbName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->pattern);
-          this->__isset.pattern = true;
+          xfer += iprot->readString(this->tableName);
+          this->__isset.tableName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->partNames.clear();
+            uint32_t _size966;
+            ::apache::thrift::protocol::TType _etype969;
+            xfer += iprot->readListBegin(_etype969, _size966);
+            this->partNames.resize(_size966);
+            uint32_t _i970;
+            for (_i970 = 0; _i970 < _size966; ++_i970)
+            {
+              xfer += iprot->readString(this->partNames[_i970]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.partNames = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -6231,17 +6506,29 @@ uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_args");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->pattern);
+  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tableName);
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter971;
+    for (_iter971 = this->partNames.begin(); _iter971 != this->partNames.end(); ++_iter971)
+    {
+      xfer += oprot->writeString((*_iter971));
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -6250,21 +6537,33 @@ uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() throw() {
+ThriftHiveMetastore_truncate_table_pargs::~ThriftHiveMetastore_truncate_table_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_pargs");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->dbName)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->pattern)));
+  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->tableName)));
+  xfer += oprot->writeFieldEnd();
+
+  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 _iter972;
+    for (_iter972 = (*(this->partNames)).begin(); _iter972 != (*(this->partNames)).end(); ++_iter972)
+    {
+      xfer += oprot->writeString((*_iter972));
+    }
+    xfer += oprot->writeListEnd();
+  }
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -6273,11 +6572,11 @@ uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() throw() {
+ThriftHiveMetastore_truncate_table_result::~ThriftHiveMetastore_truncate_table_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_truncate_table_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6298,26 +6597,6 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size919;
-            ::apache::thrift::protocol::TType _etype922;
-            xfer += iprot->readListBegin(_etype922, _size919);
-            this->success.resize(_size919);
-            uint32_t _i923;
-            for (_i923 = 0; _i923 < _size919; ++_i923)
-            {
-              xfer += iprot->readString(this->success[_i923]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -6338,25 +6617,13 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_truncate_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter924;
-      for (_iter924 = this->success.begin(); _iter924 != this->success.end(); ++_iter924)
-      {
-        xfer += oprot->writeString((*_iter924));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
+  if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
@@ -6367,11 +6634,11 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_tables_presult::~ThriftHiveMetastore_get_tables_presult() throw() {
+ThriftHiveMetastore_truncate_table_presult::~ThriftHiveMetastore_truncate_table_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_truncate_table_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6392,26 +6659,6 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size925;
-            ::apache::thrift::protocol::TType _etype928;
-            xfer += iprot->readListBegin(_etype928, _size925);
-            (*(this->success)).resize(_size925);
-            uint32_t _i929;
-            for (_i929 = 0; _i929 < _size925; ++_i929)
-            {
-              xfer += iprot->readString((*(this->success))[_i929]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -6433,11 +6680,11 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() throw() {
+ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_tables_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6474,14 +6721,6 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_args::read(::apache::thrift::pro
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->tableType);
-          this->__isset.tableType = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -6494,10 +6733,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_args::read(::apache::thrift::pro
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args");
 
   xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->db_name);
@@ -6507,24 +6746,20 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::pr
   xfer += oprot->writeString(this->pattern);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString(this->tableType);
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() throw() {
+ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs");
 
   xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString((*(this->db_name)));
@@ -6534,21 +6769,17 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::p
   xfer += oprot->writeString((*(this->pattern)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeString((*(this->tableType)));
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() throw() {
+ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6573,14 +6804,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 _size930;
-            ::apache::thrift::protocol::TType _etype933;
-            xfer += iprot->readListBegin(_etype933, _size930);
-            this->success.resize(_size930);
-            uint32_t _i934;
-            for (_i934 = 0; _i934 < _size930; ++_i934)
+            uint32_t _size973;
+            ::apache::thrift::protocol::TType _etype976;
+            xfer += iprot->readListBegin(_etype976, _size973);
+            this->success.resize(_size973);
+            uint32_t _i977;
+            for (_i977 = 0; _i977 < _size973; ++_i977)
             {
-              xfer += iprot->readString(this->success[_i934]);
+              xfer += iprot->readString(this->success[_i977]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6609,20 +6840,20 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter935;
-      for (_iter935 = this->success.begin(); _iter935 != this->success.end(); ++_iter935)
+      std::vector<std::string> ::const_iterator _iter978;
+      for (_iter978 = this->success.begin(); _iter978 != this->success.end(); ++_iter978)
       {
-        xfer += oprot->writeString((*_iter935));
+        xfer += oprot->writeString((*_iter978));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6638,11 +6869,11 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+ThriftHiveMetastore_get_tables_presult::~ThriftHiveMetastore_get_tables_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6667,14 +6898,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size936;
-            ::apache::thrift::protocol::TType _etype939;
-            xfer += iprot->readListBegin(_etype939, _size936);
-            (*(this->success)).resize(_size936);
-            uint32_t _i940;
-            for (_i940 = 0; _i940 < _size936; ++_i940)
+            uint32_t _size979;
+            ::apache::thrift::protocol::TType _etype982;
+            xfer += iprot->readListBegin(_etype982, _size979);
+            (*(this->success)).resize(_size979);
+            uint32_t _i983;
+            for (_i983 = 0; _i983 < _size979; ++_i983)
             {
-              xfer += iprot->readString((*(this->success))[_i940]);
+              xfer += iprot->readString((*(this->success))[_i983]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6704,11 +6935,11 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() throw() {
+ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_tables_by_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6731,36 +6962,24 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_patterns);
-          this->__isset.db_patterns = true;
+          xfer += iprot->readString(this->db_name);
+          this->__isset.db_name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->tbl_patterns);
-          this->__isset.tbl_patterns = true;
+          xfer += iprot->readString(this->pattern);
+          this->__isset.pattern = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 3:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->tbl_types.clear();
-            uint32_t _size941;
-            ::apache::thrift::protocol::TType _etype944;
-            xfer += iprot->readListBegin(_etype944, _size941);
-            this->tbl_types.resize(_size941);
-            uint32_t _i945;
-            for (_i945 = 0; _i945 < _size941; ++_i945)
-            {
-              xfer += iprot->readString(this->tbl_types[_i945]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.tbl_types = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableType);
+          this->__isset.tableType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -6777,29 +6996,21 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args");
 
-  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_patterns);
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->tbl_patterns);
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->pattern);
   xfer += oprot->writeFieldEnd();
 
-  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 _iter946;
-    for (_iter946 = this->tbl_types.begin(); _iter946 != this->tbl_types.end(); ++_iter946)
-    {
-      xfer += oprot->writeString((*_iter946));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->tableType);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -6808,33 +7019,25 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_get_table_meta_pargs::~ThriftHiveMetastore_get_table_meta_pargs() throw() {
+ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs");
 
-  xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_patterns)));
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_name)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->tbl_patterns)));
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString((*(this->pattern)));
   xfer += oprot->writeFieldEnd();
 
-  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 _iter947;
-    for (_iter947 = (*(this->tbl_types)).begin(); _iter947 != (*(this->tbl_types)).end(); ++_iter947)
-    {
-      xfer += oprot->writeString((*_iter947));
-    }
-    xfer += oprot->writeListEnd();
-  }
+  xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString((*(this->tableType)));
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -6843,11 +7046,11 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
 }
 
 
-ThriftHiveMetastore_get_table_meta_result::~ThriftHiveMetastore_get_table_meta_result() throw() {
+ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -6872,14 +7075,313 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size948;
-            ::apache::thrift::protocol::TType _etype951;
-            xfer += iprot->readListBegin(_etype951, _size948);
-            this->success.resize(_size948);
-            uint32_t _i952;
-            for (_i952 = 0; _i952 < _size948; ++_i952)
+            uint32_t _size984;
+            ::apache::thrift::protocol::TType _etype987;
+            xfer += iprot->readListBegin(_etype987, _size984);
+            this->success.resize(_size984);
+            uint32_t _i988;
+            for (_i988 = 0; _i988 < _size984; ++_i988)
+            {
+              xfer += iprot->readString(this->success[_i988]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
+      std::vector<std::string> ::const_iterator _iter989;
+      for (_iter989 = this->success.begin(); _iter989 != this->success.end(); ++_iter989)
+      {
+        xfer += oprot->writeString((*_iter989));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size990;
+            ::apache::thrift::protocol::TType _etype993;
+            xfer += iprot->readListBegin(_etype993, _size990);
+            (*(this->success)).resize(_size990);
+            uint32_t _i994;
+            for (_i994 = 0; _i994 < _size990; ++_i994)
             {
-              xfer += this->success[_i952].read(iprot);
+              xfer

<TRUNCATED>

[21/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 7bb10b3..cc336c5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -713,13 +713,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list574.size);
-                String _elem575;
-                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list606.size);
+                String _elem607;
+                for (int _i608 = 0; _i608 < _list606.size; ++_i608)
                 {
-                  _elem575 = iprot.readString();
-                  struct.partitionVals.add(_elem575);
+                  _elem607 = iprot.readString();
+                  struct.partitionVals.add(_elem607);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           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 _iter577 : struct.partitionVals)
+            for (String _iter609 : struct.partitionVals)
             {
-              oprot.writeString(_iter577);
+              oprot.writeString(_iter609);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter578 : struct.partitionVals)
+          for (String _iter610 : struct.partitionVals)
           {
-            oprot.writeString(_iter578);
+            oprot.writeString(_iter610);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list579.size);
-          String _elem580;
-          for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+          org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list611.size);
+          String _elem612;
+          for (int _i613 = 0; _i613 < _list611.size; ++_i613)
           {
-            _elem580 = iprot.readString();
-            struct.partitionVals.add(_elem580);
+            _elem612 = iprot.readString();
+            struct.partitionVals.add(_elem612);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index 56a4d30..ad12ab1 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -997,14 +997,14 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list452.size);
-                ResourceUri _elem453;
-                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
+                org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list484.size);
+                ResourceUri _elem485;
+                for (int _i486 = 0; _i486 < _list484.size; ++_i486)
                 {
-                  _elem453 = new ResourceUri();
-                  _elem453.read(iprot);
-                  struct.resourceUris.add(_elem453);
+                  _elem485 = new ResourceUri();
+                  _elem485.read(iprot);
+                  struct.resourceUris.add(_elem485);
                 }
                 iprot.readListEnd();
               }
@@ -1063,9 +1063,9 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
         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 _iter455 : struct.resourceUris)
+          for (ResourceUri _iter487 : struct.resourceUris)
           {
-            _iter455.write(oprot);
+            _iter487.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1138,9 +1138,9 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter456 : struct.resourceUris)
+          for (ResourceUri _iter488 : struct.resourceUris)
           {
-            _iter456.write(oprot);
+            _iter488.write(oprot);
           }
         }
       }
@@ -1180,14 +1180,14 @@ public class Function implements org.apache.thrift.TBase<Function, Function._Fie
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list457.size);
-          ResourceUri _elem458;
-          for (int _i459 = 0; _i459 < _list457.size; ++_i459)
+          org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list489.size);
+          ResourceUri _elem490;
+          for (int _i491 = 0; _i491 < _list489.size; ++_i491)
           {
-            _elem458 = new ResourceUri();
-            _elem458.read(iprot);
-            struct.resourceUris.add(_elem458);
+            _elem490 = new ResourceUri();
+            _elem490.read(iprot);
+            struct.resourceUris.add(_elem490);
           }
         }
         struct.setResourceUrisIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 49a1be2..bb11fe3 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list642.size);
-                Function _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list674.size);
+                Function _elem675;
+                for (int _i676 = 0; _i676 < _list674.size; ++_i676)
                 {
-                  _elem643 = new Function();
-                  _elem643.read(iprot);
-                  struct.functions.add(_elem643);
+                  _elem675 = new Function();
+                  _elem675.read(iprot);
+                  struct.functions.add(_elem675);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter645 : struct.functions)
+            for (Function _iter677 : struct.functions)
             {
-              _iter645.write(oprot);
+              _iter677.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter646 : struct.functions)
+          for (Function _iter678 : struct.functions)
           {
-            _iter646.write(oprot);
+            _iter678.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list647.size);
-          Function _elem648;
-          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+          org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list679.size);
+          Function _elem680;
+          for (int _i681 = 0; _i681 < _list679.size; ++_i681)
           {
-            _elem648 = new Function();
-            _elem648.read(iprot);
-            struct.functions.add(_elem648);
+            _elem680 = new Function();
+            _elem680.read(iprot);
+            struct.functions.add(_elem680);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 20b82cf..ef758cd 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list592.size);
-                long _elem593;
-                for (int _i594 = 0; _i594 < _list592.size; ++_i594)
+                org.apache.thrift.protocol.TList _list624 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list624.size);
+                long _elem625;
+                for (int _i626 = 0; _i626 < _list624.size; ++_i626)
                 {
-                  _elem593 = iprot.readI64();
-                  struct.fileIds.add(_elem593);
+                  _elem625 = iprot.readI64();
+                  struct.fileIds.add(_elem625);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
         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 _iter595 : struct.fileIds)
+          for (long _iter627 : struct.fileIds)
           {
-            oprot.writeI64(_iter595);
+            oprot.writeI64(_iter627);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter596 : struct.fileIds)
+        for (long _iter628 : struct.fileIds)
         {
-          oprot.writeI64(_iter596);
+          oprot.writeI64(_iter628);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
     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 _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list597.size);
-        long _elem598;
-        for (int _i599 = 0; _i599 < _list597.size; ++_i599)
+        org.apache.thrift.protocol.TList _list629 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list629.size);
+        long _elem630;
+        for (int _i631 = 0; _i631 < _list629.size; ++_i631)
         {
-          _elem598 = iprot.readI64();
-          struct.fileIds.add(_elem598);
+          _elem630 = iprot.readI64();
+          struct.fileIds.add(_elem630);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index 9975dfc..ee94380 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map582 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map582.size);
-                long _key583;
-                MetadataPpdResult _val584;
-                for (int _i585 = 0; _i585 < _map582.size; ++_i585)
+                org.apache.thrift.protocol.TMap _map614 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map614.size);
+                long _key615;
+                MetadataPpdResult _val616;
+                for (int _i617 = 0; _i617 < _map614.size; ++_i617)
                 {
-                  _key583 = iprot.readI64();
-                  _val584 = new MetadataPpdResult();
-                  _val584.read(iprot);
-                  struct.metadata.put(_key583, _val584);
+                  _key615 = iprot.readI64();
+                  _val616 = new MetadataPpdResult();
+                  _val616.read(iprot);
+                  struct.metadata.put(_key615, _val616);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
         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> _iter586 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter618 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter586.getKey());
-            _iter586.getValue().write(oprot);
+            oprot.writeI64(_iter618.getKey());
+            _iter618.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter587 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter619 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter587.getKey());
-          _iter587.getValue().write(oprot);
+          oprot.writeI64(_iter619.getKey());
+          _iter619.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
     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 _map588 = 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*_map588.size);
-        long _key589;
-        MetadataPpdResult _val590;
-        for (int _i591 = 0; _i591 < _map588.size; ++_i591)
+        org.apache.thrift.protocol.TMap _map620 = 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*_map620.size);
+        long _key621;
+        MetadataPpdResult _val622;
+        for (int _i623 = 0; _i623 < _map620.size; ++_i623)
         {
-          _key589 = iprot.readI64();
-          _val590 = new MetadataPpdResult();
-          _val590.read(iprot);
-          struct.metadata.put(_key589, _val590);
+          _key621 = iprot.readI64();
+          _val622 = new MetadataPpdResult();
+          _val622.read(iprot);
+          struct.metadata.put(_key621, _val622);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index 7aebede..f8c8258 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list610.size);
-                long _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list642.size);
+                long _elem643;
+                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                 {
-                  _elem611 = iprot.readI64();
-                  struct.fileIds.add(_elem611);
+                  _elem643 = iprot.readI64();
+                  struct.fileIds.add(_elem643);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
         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 _iter613 : struct.fileIds)
+          for (long _iter645 : struct.fileIds)
           {
-            oprot.writeI64(_iter613);
+            oprot.writeI64(_iter645);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter614 : struct.fileIds)
+        for (long _iter646 : struct.fileIds)
         {
-          oprot.writeI64(_iter614);
+          oprot.writeI64(_iter646);
         }
       }
     }
@@ -421,13 +421,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
     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 _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list615.size);
-        long _elem616;
-        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list647.size);
+        long _elem648;
+        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
         {
-          _elem616 = iprot.readI64();
-          struct.fileIds.add(_elem616);
+          _elem648 = iprot.readI64();
+          struct.fileIds.add(_elem648);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index fe83a6e..73888e9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map600 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map600.size);
-                long _key601;
-                ByteBuffer _val602;
-                for (int _i603 = 0; _i603 < _map600.size; ++_i603)
+                org.apache.thrift.protocol.TMap _map632 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map632.size);
+                long _key633;
+                ByteBuffer _val634;
+                for (int _i635 = 0; _i635 < _map632.size; ++_i635)
                 {
-                  _key601 = iprot.readI64();
-                  _val602 = iprot.readBinary();
-                  struct.metadata.put(_key601, _val602);
+                  _key633 = iprot.readI64();
+                  _val634 = iprot.readBinary();
+                  struct.metadata.put(_key633, _val634);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
         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> _iter604 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter636 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter604.getKey());
-            oprot.writeBinary(_iter604.getValue());
+            oprot.writeI64(_iter636.getKey());
+            oprot.writeBinary(_iter636.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter605 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter637 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter605.getKey());
-          oprot.writeBinary(_iter605.getValue());
+          oprot.writeI64(_iter637.getKey());
+          oprot.writeBinary(_iter637.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
     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 _map606 = 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*_map606.size);
-        long _key607;
-        ByteBuffer _val608;
-        for (int _i609 = 0; _i609 < _map606.size; ++_i609)
+        org.apache.thrift.protocol.TMap _map638 = 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*_map638.size);
+        long _key639;
+        ByteBuffer _val640;
+        for (int _i641 = 0; _i641 < _map638.size; ++_i641)
         {
-          _key607 = iprot.readI64();
-          _val608 = iprot.readBinary();
-          struct.metadata.put(_key607, _val608);
+          _key639 = iprot.readI64();
+          _val640 = iprot.readBinary();
+          struct.metadata.put(_key639, _val640);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index e68793b..18a8e62 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -447,14 +447,14 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list460.size);
-                TxnInfo _elem461;
-                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
+                org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list492.size);
+                TxnInfo _elem493;
+                for (int _i494 = 0; _i494 < _list492.size; ++_i494)
                 {
-                  _elem461 = new TxnInfo();
-                  _elem461.read(iprot);
-                  struct.open_txns.add(_elem461);
+                  _elem493 = new TxnInfo();
+                  _elem493.read(iprot);
+                  struct.open_txns.add(_elem493);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
         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 _iter463 : struct.open_txns)
+          for (TxnInfo _iter495 : struct.open_txns)
           {
-            _iter463.write(oprot);
+            _iter495.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter464 : struct.open_txns)
+        for (TxnInfo _iter496 : struct.open_txns)
         {
-          _iter464.write(oprot);
+          _iter496.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase<GetOpenT
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list465.size);
-        TxnInfo _elem466;
-        for (int _i467 = 0; _i467 < _list465.size; ++_i467)
+        org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list497.size);
+        TxnInfo _elem498;
+        for (int _i499 = 0; _i499 < _list497.size; ++_i499)
         {
-          _elem466 = new TxnInfo();
-          _elem466.read(iprot);
-          struct.open_txns.add(_elem466);
+          _elem498 = new TxnInfo();
+          _elem498.read(iprot);
+          struct.open_txns.add(_elem498);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 2852310..386c105 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -615,13 +615,13 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list468.size);
-                long _elem469;
-                for (int _i470 = 0; _i470 < _list468.size; ++_i470)
+                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<Long>(_list500.size);
+                long _elem501;
+                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
                 {
-                  _elem469 = iprot.readI64();
-                  struct.open_txns.add(_elem469);
+                  _elem501 = iprot.readI64();
+                  struct.open_txns.add(_elem501);
                 }
                 iprot.readListEnd();
               }
@@ -666,9 +666,9 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
         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 _iter471 : struct.open_txns)
+          for (long _iter503 : struct.open_txns)
           {
-            oprot.writeI64(_iter471);
+            oprot.writeI64(_iter503);
           }
           oprot.writeListEnd();
         }
@@ -704,9 +704,9 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter472 : struct.open_txns)
+        for (long _iter504 : struct.open_txns)
         {
-          oprot.writeI64(_iter472);
+          oprot.writeI64(_iter504);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -726,13 +726,13 @@ public class GetOpenTxnsResponse implements org.apache.thrift.TBase<GetOpenTxnsR
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list473.size);
-        long _elem474;
-        for (int _i475 = 0; _i475 < _list473.size; ++_i475)
+        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new ArrayList<Long>(_list505.size);
+        long _elem506;
+        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
         {
-          _elem474 = iprot.readI64();
-          struct.open_txns.add(_elem474);
+          _elem506 = iprot.readI64();
+          struct.open_txns.add(_elem506);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 225fda9..5195427 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -525,13 +525,13 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list658.size);
-                String _elem659;
-                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                org.apache.thrift.protocol.TList _list690 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list690.size);
+                String _elem691;
+                for (int _i692 = 0; _i692 < _list690.size; ++_i692)
                 {
-                  _elem659 = iprot.readString();
-                  struct.tblNames.add(_elem659);
+                  _elem691 = iprot.readString();
+                  struct.tblNames.add(_elem691);
                 }
                 iprot.readListEnd();
               }
@@ -572,9 +572,9 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
           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 _iter661 : struct.tblNames)
+            for (String _iter693 : struct.tblNames)
             {
-              oprot.writeString(_iter661);
+              oprot.writeString(_iter693);
             }
             oprot.writeListEnd();
           }
@@ -617,9 +617,9 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter662 : struct.tblNames)
+          for (String _iter694 : struct.tblNames)
           {
-            oprot.writeString(_iter662);
+            oprot.writeString(_iter694);
           }
         }
       }
@@ -636,13 +636,13 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list663.size);
-          String _elem664;
-          for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+          org.apache.thrift.protocol.TList _list695 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list695.size);
+          String _elem696;
+          for (int _i697 = 0; _i697 < _list695.size; ++_i697)
           {
-            _elem664 = iprot.readString();
-            struct.tblNames.add(_elem664);
+            _elem696 = iprot.readString();
+            struct.tblNames.add(_elem696);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 91cb198..1ade2f5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list666.size);
-                Table _elem667;
-                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
+                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list698.size);
+                Table _elem699;
+                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
                 {
-                  _elem667 = new Table();
-                  _elem667.read(iprot);
-                  struct.tables.add(_elem667);
+                  _elem699 = new Table();
+                  _elem699.read(iprot);
+                  struct.tables.add(_elem699);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter669 : struct.tables)
+          for (Table _iter701 : struct.tables)
           {
-            _iter669.write(oprot);
+            _iter701.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter670 : struct.tables)
+        for (Table _iter702 : struct.tables)
         {
-          _iter670.write(oprot);
+          _iter702.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
     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 _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list671.size);
-        Table _elem672;
-        for (int _i673 = 0; _i673 < _list671.size; ++_i673)
+        org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list703.size);
+        Table _elem704;
+        for (int _i705 = 0; _i705 < _list703.size; ++_i705)
         {
-          _elem672 = new Table();
-          _elem672.read(iprot);
-          struct.tables.add(_elem672);
+          _elem704 = new Table();
+          _elem704.read(iprot);
+          struct.tables.add(_elem704);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 164ff51..a33210d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set508 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set508.size);
-                long _elem509;
-                for (int _i510 = 0; _i510 < _set508.size; ++_i510)
+                org.apache.thrift.protocol.TSet _set540 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set540.size);
+                long _elem541;
+                for (int _i542 = 0; _i542 < _set540.size; ++_i542)
                 {
-                  _elem509 = iprot.readI64();
-                  struct.aborted.add(_elem509);
+                  _elem541 = iprot.readI64();
+                  struct.aborted.add(_elem541);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set511 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set511.size);
-                long _elem512;
-                for (int _i513 = 0; _i513 < _set511.size; ++_i513)
+                org.apache.thrift.protocol.TSet _set543 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set543.size);
+                long _elem544;
+                for (int _i545 = 0; _i545 < _set543.size; ++_i545)
                 {
-                  _elem512 = iprot.readI64();
-                  struct.nosuch.add(_elem512);
+                  _elem544 = iprot.readI64();
+                  struct.nosuch.add(_elem544);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter514 : struct.aborted)
+          for (long _iter546 : struct.aborted)
           {
-            oprot.writeI64(_iter514);
+            oprot.writeI64(_iter546);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter515 : struct.nosuch)
+          for (long _iter547 : struct.nosuch)
           {
-            oprot.writeI64(_iter515);
+            oprot.writeI64(_iter547);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter516 : struct.aborted)
+        for (long _iter548 : struct.aborted)
         {
-          oprot.writeI64(_iter516);
+          oprot.writeI64(_iter548);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter517 : struct.nosuch)
+        for (long _iter549 : struct.nosuch)
         {
-          oprot.writeI64(_iter517);
+          oprot.writeI64(_iter549);
         }
       }
     }
@@ -560,24 +560,24 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
     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 _set518 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set518.size);
-        long _elem519;
-        for (int _i520 = 0; _i520 < _set518.size; ++_i520)
+        org.apache.thrift.protocol.TSet _set550 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set550.size);
+        long _elem551;
+        for (int _i552 = 0; _i552 < _set550.size; ++_i552)
         {
-          _elem519 = iprot.readI64();
-          struct.aborted.add(_elem519);
+          _elem551 = iprot.readI64();
+          struct.aborted.add(_elem551);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set521 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set521.size);
-        long _elem522;
-        for (int _i523 = 0; _i523 < _set521.size; ++_i523)
+        org.apache.thrift.protocol.TSet _set553 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set553.size);
+        long _elem554;
+        for (int _i555 = 0; _i555 < _set553.size; ++_i555)
         {
-          _elem522 = iprot.readI64();
-          struct.nosuch.add(_elem522);
+          _elem554 = iprot.readI64();
+          struct.nosuch.add(_elem554);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 354e634..84f1df1 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -538,13 +538,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list558.size);
-                String _elem559;
-                for (int _i560 = 0; _i560 < _list558.size; ++_i560)
+                org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list590.size);
+                String _elem591;
+                for (int _i592 = 0; _i592 < _list590.size; ++_i592)
                 {
-                  _elem559 = iprot.readString();
-                  struct.filesAdded.add(_elem559);
+                  _elem591 = iprot.readString();
+                  struct.filesAdded.add(_elem591);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +556,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list561 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list561.size);
-                String _elem562;
-                for (int _i563 = 0; _i563 < _list561.size; ++_i563)
+                org.apache.thrift.protocol.TList _list593 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list593.size);
+                String _elem594;
+                for (int _i595 = 0; _i595 < _list593.size; ++_i595)
                 {
-                  _elem562 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem562);
+                  _elem594 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem594);
                 }
                 iprot.readListEnd();
               }
@@ -593,9 +593,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         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 _iter564 : struct.filesAdded)
+          for (String _iter596 : struct.filesAdded)
           {
-            oprot.writeString(_iter564);
+            oprot.writeString(_iter596);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           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 _iter565 : struct.filesAddedChecksum)
+            for (String _iter597 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter565);
+              oprot.writeString(_iter597);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +634,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter566 : struct.filesAdded)
+        for (String _iter598 : struct.filesAdded)
         {
-          oprot.writeString(_iter566);
+          oprot.writeString(_iter598);
         }
       }
       BitSet optionals = new BitSet();
@@ -653,9 +653,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter567 : struct.filesAddedChecksum)
+          for (String _iter599 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter567);
+            oprot.writeString(_iter599);
           }
         }
       }
@@ -665,13 +665,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     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 _list568 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list568.size);
-        String _elem569;
-        for (int _i570 = 0; _i570 < _list568.size; ++_i570)
+        org.apache.thrift.protocol.TList _list600 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list600.size);
+        String _elem601;
+        for (int _i602 = 0; _i602 < _list600.size; ++_i602)
         {
-          _elem569 = iprot.readString();
-          struct.filesAdded.add(_elem569);
+          _elem601 = iprot.readString();
+          struct.filesAdded.add(_elem601);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -682,13 +682,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list571.size);
-          String _elem572;
-          for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+          org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list603.size);
+          String _elem604;
+          for (int _i605 = 0; _i605 < _list603.size; ++_i605)
           {
-            _elem572 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem572);
+            _elem604 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem604);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index a1a1cac..833adbb 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list492.size);
-                LockComponent _elem493;
-                for (int _i494 = 0; _i494 < _list492.size; ++_i494)
+                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list524.size);
+                LockComponent _elem525;
+                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
                 {
-                  _elem493 = new LockComponent();
-                  _elem493.read(iprot);
-                  struct.component.add(_elem493);
+                  _elem525 = new LockComponent();
+                  _elem525.read(iprot);
+                  struct.component.add(_elem525);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter495 : struct.component)
+          for (LockComponent _iter527 : struct.component)
           {
-            _iter495.write(oprot);
+            _iter527.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter496 : struct.component)
+        for (LockComponent _iter528 : struct.component)
         {
-          _iter496.write(oprot);
+          _iter528.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
     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 _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list497.size);
-        LockComponent _elem498;
-        for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list529.size);
+        LockComponent _elem530;
+        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
         {
-          _elem498 = new LockComponent();
-          _elem498.read(iprot);
-          struct.component.add(_elem498);
+          _elem530 = new LockComponent();
+          _elem530.read(iprot);
+          struct.component.add(_elem530);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
new file mode 100644
index 0000000..d12e270
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.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)")
+public class NotNullConstraintsRequest implements org.apache.thrift.TBase<NotNullConstraintsRequest, NotNullConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<NotNullConstraintsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotNullConstraintsRequest");
+
+  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 NotNullConstraintsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NotNullConstraintsRequestTupleSchemeFactory());
+  }
+
+  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(NotNullConstraintsRequest.class, metaDataMap);
+  }
+
+  public NotNullConstraintsRequest() {
+  }
+
+  public NotNullConstraintsRequest(
+    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 NotNullConstraintsRequest(NotNullConstraintsRequest other) {
+    if (other.isSetDb_name()) {
+      this.db_name = other.db_name;
+    }
+    if (other.isSetTbl_name()) {
+      this.tbl_name = other.tbl_name;
+    }
+  }
+
+  public NotNullConstraintsRequest deepCopy() {
+    return new NotNullConstraintsRequest(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 NotNullConstraintsRequest)
+      return this.equals((NotNullConstraintsRequest)that);
+    return false;
+  }
+
+  public boolean equals(NotNullConstraintsRequest 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(NotNullConstraintsRequest 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("NotNullConstraintsRequest(");
+    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 NotNullConstraintsRequestStandardSchemeFactory implements SchemeFactory {
+    public NotNullConstraintsRequestStandardScheme getScheme() {
+      return new NotNullConstraintsRequestStandardScheme();
+    }
+  }
+
+  private static class NotNullConstraintsRequestStandardScheme extends StandardScheme<NotNullConstraintsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NotNullConstraintsRequest 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, NotNullConstraintsRequest 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 NotNullConstraintsRequestTupleSchemeFactory implements SchemeFactory {
+    public NotNullConstraintsRequestTupleScheme getScheme() {
+      return new NotNullConstraintsRequestTupleScheme();
+    }
+  }
+
+  private static class NotNullConstraintsRequestTupleScheme extends TupleScheme<NotNullConstraintsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsRequest 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, NotNullConstraintsRequest 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);
+    }
+  }
+
+}
+


[22/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index c21ded1..20aeb96 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -174,6 +174,10 @@ class SQLPrimaryKey;
 
 class SQLForeignKey;
 
+class SQLUniqueConstraint;
+
+class SQLNotNullConstraint;
+
 class Type;
 
 class HiveObjectRef;
@@ -272,12 +276,24 @@ class ForeignKeysRequest;
 
 class ForeignKeysResponse;
 
+class UniqueConstraintsRequest;
+
+class UniqueConstraintsResponse;
+
+class NotNullConstraintsRequest;
+
+class NotNullConstraintsResponse;
+
 class DropConstraintRequest;
 
 class AddPrimaryKeyRequest;
 
 class AddForeignKeyRequest;
 
+class AddUniqueConstraintRequest;
+
+class AddNotNullConstraintRequest;
+
 class PartitionsByExprResult;
 
 class PartitionsByExprRequest;
@@ -762,6 +778,176 @@ inline std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj)
   return out;
 }
 
+typedef struct _SQLUniqueConstraint__isset {
+  _SQLUniqueConstraint__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), uk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool table_db :1;
+  bool table_name :1;
+  bool column_name :1;
+  bool key_seq :1;
+  bool uk_name :1;
+  bool enable_cstr :1;
+  bool validate_cstr :1;
+  bool rely_cstr :1;
+} _SQLUniqueConstraint__isset;
+
+class SQLUniqueConstraint {
+ public:
+
+  SQLUniqueConstraint(const SQLUniqueConstraint&);
+  SQLUniqueConstraint& operator=(const SQLUniqueConstraint&);
+  SQLUniqueConstraint() : table_db(), table_name(), column_name(), key_seq(0), uk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  }
+
+  virtual ~SQLUniqueConstraint() throw();
+  std::string table_db;
+  std::string table_name;
+  std::string column_name;
+  int32_t key_seq;
+  std::string uk_name;
+  bool enable_cstr;
+  bool validate_cstr;
+  bool rely_cstr;
+
+  _SQLUniqueConstraint__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_key_seq(const int32_t val);
+
+  void __set_uk_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 SQLUniqueConstraint & 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 (!(key_seq == rhs.key_seq))
+      return false;
+    if (!(uk_name == rhs.uk_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 SQLUniqueConstraint &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SQLUniqueConstraint & ) 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(SQLUniqueConstraint &a, SQLUniqueConstraint &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _SQLNotNullConstraint__isset {
+  _SQLNotNullConstraint__isset() : table_db(false), table_name(false), column_name(false), nn_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool table_db :1;
+  bool table_name :1;
+  bool column_name :1;
+  bool nn_name :1;
+  bool enable_cstr :1;
+  bool validate_cstr :1;
+  bool rely_cstr :1;
+} _SQLNotNullConstraint__isset;
+
+class SQLNotNullConstraint {
+ public:
+
+  SQLNotNullConstraint(const SQLNotNullConstraint&);
+  SQLNotNullConstraint& operator=(const SQLNotNullConstraint&);
+  SQLNotNullConstraint() : table_db(), table_name(), column_name(), nn_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  }
+
+  virtual ~SQLNotNullConstraint() throw();
+  std::string table_db;
+  std::string table_name;
+  std::string column_name;
+  std::string nn_name;
+  bool enable_cstr;
+  bool validate_cstr;
+  bool rely_cstr;
+
+  _SQLNotNullConstraint__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_nn_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 SQLNotNullConstraint & 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 (!(nn_name == rhs.nn_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 SQLNotNullConstraint &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SQLNotNullConstraint & ) 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(SQLNotNullConstraint &a, SQLNotNullConstraint &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _Type__isset {
   _Type__isset() : name(false), type1(false), type2(false), fields(false) {}
   bool name :1;
@@ -3846,6 +4032,176 @@ inline std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& ob
 }
 
 
+class UniqueConstraintsRequest {
+ public:
+
+  UniqueConstraintsRequest(const UniqueConstraintsRequest&);
+  UniqueConstraintsRequest& operator=(const UniqueConstraintsRequest&);
+  UniqueConstraintsRequest() : db_name(), tbl_name() {
+  }
+
+  virtual ~UniqueConstraintsRequest() 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 UniqueConstraintsRequest & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(tbl_name == rhs.tbl_name))
+      return false;
+    return true;
+  }
+  bool operator != (const UniqueConstraintsRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const UniqueConstraintsRequest & ) 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(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const UniqueConstraintsRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class UniqueConstraintsResponse {
+ public:
+
+  UniqueConstraintsResponse(const UniqueConstraintsResponse&);
+  UniqueConstraintsResponse& operator=(const UniqueConstraintsResponse&);
+  UniqueConstraintsResponse() {
+  }
+
+  virtual ~UniqueConstraintsResponse() throw();
+  std::vector<SQLUniqueConstraint>  uniqueConstraints;
+
+  void __set_uniqueConstraints(const std::vector<SQLUniqueConstraint> & val);
+
+  bool operator == (const UniqueConstraintsResponse & rhs) const
+  {
+    if (!(uniqueConstraints == rhs.uniqueConstraints))
+      return false;
+    return true;
+  }
+  bool operator != (const UniqueConstraintsResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const UniqueConstraintsResponse & ) 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(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const UniqueConstraintsResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class NotNullConstraintsRequest {
+ public:
+
+  NotNullConstraintsRequest(const NotNullConstraintsRequest&);
+  NotNullConstraintsRequest& operator=(const NotNullConstraintsRequest&);
+  NotNullConstraintsRequest() : db_name(), tbl_name() {
+  }
+
+  virtual ~NotNullConstraintsRequest() 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 NotNullConstraintsRequest & rhs) const
+  {
+    if (!(db_name == rhs.db_name))
+      return false;
+    if (!(tbl_name == rhs.tbl_name))
+      return false;
+    return true;
+  }
+  bool operator != (const NotNullConstraintsRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const NotNullConstraintsRequest & ) 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(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class NotNullConstraintsResponse {
+ public:
+
+  NotNullConstraintsResponse(const NotNullConstraintsResponse&);
+  NotNullConstraintsResponse& operator=(const NotNullConstraintsResponse&);
+  NotNullConstraintsResponse() {
+  }
+
+  virtual ~NotNullConstraintsResponse() throw();
+  std::vector<SQLNotNullConstraint>  notNullConstraints;
+
+  void __set_notNullConstraints(const std::vector<SQLNotNullConstraint> & val);
+
+  bool operator == (const NotNullConstraintsResponse & rhs) const
+  {
+    if (!(notNullConstraints == rhs.notNullConstraints))
+      return false;
+    return true;
+  }
+  bool operator != (const NotNullConstraintsResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const NotNullConstraintsResponse & ) 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(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const NotNullConstraintsResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class DropConstraintRequest {
  public:
 
@@ -3976,6 +4332,86 @@ inline std::ostream& operator<<(std::ostream& out, const AddForeignKeyRequest& o
 }
 
 
+class AddUniqueConstraintRequest {
+ public:
+
+  AddUniqueConstraintRequest(const AddUniqueConstraintRequest&);
+  AddUniqueConstraintRequest& operator=(const AddUniqueConstraintRequest&);
+  AddUniqueConstraintRequest() {
+  }
+
+  virtual ~AddUniqueConstraintRequest() throw();
+  std::vector<SQLUniqueConstraint>  uniqueConstraintCols;
+
+  void __set_uniqueConstraintCols(const std::vector<SQLUniqueConstraint> & val);
+
+  bool operator == (const AddUniqueConstraintRequest & rhs) const
+  {
+    if (!(uniqueConstraintCols == rhs.uniqueConstraintCols))
+      return false;
+    return true;
+  }
+  bool operator != (const AddUniqueConstraintRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const AddUniqueConstraintRequest & ) 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(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const AddUniqueConstraintRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class AddNotNullConstraintRequest {
+ public:
+
+  AddNotNullConstraintRequest(const AddNotNullConstraintRequest&);
+  AddNotNullConstraintRequest& operator=(const AddNotNullConstraintRequest&);
+  AddNotNullConstraintRequest() {
+  }
+
+  virtual ~AddNotNullConstraintRequest() throw();
+  std::vector<SQLNotNullConstraint>  notNullConstraintCols;
+
+  void __set_notNullConstraintCols(const std::vector<SQLNotNullConstraint> & val);
+
+  bool operator == (const AddNotNullConstraintRequest & rhs) const
+  {
+    if (!(notNullConstraintCols == rhs.notNullConstraintCols))
+      return false;
+    return true;
+  }
+  bool operator != (const AddNotNullConstraintRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const AddNotNullConstraintRequest & ) 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(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class PartitionsByExprResult {
  public:
 

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
index d89eb97..eee1e64 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -351,13 +351,13 @@ public class AbortTxnsRequest implements org.apache.thrift.TBase<AbortTxnsReques
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list484.size);
-                long _elem485;
-                for (int _i486 = 0; _i486 < _list484.size; ++_i486)
+                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list516.size);
+                long _elem517;
+                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
                 {
-                  _elem485 = iprot.readI64();
-                  struct.txn_ids.add(_elem485);
+                  _elem517 = iprot.readI64();
+                  struct.txn_ids.add(_elem517);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class AbortTxnsRequest implements org.apache.thrift.TBase<AbortTxnsReques
         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 _iter487 : struct.txn_ids)
+          for (long _iter519 : struct.txn_ids)
           {
-            oprot.writeI64(_iter487);
+            oprot.writeI64(_iter519);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class AbortTxnsRequest implements org.apache.thrift.TBase<AbortTxnsReques
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter488 : struct.txn_ids)
+        for (long _iter520 : struct.txn_ids)
         {
-          oprot.writeI64(_iter488);
+          oprot.writeI64(_iter520);
         }
       }
     }
@@ -421,13 +421,13 @@ public class AbortTxnsRequest implements org.apache.thrift.TBase<AbortTxnsReques
     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 _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list489.size);
-        long _elem490;
-        for (int _i491 = 0; _i491 < _list489.size; ++_i491)
+        org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list521.size);
+        long _elem522;
+        for (int _i523 = 0; _i523 < _list521.size; ++_i523)
         {
-          _elem490 = iprot.readI64();
-          struct.txn_ids.add(_elem490);
+          _elem522 = iprot.readI64();
+          struct.txn_ids.add(_elem522);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index ba06a56..054cf1b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -727,13 +727,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list542 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list542.size);
-                String _elem543;
-                for (int _i544 = 0; _i544 < _list542.size; ++_i544)
+                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list574.size);
+                String _elem575;
+                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
                 {
-                  _elem543 = iprot.readString();
-                  struct.partitionnames.add(_elem543);
+                  _elem575 = iprot.readString();
+                  struct.partitionnames.add(_elem575);
                 }
                 iprot.readListEnd();
               }
@@ -780,9 +780,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter545 : struct.partitionnames)
+          for (String _iter577 : struct.partitionnames)
           {
-            oprot.writeString(_iter545);
+            oprot.writeString(_iter577);
           }
           oprot.writeListEnd();
         }
@@ -817,9 +817,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter546 : struct.partitionnames)
+        for (String _iter578 : struct.partitionnames)
         {
-          oprot.writeString(_iter546);
+          oprot.writeString(_iter578);
         }
       }
       BitSet optionals = new BitSet();
@@ -842,13 +842,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list547.size);
-        String _elem548;
-        for (int _i549 = 0; _i549 < _list547.size; ++_i549)
+        org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list579.size);
+        String _elem580;
+        for (int _i581 = 0; _i581 < _list579.size; ++_i581)
         {
-          _elem548 = iprot.readString();
-          struct.partitionnames.add(_elem548);
+          _elem580 = iprot.readString();
+          struct.partitionnames.add(_elem580);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
index 43f7ca7..3123787 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
@@ -354,14 +354,14 @@ public class AddForeignKeyRequest implements org.apache.thrift.TBase<AddForeignK
           case 1: // FOREIGN_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
-                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list346.size);
-                SQLForeignKey _elem347;
-                for (int _i348 = 0; _i348 < _list346.size; ++_i348)
+                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)
                 {
-                  _elem347 = new SQLForeignKey();
-                  _elem347.read(iprot);
-                  struct.foreignKeyCols.add(_elem347);
+                  _elem363 = new SQLForeignKey();
+                  _elem363.read(iprot);
+                  struct.foreignKeyCols.add(_elem363);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class AddForeignKeyRequest implements org.apache.thrift.TBase<AddForeignK
         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 _iter349 : struct.foreignKeyCols)
+          for (SQLForeignKey _iter365 : struct.foreignKeyCols)
           {
-            _iter349.write(oprot);
+            _iter365.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class AddForeignKeyRequest implements org.apache.thrift.TBase<AddForeignK
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeyCols.size());
-        for (SQLForeignKey _iter350 : struct.foreignKeyCols)
+        for (SQLForeignKey _iter366 : struct.foreignKeyCols)
         {
-          _iter350.write(oprot);
+          _iter366.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class AddForeignKeyRequest implements org.apache.thrift.TBase<AddForeignK
     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 _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list351.size);
-        SQLForeignKey _elem352;
-        for (int _i353 = 0; _i353 < _list351.size; ++_i353)
+        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)
         {
-          _elem352 = new SQLForeignKey();
-          _elem352.read(iprot);
-          struct.foreignKeyCols.add(_elem352);
+          _elem368 = new SQLForeignKey();
+          _elem368.read(iprot);
+          struct.foreignKeyCols.add(_elem368);
         }
       }
       struct.setForeignKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
new file mode 100644
index 0000000..3b79e98
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.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)")
+public class AddNotNullConstraintRequest implements org.apache.thrift.TBase<AddNotNullConstraintRequest, AddNotNullConstraintRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AddNotNullConstraintRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddNotNullConstraintRequest");
+
+  private static final org.apache.thrift.protocol.TField NOT_NULL_CONSTRAINT_COLS_FIELD_DESC = new org.apache.thrift.protocol.TField("notNullConstraintCols", 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 AddNotNullConstraintRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AddNotNullConstraintRequestTupleSchemeFactory());
+  }
+
+  private List<SQLNotNullConstraint> notNullConstraintCols; // 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 {
+    NOT_NULL_CONSTRAINT_COLS((short)1, "notNullConstraintCols");
+
+    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: // NOT_NULL_CONSTRAINT_COLS
+          return NOT_NULL_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.NOT_NULL_CONSTRAINT_COLS, new org.apache.thrift.meta_data.FieldMetaData("notNullConstraintCols", 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, SQLNotNullConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddNotNullConstraintRequest.class, metaDataMap);
+  }
+
+  public AddNotNullConstraintRequest() {
+  }
+
+  public AddNotNullConstraintRequest(
+    List<SQLNotNullConstraint> notNullConstraintCols)
+  {
+    this();
+    this.notNullConstraintCols = notNullConstraintCols;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AddNotNullConstraintRequest(AddNotNullConstraintRequest other) {
+    if (other.isSetNotNullConstraintCols()) {
+      List<SQLNotNullConstraint> __this__notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(other.notNullConstraintCols.size());
+      for (SQLNotNullConstraint other_element : other.notNullConstraintCols) {
+        __this__notNullConstraintCols.add(new SQLNotNullConstraint(other_element));
+      }
+      this.notNullConstraintCols = __this__notNullConstraintCols;
+    }
+  }
+
+  public AddNotNullConstraintRequest deepCopy() {
+    return new AddNotNullConstraintRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.notNullConstraintCols = null;
+  }
+
+  public int getNotNullConstraintColsSize() {
+    return (this.notNullConstraintCols == null) ? 0 : this.notNullConstraintCols.size();
+  }
+
+  public java.util.Iterator<SQLNotNullConstraint> getNotNullConstraintColsIterator() {
+    return (this.notNullConstraintCols == null) ? null : this.notNullConstraintCols.iterator();
+  }
+
+  public void addToNotNullConstraintCols(SQLNotNullConstraint elem) {
+    if (this.notNullConstraintCols == null) {
+      this.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>();
+    }
+    this.notNullConstraintCols.add(elem);
+  }
+
+  public List<SQLNotNullConstraint> getNotNullConstraintCols() {
+    return this.notNullConstraintCols;
+  }
+
+  public void setNotNullConstraintCols(List<SQLNotNullConstraint> notNullConstraintCols) {
+    this.notNullConstraintCols = notNullConstraintCols;
+  }
+
+  public void unsetNotNullConstraintCols() {
+    this.notNullConstraintCols = null;
+  }
+
+  /** Returns true if field notNullConstraintCols is set (has been assigned a value) and false otherwise */
+  public boolean isSetNotNullConstraintCols() {
+    return this.notNullConstraintCols != null;
+  }
+
+  public void setNotNullConstraintColsIsSet(boolean value) {
+    if (!value) {
+      this.notNullConstraintCols = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NOT_NULL_CONSTRAINT_COLS:
+      if (value == null) {
+        unsetNotNullConstraintCols();
+      } else {
+        setNotNullConstraintCols((List<SQLNotNullConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NOT_NULL_CONSTRAINT_COLS:
+      return getNotNullConstraintCols();
+
+    }
+    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 NOT_NULL_CONSTRAINT_COLS:
+      return isSetNotNullConstraintCols();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AddNotNullConstraintRequest)
+      return this.equals((AddNotNullConstraintRequest)that);
+    return false;
+  }
+
+  public boolean equals(AddNotNullConstraintRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_notNullConstraintCols = true && this.isSetNotNullConstraintCols();
+    boolean that_present_notNullConstraintCols = true && that.isSetNotNullConstraintCols();
+    if (this_present_notNullConstraintCols || that_present_notNullConstraintCols) {
+      if (!(this_present_notNullConstraintCols && that_present_notNullConstraintCols))
+        return false;
+      if (!this.notNullConstraintCols.equals(that.notNullConstraintCols))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_notNullConstraintCols = true && (isSetNotNullConstraintCols());
+    list.add(present_notNullConstraintCols);
+    if (present_notNullConstraintCols)
+      list.add(notNullConstraintCols);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AddNotNullConstraintRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetNotNullConstraintCols()).compareTo(other.isSetNotNullConstraintCols());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNotNullConstraintCols()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.notNullConstraintCols, other.notNullConstraintCols);
+      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("AddNotNullConstraintRequest(");
+    boolean first = true;
+
+    sb.append("notNullConstraintCols:");
+    if (this.notNullConstraintCols == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.notNullConstraintCols);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetNotNullConstraintCols()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'notNullConstraintCols' 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 AddNotNullConstraintRequestStandardSchemeFactory implements SchemeFactory {
+    public AddNotNullConstraintRequestStandardScheme getScheme() {
+      return new AddNotNullConstraintRequestStandardScheme();
+    }
+  }
+
+  private static class AddNotNullConstraintRequestStandardScheme extends StandardScheme<AddNotNullConstraintRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AddNotNullConstraintRequest 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: // 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)
+                {
+                  _elem379 = new SQLNotNullConstraint();
+                  _elem379.read(iprot);
+                  struct.notNullConstraintCols.add(_elem379);
+                }
+                iprot.readListEnd();
+              }
+              struct.setNotNullConstraintColsIsSet(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, AddNotNullConstraintRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.notNullConstraintCols != null) {
+        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)
+          {
+            _iter381.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AddNotNullConstraintRequestTupleSchemeFactory implements SchemeFactory {
+    public AddNotNullConstraintRequestTupleScheme getScheme() {
+      return new AddNotNullConstraintRequestTupleScheme();
+    }
+  }
+
+  private static class AddNotNullConstraintRequestTupleScheme extends TupleScheme<AddNotNullConstraintRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AddNotNullConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.notNullConstraintCols.size());
+        for (SQLNotNullConstraint _iter382 : struct.notNullConstraintCols)
+        {
+          _iter382.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    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)
+        {
+          _elem384 = new SQLNotNullConstraint();
+          _elem384.read(iprot);
+          struct.notNullConstraintCols.add(_elem384);
+        }
+      }
+      struct.setNotNullConstraintColsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index da23f72..6e9ac48 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -704,14 +704,14 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
           case 3: // PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list420 = iprot.readListBegin();
-                struct.parts = new ArrayList<Partition>(_list420.size);
-                Partition _elem421;
-                for (int _i422 = 0; _i422 < _list420.size; ++_i422)
+                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)
                 {
-                  _elem421 = new Partition();
-                  _elem421.read(iprot);
-                  struct.parts.add(_elem421);
+                  _elem453 = new Partition();
+                  _elem453.read(iprot);
+                  struct.parts.add(_elem453);
                 }
                 iprot.readListEnd();
               }
@@ -763,9 +763,9 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
         oprot.writeFieldBegin(PARTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.parts.size()));
-          for (Partition _iter423 : struct.parts)
+          for (Partition _iter455 : struct.parts)
           {
-            _iter423.write(oprot);
+            _iter455.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -800,9 +800,9 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.parts.size());
-        for (Partition _iter424 : struct.parts)
+        for (Partition _iter456 : struct.parts)
         {
-          _iter424.write(oprot);
+          _iter456.write(oprot);
         }
       }
       oprot.writeBool(struct.ifNotExists);
@@ -824,14 +824,14 @@ public class AddPartitionsRequest implements org.apache.thrift.TBase<AddPartitio
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list425 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.parts = new ArrayList<Partition>(_list425.size);
-        Partition _elem426;
-        for (int _i427 = 0; _i427 < _list425.size; ++_i427)
+        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)
         {
-          _elem426 = new Partition();
-          _elem426.read(iprot);
-          struct.parts.add(_elem426);
+          _elem458 = new Partition();
+          _elem458.read(iprot);
+          struct.parts.add(_elem458);
         }
       }
       struct.setPartsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index bfd483e..0dfed78 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@ -346,14 +346,14 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list412 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list412.size);
-                Partition _elem413;
-                for (int _i414 = 0; _i414 < _list412.size; ++_i414)
+                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)
                 {
-                  _elem413 = new Partition();
-                  _elem413.read(iprot);
-                  struct.partitions.add(_elem413);
+                  _elem445 = new Partition();
+                  _elem445.read(iprot);
+                  struct.partitions.add(_elem445);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter415 : struct.partitions)
+            for (Partition _iter447 : struct.partitions)
             {
-              _iter415.write(oprot);
+              _iter447.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter416 : struct.partitions)
+          for (Partition _iter448 : struct.partitions)
           {
-            _iter416.write(oprot);
+            _iter448.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class AddPartitionsResult implements org.apache.thrift.TBase<AddPartition
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list417 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list417.size);
-          Partition _elem418;
-          for (int _i419 = 0; _i419 < _list417.size; ++_i419)
+          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)
           {
-            _elem418 = new Partition();
-            _elem418.read(iprot);
-            struct.partitions.add(_elem418);
+            _elem450 = new Partition();
+            _elem450.read(iprot);
+            struct.partitions.add(_elem450);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
index 987b031..55e606d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
@@ -354,14 +354,14 @@ public class AddPrimaryKeyRequest implements org.apache.thrift.TBase<AddPrimaryK
           case 1: // PRIMARY_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
-                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list338.size);
-                SQLPrimaryKey _elem339;
-                for (int _i340 = 0; _i340 < _list338.size; ++_i340)
+                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)
                 {
-                  _elem339 = new SQLPrimaryKey();
-                  _elem339.read(iprot);
-                  struct.primaryKeyCols.add(_elem339);
+                  _elem355 = new SQLPrimaryKey();
+                  _elem355.read(iprot);
+                  struct.primaryKeyCols.add(_elem355);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class AddPrimaryKeyRequest implements org.apache.thrift.TBase<AddPrimaryK
         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 _iter341 : struct.primaryKeyCols)
+          for (SQLPrimaryKey _iter357 : struct.primaryKeyCols)
           {
-            _iter341.write(oprot);
+            _iter357.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class AddPrimaryKeyRequest implements org.apache.thrift.TBase<AddPrimaryK
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeyCols.size());
-        for (SQLPrimaryKey _iter342 : struct.primaryKeyCols)
+        for (SQLPrimaryKey _iter358 : struct.primaryKeyCols)
         {
-          _iter342.write(oprot);
+          _iter358.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class AddPrimaryKeyRequest implements org.apache.thrift.TBase<AddPrimaryK
     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 _list343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list343.size);
-        SQLPrimaryKey _elem344;
-        for (int _i345 = 0; _i345 < _list343.size; ++_i345)
+        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)
         {
-          _elem344 = new SQLPrimaryKey();
-          _elem344.read(iprot);
-          struct.primaryKeyCols.add(_elem344);
+          _elem360 = new SQLPrimaryKey();
+          _elem360.read(iprot);
+          struct.primaryKeyCols.add(_elem360);
         }
       }
       struct.setPrimaryKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
new file mode 100644
index 0000000..71d1451
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.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)")
+public class AddUniqueConstraintRequest implements org.apache.thrift.TBase<AddUniqueConstraintRequest, AddUniqueConstraintRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AddUniqueConstraintRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AddUniqueConstraintRequest");
+
+  private static final org.apache.thrift.protocol.TField UNIQUE_CONSTRAINT_COLS_FIELD_DESC = new org.apache.thrift.protocol.TField("uniqueConstraintCols", 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 AddUniqueConstraintRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AddUniqueConstraintRequestTupleSchemeFactory());
+  }
+
+  private List<SQLUniqueConstraint> uniqueConstraintCols; // 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 {
+    UNIQUE_CONSTRAINT_COLS((short)1, "uniqueConstraintCols");
+
+    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: // UNIQUE_CONSTRAINT_COLS
+          return UNIQUE_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.UNIQUE_CONSTRAINT_COLS, new org.apache.thrift.meta_data.FieldMetaData("uniqueConstraintCols", 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, SQLUniqueConstraint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddUniqueConstraintRequest.class, metaDataMap);
+  }
+
+  public AddUniqueConstraintRequest() {
+  }
+
+  public AddUniqueConstraintRequest(
+    List<SQLUniqueConstraint> uniqueConstraintCols)
+  {
+    this();
+    this.uniqueConstraintCols = uniqueConstraintCols;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AddUniqueConstraintRequest(AddUniqueConstraintRequest other) {
+    if (other.isSetUniqueConstraintCols()) {
+      List<SQLUniqueConstraint> __this__uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(other.uniqueConstraintCols.size());
+      for (SQLUniqueConstraint other_element : other.uniqueConstraintCols) {
+        __this__uniqueConstraintCols.add(new SQLUniqueConstraint(other_element));
+      }
+      this.uniqueConstraintCols = __this__uniqueConstraintCols;
+    }
+  }
+
+  public AddUniqueConstraintRequest deepCopy() {
+    return new AddUniqueConstraintRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.uniqueConstraintCols = null;
+  }
+
+  public int getUniqueConstraintColsSize() {
+    return (this.uniqueConstraintCols == null) ? 0 : this.uniqueConstraintCols.size();
+  }
+
+  public java.util.Iterator<SQLUniqueConstraint> getUniqueConstraintColsIterator() {
+    return (this.uniqueConstraintCols == null) ? null : this.uniqueConstraintCols.iterator();
+  }
+
+  public void addToUniqueConstraintCols(SQLUniqueConstraint elem) {
+    if (this.uniqueConstraintCols == null) {
+      this.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>();
+    }
+    this.uniqueConstraintCols.add(elem);
+  }
+
+  public List<SQLUniqueConstraint> getUniqueConstraintCols() {
+    return this.uniqueConstraintCols;
+  }
+
+  public void setUniqueConstraintCols(List<SQLUniqueConstraint> uniqueConstraintCols) {
+    this.uniqueConstraintCols = uniqueConstraintCols;
+  }
+
+  public void unsetUniqueConstraintCols() {
+    this.uniqueConstraintCols = null;
+  }
+
+  /** Returns true if field uniqueConstraintCols is set (has been assigned a value) and false otherwise */
+  public boolean isSetUniqueConstraintCols() {
+    return this.uniqueConstraintCols != null;
+  }
+
+  public void setUniqueConstraintColsIsSet(boolean value) {
+    if (!value) {
+      this.uniqueConstraintCols = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case UNIQUE_CONSTRAINT_COLS:
+      if (value == null) {
+        unsetUniqueConstraintCols();
+      } else {
+        setUniqueConstraintCols((List<SQLUniqueConstraint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case UNIQUE_CONSTRAINT_COLS:
+      return getUniqueConstraintCols();
+
+    }
+    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 UNIQUE_CONSTRAINT_COLS:
+      return isSetUniqueConstraintCols();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AddUniqueConstraintRequest)
+      return this.equals((AddUniqueConstraintRequest)that);
+    return false;
+  }
+
+  public boolean equals(AddUniqueConstraintRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_uniqueConstraintCols = true && this.isSetUniqueConstraintCols();
+    boolean that_present_uniqueConstraintCols = true && that.isSetUniqueConstraintCols();
+    if (this_present_uniqueConstraintCols || that_present_uniqueConstraintCols) {
+      if (!(this_present_uniqueConstraintCols && that_present_uniqueConstraintCols))
+        return false;
+      if (!this.uniqueConstraintCols.equals(that.uniqueConstraintCols))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_uniqueConstraintCols = true && (isSetUniqueConstraintCols());
+    list.add(present_uniqueConstraintCols);
+    if (present_uniqueConstraintCols)
+      list.add(uniqueConstraintCols);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AddUniqueConstraintRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetUniqueConstraintCols()).compareTo(other.isSetUniqueConstraintCols());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUniqueConstraintCols()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uniqueConstraintCols, other.uniqueConstraintCols);
+      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("AddUniqueConstraintRequest(");
+    boolean first = true;
+
+    sb.append("uniqueConstraintCols:");
+    if (this.uniqueConstraintCols == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.uniqueConstraintCols);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetUniqueConstraintCols()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uniqueConstraintCols' 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 AddUniqueConstraintRequestStandardSchemeFactory implements SchemeFactory {
+    public AddUniqueConstraintRequestStandardScheme getScheme() {
+      return new AddUniqueConstraintRequestStandardScheme();
+    }
+  }
+
+  private static class AddUniqueConstraintRequestStandardScheme extends StandardScheme<AddUniqueConstraintRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AddUniqueConstraintRequest 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: // 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)
+                {
+                  _elem371 = new SQLUniqueConstraint();
+                  _elem371.read(iprot);
+                  struct.uniqueConstraintCols.add(_elem371);
+                }
+                iprot.readListEnd();
+              }
+              struct.setUniqueConstraintColsIsSet(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, AddUniqueConstraintRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.uniqueConstraintCols != null) {
+        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)
+          {
+            _iter373.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AddUniqueConstraintRequestTupleSchemeFactory implements SchemeFactory {
+    public AddUniqueConstraintRequestTupleScheme getScheme() {
+      return new AddUniqueConstraintRequestTupleScheme();
+    }
+  }
+
+  private static class AddUniqueConstraintRequestTupleScheme extends TupleScheme<AddUniqueConstraintRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AddUniqueConstraintRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.uniqueConstraintCols.size());
+        for (SQLUniqueConstraint _iter374 : struct.uniqueConstraintCols)
+        {
+          _iter374.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    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)
+        {
+          _elem376 = new SQLUniqueConstraint();
+          _elem376.read(iprot);
+          struct.uniqueConstraintCols.add(_elem376);
+        }
+      }
+      struct.setUniqueConstraintColsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index 0da09bf..0b6574d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list634.size);
-                long _elem635;
-                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list666.size);
+                long _elem667;
+                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                 {
-                  _elem635 = iprot.readI64();
-                  struct.fileIds.add(_elem635);
+                  _elem667 = iprot.readI64();
+                  struct.fileIds.add(_elem667);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
         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 _iter637 : struct.fileIds)
+          for (long _iter669 : struct.fileIds)
           {
-            oprot.writeI64(_iter637);
+            oprot.writeI64(_iter669);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter638 : struct.fileIds)
+        for (long _iter670 : struct.fileIds)
         {
-          oprot.writeI64(_iter638);
+          oprot.writeI64(_iter670);
         }
       }
     }
@@ -421,13 +421,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
     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 _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list639.size);
-        long _elem640;
-        for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+        org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list671.size);
+        long _elem672;
+        for (int _i673 = 0; _i673 < _list671.size; ++_i673)
         {
-          _elem640 = iprot.readI64();
-          struct.fileIds.add(_elem640);
+          _elem672 = iprot.readI64();
+          struct.fileIds.add(_elem672);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 81534fe..19e671b 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list650.size);
-                ClientCapability _elem651;
-                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list682.size);
+                ClientCapability _elem683;
+                for (int _i684 = 0; _i684 < _list682.size; ++_i684)
                 {
-                  _elem651 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem651);
+                  _elem683 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem683);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter653 : struct.values)
+          for (ClientCapability _iter685 : struct.values)
           {
-            oprot.writeI32(_iter653.getValue());
+            oprot.writeI32(_iter685.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter654 : struct.values)
+        for (ClientCapability _iter686 : struct.values)
         {
-          oprot.writeI32(_iter654.getValue());
+          oprot.writeI32(_iter686.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
     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 _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list655.size);
-        ClientCapability _elem656;
-        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+        org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list687.size);
+        ClientCapability _elem688;
+        for (int _i689 = 0; _i689 < _list687.size; ++_i689)
         {
-          _elem656 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem656);
+          _elem688 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem688);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index d3fc92a..3acb203 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map524 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map524.size);
-                String _key525;
-                String _val526;
-                for (int _i527 = 0; _i527 < _map524.size; ++_i527)
+                org.apache.thrift.protocol.TMap _map556 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map556.size);
+                String _key557;
+                String _val558;
+                for (int _i559 = 0; _i559 < _map556.size; ++_i559)
                 {
-                  _key525 = iprot.readString();
-                  _val526 = iprot.readString();
-                  struct.properties.put(_key525, _val526);
+                  _key557 = iprot.readString();
+                  _val558 = iprot.readString();
+                  struct.properties.put(_key557, _val558);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
           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> _iter528 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter560 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter528.getKey());
-              oprot.writeString(_iter528.getValue());
+              oprot.writeString(_iter560.getKey());
+              oprot.writeString(_iter560.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter529 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter561 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter529.getKey());
-            oprot.writeString(_iter529.getValue());
+            oprot.writeString(_iter561.getKey());
+            oprot.writeString(_iter561.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map530 = 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*_map530.size);
-          String _key531;
-          String _val532;
-          for (int _i533 = 0; _i533 < _map530.size; ++_i533)
+          org.apache.thrift.protocol.TMap _map562 = 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*_map562.size);
+          String _key563;
+          String _val564;
+          for (int _i565 = 0; _i565 < _map562.size; ++_i565)
           {
-            _key531 = iprot.readString();
-            _val532 = iprot.readString();
-            struct.properties.put(_key531, _val532);
+            _key563 = iprot.readString();
+            _val564 = iprot.readString();
+            struct.properties.put(_key563, _val564);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index 96cfbd2..21290ee 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@ -346,14 +346,14 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list428.size);
-                Partition _elem429;
-                for (int _i430 = 0; _i430 < _list428.size; ++_i430)
+                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)
                 {
-                  _elem429 = new Partition();
-                  _elem429.read(iprot);
-                  struct.partitions.add(_elem429);
+                  _elem461 = new Partition();
+                  _elem461.read(iprot);
+                  struct.partitions.add(_elem461);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter431 : struct.partitions)
+            for (Partition _iter463 : struct.partitions)
             {
-              _iter431.write(oprot);
+              _iter463.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter432 : struct.partitions)
+          for (Partition _iter464 : struct.partitions)
           {
-            _iter432.write(oprot);
+            _iter464.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class DropPartitionsResult implements org.apache.thrift.TBase<DropPartiti
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list433 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list433.size);
-          Partition _elem434;
-          for (int _i435 = 0; _i435 < _list433.size; ++_i435)
+          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)
           {
-            _elem434 = new Partition();
-            _elem434.read(iprot);
-            struct.partitions.add(_elem434);
+            _elem466 = new Partition();
+            _elem466.read(iprot);
+            struct.partitions.add(_elem466);
           }
         }
         struct.setPartitionsIsSet(true);


[29/31] hive git commit: HIVE-16760 : Update errata.txt for HIVE-16743 (Wei Zheng)

Posted by we...@apache.org.
HIVE-16760 : Update errata.txt for HIVE-16743 (Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: 811e5995188ba857d2f1a2a1d3c16b9ce84977f2
Parents: b185774
Author: Wei Zheng <we...@apache.org>
Authored: Thu May 25 11:20:49 2017 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Thu May 25 11:20:49 2017 -0700

----------------------------------------------------------------------
 errata.txt | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/811e5995/errata.txt
----------------------------------------------------------------------
diff --git a/errata.txt b/errata.txt
index 949ed9a..c6c177a 100644
--- a/errata.txt
+++ b/errata.txt
@@ -89,4 +89,5 @@ d8298e1c85a515150562b0df68af89c18c468638 llap       HIVE-9418  https://issues.ap
 d16d4f1bcc43d6ebcab0eaf5bc635fb88b60be5f master     HIVE-9423  https://issues.apache.org/jira/browse/HIVE-9423
 130617443bb05d79c18420c0c4e903a76da3651c master     HIVE-14909 https://issues.apache.org/jira/browse/HIVE-14909
 6dace60af4b6ab4d5200310a0ad94c4530c2bec3 master     HIVE-13335 https://issues.apache.org/jira/browse/HIVE-13335
-5facfbb863366d7a661c21c57011b8dbe43f52e0 master     HIVE-16307 https://issues.apache.org/jira/browse/HIVE-16307
\ No newline at end of file
+5facfbb863366d7a661c21c57011b8dbe43f52e0 master     HIVE-16307 https://issues.apache.org/jira/browse/HIVE-16307
+1c3039333ba71665e8b954fbee88188757bb4050 master     HIVE-16743 https://issues.apache.org/jira/browse/HIVE-16743


[14/31] hive git commit: HIVE-16575: Support for 'UNIQUE' and 'NOT NULL' constraints (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/696be9f5/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 9faf830..67e9422 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -42,10 +42,12 @@ 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)')
+  print('  void create_table_with_constraints(Table tbl,  primaryKeys,  foreignKeys,  uniqueConstraints,  notNullConstraints)')
   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 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)')
@@ -111,6 +113,8 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('   get_index_names(string db_name, string tbl_name, i16 max_indexes)')
   print('  PrimaryKeysResponse get_primary_keys(PrimaryKeysRequest request)')
   print('  ForeignKeysResponse get_foreign_keys(ForeignKeysRequest request)')
+  print('  UniqueConstraintsResponse get_unique_constraints(UniqueConstraintsRequest request)')
+  print('  NotNullConstraintsResponse get_not_null_constraints(NotNullConstraintsRequest 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)')
@@ -357,10 +361,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) != 3:
-    print('create_table_with_constraints requires 3 args')
+  if len(args) != 5:
+    print('create_table_with_constraints requires 5 args')
     sys.exit(1)
-  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),))
+  pp.pprint(client.create_table_with_constraints(eval(args[0]),eval(args[1]),eval(args[2]),eval(args[3]),eval(args[4]),))
 
 elif cmd == 'drop_constraint':
   if len(args) != 1:
@@ -380,6 +384,18 @@ elif cmd == 'add_foreign_key':
     sys.exit(1)
   pp.pprint(client.add_foreign_key(eval(args[0]),))
 
+elif cmd == 'add_unique_constraint':
+  if len(args) != 1:
+    print('add_unique_constraint requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_unique_constraint(eval(args[0]),))
+
+elif cmd == 'add_not_null_constraint':
+  if len(args) != 1:
+    print('add_not_null_constraint requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_not_null_constraint(eval(args[0]),))
+
 elif cmd == 'drop_table':
   if len(args) != 3:
     print('drop_table requires 3 args')
@@ -770,6 +786,18 @@ elif cmd == 'get_foreign_keys':
     sys.exit(1)
   pp.pprint(client.get_foreign_keys(eval(args[0]),))
 
+elif cmd == 'get_unique_constraints':
+  if len(args) != 1:
+    print('get_unique_constraints requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_unique_constraints(eval(args[0]),))
+
+elif cmd == 'get_not_null_constraints':
+  if len(args) != 1:
+    print('get_not_null_constraints requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_not_null_constraints(eval(args[0]),))
+
 elif cmd == 'update_table_column_statistics':
   if len(args) != 1:
     print('update_table_column_statistics requires 1 args')


[06/31] hive git commit: HIVE-16554: ACID: Make HouseKeeperService threads daemon (Vaibhav Gumashta reviewed by Eugene Koifman)

Posted by we...@apache.org.
HIVE-16554: ACID: Make HouseKeeperService threads daemon (Vaibhav Gumashta reviewed by Eugene Koifman)


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

Branch: refs/heads/hive-14535
Commit: 823f01c386989eed2b5c60492f16140521b583a8
Parents: 3c1b354
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Thu May 25 00:53:09 2017 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Thu May 25 00:53:09 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java   | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/823f01c3/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
index 0b7332c..0aa160c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
@@ -48,7 +48,11 @@ public abstract class HouseKeeperServiceBase implements HouseKeeperService {
       private final AtomicInteger threadCounter = new AtomicInteger();
       @Override
       public Thread newThread(Runnable r) {
-        return new Thread(r, HouseKeeperServiceBase.this.getClass().getName() + "-" + threadCounter.getAndIncrement());
+        Thread t =
+            new Thread(r, HouseKeeperServiceBase.this.getClass().getName() + "-"
+                + threadCounter.getAndIncrement());
+        t.setDaemon(true);
+        return t;
       }
     });