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:49 UTC

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

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';