You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2016/04/26 19:54:01 UTC

[38/50] [abbrv] hive git commit: HIVE-13290: Support primary keys/foreign keys constraint as part of create table command in Hive (Hari Subramaniyan, reviewed by Ashutosh Chauhan)

HIVE-13290: Support primary keys/foreign keys constraint as part of create table command in Hive (Hari Subramaniyan, 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/53249a35
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/53249a35
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/53249a35

Branch: refs/heads/llap
Commit: 53249a3579dce000736a05348b64faed32fb610c
Parents: caa3ec7
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Mon Apr 25 13:57:17 2016 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Mon Apr 25 13:57:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/ObjectStore.java      |  41 +++-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   3 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  11 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  24 ++-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     | 209 +++++++++++++++++--
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   9 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  92 +++++++-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  11 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  10 +-
 .../hadoop/hive/ql/plan/CreateTableDesc.java    |  43 +++-
 .../hive/ql/parse/TestHiveDecimalParse.java     |   2 +-
 .../create_with_constraints_duplicate_name.q    |   2 +
 .../create_with_constraints_enable.q            |   1 +
 .../create_with_constraints_validate.q          |   1 +
 .../clientpositive/create_with_constraints.q    |  12 ++
 ...create_with_constraints_duplicate_name.q.out |  13 ++
 .../create_with_constraints_enable.q.out        |   1 +
 .../create_with_constraints_validate.q.out      |   1 +
 .../create_with_constraints.q.out               |  68 ++++++
 20 files changed, 520 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 ae6f084..24fbf70 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1022,6 +1022,11 @@ public class ObjectStore implements RawStore, Configurable {
           " table " + tableName + " record to delete");
         }
 
+        List<MConstraint> tabConstraints = listAllTableConstraints(dbName, tableName);
+        if (tabConstraints != null && tabConstraints.size() > 0) {
+          pm.deletePersistentAll(tabConstraints);
+        }
+
         preDropStorageDescriptor(tbl.getSd());
         // then remove the table
         pm.deletePersistentAll(tbl);
@@ -1035,7 +1040,41 @@ public class ObjectStore implements RawStore, Configurable {
     return success;
   }
 
-  @Override
+  private List<MConstraint> listAllTableConstraints(String dbName, String tableName) {
+    List<MConstraint> mConstraints = null;
+    List<String> constraintNames = new ArrayList<String>();
+    Query query = null;
+
+    try {
+      query = pm.newQuery("select constraintName from org.apache.hadoop.hive.metastore.model.MConstraint  where "
+        + "(parentTable.tableName == ptblname && parentTable.database.name == pdbname) || "
+        + "(childTable != null && childTable.tableName == ctblname && childTable.database.name == cdbname)");
+      query.declareParameters("java.lang.String ptblname, java.lang.String pdbname,"
+      + "java.lang.String ctblname, java.lang.String cdbname");
+      Collection<?> constraintNamesColl = (Collection<?>) query.
+        executeWithArray(tableName, dbName, tableName, dbName);
+      for (Iterator<?> i = constraintNamesColl.iterator(); i.hasNext();) {
+        String currName = (String) i.next();
+        constraintNames.add(currName);
+      }
+      query = pm.newQuery(MConstraint.class);
+      query.setFilter("param.contains(constraintName)");
+      query.declareParameters("java.util.Collection param");
+      Collection<?> constraints = (Collection<?>)query.execute(constraintNames);
+      mConstraints = new ArrayList<MConstraint>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currConstraint = (MConstraint) i.next();
+        mConstraints.add(currConstraint);
+      }
+    } finally {
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return mConstraints;
+  }
+
+@Override
   public Table getTable(String dbName, String tableName) throws MetaException {
     boolean commited = false;
     Table tbl = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 f091f67..1de3309 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -447,6 +447,8 @@ public enum ErrorMsg {
   INVALID_LOAD_TABLE_FILE_WORK(10322, "Invalid Load Table Work or Load File Work"),
   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"),
   //========================== 20000 range starts here ========================//
   SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."),
   SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. "
@@ -463,7 +465,6 @@ public enum ErrorMsg {
   OP_NOT_ALLOWED_IN_AUTOCOMMIT(20006, "Operation {0} is not allowed when autoCommit=true.", true),//todo: better SQLState?
   OP_NOT_ALLOWED_IN_TXN(20007, "Operation {0} is not allowed in a transaction.  TransactionID={1}.", true),
   OP_NOT_ALLOWED_WITHOUT_TXN(20008, "Operation {0} is not allowed since autoCommit=false and there is no active transaction", true),
-
   //========================== 30000 range starts here ========================//
   STATSPUBLISHER_NOT_OBTAINED(30000, "StatsPublisher cannot be obtained. " +
     "There was a error to retrieve the StatsPublisher, and retrying " +

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 d2c3ca8..cbeb361 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
@@ -78,6 +78,8 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
@@ -3926,6 +3928,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   private int createTable(Hive db, CreateTableDesc crtTbl) throws HiveException {
     // create the table
     Table tbl = crtTbl.toTable(conf);
+    List<SQLPrimaryKey> primaryKeys = crtTbl.getPrimaryKeys();
+    List<SQLForeignKey> foreignKeys = crtTbl.getForeignKeys();
     LOG.info("creating table " + tbl.getDbName() + "." + tbl.getTableName() + " on " +
             tbl.getDataLocation());
 
@@ -3938,7 +3942,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         throw new HiveException("Unable to alter table. " + e.getMessage(), e);
       }
     } else {
-      db.createTable(tbl, crtTbl.getIfNotExists());
+      if ((foreignKeys != null && foreignKeys.size() > 0 ) ||
+          (primaryKeys != null && primaryKeys.size() > 0)) {
+        db.createTable(tbl, crtTbl.getIfNotExists(), primaryKeys, foreignKeys);
+      } else {
+        db.createTable(tbl, crtTbl.getIfNotExists());
+      }
       if ( crtTbl.isCTAS()) {
         Table createdTable = db.getTable(tbl.getDbName(), tbl.getTableName());
         DataContainer dc = new DataContainer(createdTable.getTTable());

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 4c9acce..ab165f1 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
@@ -104,6 +104,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
@@ -780,15 +782,21 @@ public class Hive {
   }
 
   /**
-   * Creates the table with the give objects
+   * Creates the table with the given objects. It takes additional arguments for
+   * primary keys and foreign keys associated with the table.
    *
    * @param tbl
    *          a table object
    * @param ifNotExists
    *          if true, ignore AlreadyExistsException
+   * @param primaryKeys
+   *          primary key columns associated with the table
+   * @param foreignKeys
+   *          foreign key columns associated with the table
    * @throws HiveException
    */
-  public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
+  public void createTable(Table tbl, boolean ifNotExists,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys)  throws HiveException {
     try {
       if (tbl.getDbName() == null || "".equals(tbl.getDbName().trim())) {
         tbl.setDbName(SessionState.get().getCurrentDatabase());
@@ -813,7 +821,12 @@ public class Hive {
           tTbl.setPrivileges(principalPrivs);
         }
       }
-      getMSC().createTable(tTbl);
+      if (primaryKeys == null && foreignKeys == null) {
+        getMSC().createTable(tTbl);
+      } else {
+        getMSC().createTableWithConstraints(tTbl, primaryKeys, foreignKeys);
+      }
+
     } catch (AlreadyExistsException e) {
       if (!ifNotExists) {
         throw new HiveException(e);
@@ -823,6 +836,10 @@ public class Hive {
     }
   }
 
+  public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
+   createTable(tbl, ifNotExists, null, null);
+ }
+
   public static List<FieldSchema> getFieldsFromDeserializerForMsStorage(
       Table tbl, Deserializer deserializer) throws SerDeException, MetaException {
     List<FieldSchema> schema = MetaStoreUtils.getFieldsFromDeserializer(
@@ -3583,4 +3600,5 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e);
     }
   }
+
 };

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 cbb1b0a..7fcbd6a 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
@@ -44,6 +44,8 @@ import org.apache.hadoop.hive.metastore.HiveMetaStore;
 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.SQLPrimaryKey;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -617,29 +619,204 @@ 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>());
+  }
+
+  static class PKInfo {
+   public String colName;
+   public String constraintName;
+   public boolean rely;
+
+   public PKInfo(String colName, String constraintName, boolean rely) {
+     this.colName = colName;
+     this.constraintName = constraintName;
+     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
+   */
+  private static void processPrimaryKeyInfos(
+    ASTNode child, List<PKInfo> pkInfos) throws SemanticException {
+    if (child.getChildCount() < 6) {
+      throw new SemanticException(ErrorMsg.INVALID_PK_SYNTAX.getMsg());
+    }
+    // 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() == 7)
+    // 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() == 6)
+    boolean userSpecifiedConstraintName = child.getChildCount() == 7;
+    int relyIndex =  child.getChildCount() == 7 ? 4 : 3;
+    for (int j = 0; j < child.getChild(1).getChildCount(); j++) {
+     Tree grandChild = child.getChild(1).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"));
+     }
+     pkInfos.add(
+       new PKInfo(
+         unescapeIdentifier(grandChild.getText().toLowerCase()),
+         (userSpecifiedConstraintName ?
+         unescapeIdentifier(child.getChild(3).getText().toLowerCase()) : null),
+         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
+   * @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));
+      }
+    }
+  }
+
+  /**
+   * Process the foreign keys from the AST and populate the foreign keys in the SQLForeignKey list
+   * @param parent  Parent of the foreign key token node
+   * @param child Foreign Key token node
+   * @param foreignKeys SQLForeignKey list
+   * @throws SemanticException
+   */
+  private static void processForeignKeys(
+    ASTNode parent, ASTNode child, List<SQLForeignKey> foreignKeys) throws SemanticException {
+    String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+    // The ANTLR grammar looks like :
+    // 1.  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)
+    // when the user specifies the constraint name (i.e. child.getChildCount() == 11)
+    // 2.  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)
+    // when the user does not specify the constraint name (i.e. child.getChildCount() == 10)
+    boolean userSpecifiedConstraintName = child.getChildCount() == 11;
+    int fkIndex = userSpecifiedConstraintName ? 2 : 1;
+    int pkIndex = userSpecifiedConstraintName ? 6 : 5;
+    int ptIndex = userSpecifiedConstraintName ? 4 : 3;
+    int relyIndex =  child.getChildCount() == 11 ? 8 : 7;
+
+    if (child.getChildCount() <= fkIndex ||child.getChildCount() <= pkIndex ||
+      child.getChildCount() <= ptIndex) {
+      throw new SemanticException(ErrorMsg.INVALID_FK_SYNTAX.getMsg());
+    }
+
+    String[] parentDBTbl = getQualifiedTableName((ASTNode) child.getChild(ptIndex));
+
+    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 "));
+    }
+    for (int j = 0; j < child.getChild(fkIndex).getChildCount(); j++) {
+      SQLForeignKey sqlForeignKey = new SQLForeignKey();
+      Tree fkgrandChild = child.getChild(fkIndex).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_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.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()));
+      }
+      foreignKeys.add(sqlForeignKey);
+    }
+  }
+
+  /**
+   * Get the list of FieldSchema out of the ASTNode.
+   * 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<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++) {
       FieldSchema col = new FieldSchema();
       ASTNode child = (ASTNode) ast.getChild(i);
-      Tree grandChild = child.getChild(0);
-      if(grandChild != null) {
-        String name = grandChild.getText();
-        if(lowerCase) {
-          name = name.toLowerCase();
-        }
-        // 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()));
+      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();
+          }
+          // 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()));
+          }
         }
+        nametoFS.put(col.getName(), col);
+        colList.add(col);
       }
-      colList.add(col);
+    }
+    if (!pkInfos.isEmpty()) {
+      processPrimaryKeys((ASTNode) parent, pkInfos, primaryKeys, nametoFS);
     }
     return colList;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 dd997f0..f6d380a 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
@@ -318,6 +318,15 @@ KW_LEVEL: 'LEVEL';
 KW_SNAPSHOT: 'SNAPSHOT';
 KW_AUTOCOMMIT: 'AUTOCOMMIT';
 KW_CACHE: 'CACHE';
+KW_PRIMARY: 'PRIMARY';
+KW_FOREIGN: 'FOREIGN';
+KW_REFERENCES: 'REFERENCES';
+KW_CONSTRAINT: 'CONSTRAINT';
+KW_VALIDATE: 'VALIDATE';
+KW_NOVALIDATE: 'NOVALIDATE';
+KW_RELY: 'RELY';
+KW_NORELY: 'NORELY';
+KW_KEY: 'KEY';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 50c53db..6531b03 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
@@ -102,6 +102,12 @@ TOK_METADATA;
 TOK_NULL;
 TOK_ISNULL;
 TOK_ISNOTNULL;
+TOK_PRIMARY_KEY;
+TOK_FOREIGN_KEY;
+TOK_VALIDATE;
+TOK_NOVALIDATE;
+TOK_RELY;
+TOK_NORELY;
 TOK_TINYINT;
 TOK_SMALLINT;
 TOK_INT;
@@ -515,7 +521,17 @@ 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_PRIMARY", "PRIMARY");
+    xlateMap.put("KW_FOREIGN", "FOREIGN");
+    xlateMap.put("KW_KEY", "KEY");
+    xlateMap.put("KW_REFERENCES", "REFERENCES");
+    xlateMap.put("KW_CONSTRAINT", "CONSTRAINT");
+    xlateMap.put("KW_ENABLE", "ENABLE");
+    xlateMap.put("KW_DISABLE", "DISABLE");
+    xlateMap.put("KW_VALIDATE", "VALIDATE");
+    xlateMap.put("KW_NOVALIDATE", "NOVALIDATE");
+    xlateMap.put("KW_RELY", "RELY");
+    xlateMap.put("KW_NORELY", "NORELY");
 
     // Operators
     xlateMap.put("DOT", ".");
@@ -890,7 +906,7 @@ createTableStatement
          tableFileFormat?
          tableLocation?
          tablePropertiesPrefixed?
-       | (LPAREN columnNameTypeList RPAREN)?
+       | (LPAREN columnNameTypeOrPKOrFKList RPAREN)?
          tableComment?
          tablePartition?
          tableBuckets?
@@ -903,7 +919,7 @@ createTableStatement
       )
     -> ^(TOK_CREATETABLE $name $temp? $ext? ifNotExists?
          ^(TOK_LIKETABLE $likeName?)
-         columnNameTypeList?
+         columnNameTypeOrPKOrFKList?
          tableComment?
          tablePartition?
          tableBuckets?
@@ -1943,6 +1959,11 @@ columnNameTypeList
 @after { popMsg(state); }
     : columnNameType (COMMA columnNameType)* -> ^(TOK_TABCOLLIST columnNameType+)
     ;
+columnNameTypeOrPKOrFKList
+@init { pushMsg("column name type list with PK and FK", state); }
+@after { popMsg(state); }
+    : columnNameTypeOrPKOrFK (COMMA columnNameTypeOrPKOrFK)* -> ^(TOK_TABCOLLIST columnNameTypeOrPKOrFK+)
+    ;
 
 columnNameColonTypeList
 @init { pushMsg("column name type list", state); }
@@ -1976,6 +1997,61 @@ columnNameOrderList
     : columnNameOrder (COMMA columnNameOrder)* -> ^(TOK_TABCOLNAME columnNameOrder+)
     ;
 
+columnParenthesesList
+@init { pushMsg("column parentheses list", state); }
+@after { popMsg(state); }
+    : LPAREN columnNameList RPAREN
+    ;
+
+enableSpecification
+@init { pushMsg("enable specification", state); }
+@after { popMsg(state); }
+    : KW_ENABLE -> ^(TOK_ENABLE)
+    | KW_DISABLE -> ^(TOK_DISABLE)
+    ;
+
+validateSpecification
+@init { pushMsg("validate specification", state); }
+@after { popMsg(state); }
+    : KW_VALIDATE -> ^(TOK_VALIDATE)
+    | KW_NOVALIDATE -> ^(TOK_NOVALIDATE)
+    ;
+
+relySpecification
+@init { pushMsg("rely specification", state); }
+@after { popMsg(state); }
+    :  KW_RELY -> ^(TOK_RELY)
+    |  (KW_NORELY)? -> ^(TOK_NORELY)
+    ;
+
+primaryKeyWithoutName
+@init { pushMsg("primary key without key name", state); }
+@after { popMsg(state); }
+    : KW_PRIMARY KW_KEY columnParenthesesList enableSpec=enableSpecification validateSpec=validateSpecification relySpec=relySpecification
+    -> ^(TOK_PRIMARY_KEY columnParenthesesList $relySpec $enableSpec $validateSpec)
+    ;
+
+primaryKeyWithName
+@init { pushMsg("primary key with key 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)
+    ;
+
+foreignKeyWithName
+@init { pushMsg("foreign key with key name", 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)
+    ;
+
+foreignKeyWithoutName
+@init { pushMsg("foreign key without 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)
+    ;
+
 skewedValueElement
 @init { pushMsg("skewed value element", state); }
 @after { popMsg(state); }
@@ -2087,6 +2163,16 @@ columnNameType
     ->                     ^(TOK_TABCOL $colName colType $comment)
     ;
 
+columnNameTypeOrPKOrFK
+@init { pushMsg("column name or primary key or foreign key", state); }
+@after { popMsg(state); }
+    : ( foreignKeyWithName )
+    | ( primaryKeyWithName )
+    | ( primaryKeyWithoutName )
+    | ( foreignKeyWithoutName )
+    | ( columnNameType )
+    ;
+
 columnNameColonType
 @init { pushMsg("column specification", state); }
 @after { popMsg(state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index a192fa7..3f74551 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -670,6 +670,11 @@ nonReserved
     | KW_LEVEL
     | KW_SNAPSHOT
     | KW_AUTOCOMMIT
+    | KW_RELY
+    | KW_NORELY
+    | KW_VALIDATE
+    | KW_NOVALIDATE
+    | KW_KEY
 ;
 
 //The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers.
@@ -692,5 +697,9 @@ sql11ReservedKeywordsUsedAsIdentifier
     | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE 
     | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH 
 //The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL.    
-    | KW_REGEXP | KW_RLIKE
+    | KW_REGEXP | KW_RLIKE 
+    | KW_PRIMARY
+    | KW_FOREIGN
+    | KW_CONSTRAINT
+    | KW_REFERENCES
     ;

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index a9bc271..fbf1a3c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -313,7 +313,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
         (null == table.getSd().getSkewedInfo()) ? null : table.getSd().getSkewedInfo()
             .getSkewedColNames(),
         (null == table.getSd().getSkewedInfo()) ? null : table.getSd().getSkewedInfo()
-            .getSkewedColValues());
+            .getSkewedColValues(), null, null);
     tblDesc.setStoredAsSubDirectories(table.getSd().isStoredAsSubDirectories());
     return tblDesc;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 11fd2c7..197e8f1 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
@@ -68,6 +68,8 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -11312,6 +11314,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     List<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> bucketCols = new ArrayList<String>();
+    List<SQLPrimaryKey> primaryKeys = new ArrayList<SQLPrimaryKey>();
+    List<SQLForeignKey> foreignKeys = new ArrayList<SQLForeignKey>();
     List<Order> sortCols = new ArrayList<Order>();
     int numBuckets = -1;
     String comment = null;
@@ -11404,7 +11408,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         selectStmt = child;
         break;
       case HiveParser.TOK_TABCOLLIST:
-        cols = getColumns(child);
+        cols = getColumns(child, true, primaryKeys, foreignKeys);
         break;
       case HiveParser.TOK_TABLECOMMENT:
         comment = unescapeSQLString(child.getChild(0).getText());
@@ -11514,7 +11518,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           comment,
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists, skewedColNames,
-          skewedValues);
+          skewedValues, primaryKeys, foreignKeys);
       crtTblDesc.setStoredAsSubDirectories(storedAsDirs);
       crtTblDesc.setNullFormat(rowFormatParams.nullFormat);
 
@@ -11608,7 +11612,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           rowFormatParams.lineDelim, comment, storageFormat.getInputFormat(),
           storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
           storageFormat.getStorageHandler(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
-          skewedColNames, skewedValues, true);
+	  skewedColNames, skewedValues, true, primaryKeys, foreignKeys);
       tableDesc.setMaterialization(isMaterialization);
       tableDesc.setStoredAsSubDirectories(storedAsDirs);
       tableDesc.setNullFormat(rowFormatParams.nullFormat);

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 03b4d8b..2dc4e11 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
@@ -28,6 +28,8 @@ 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.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
@@ -88,6 +90,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
   private boolean isMaterialization = false;
   private boolean replaceMode = false;
   private boolean isCTAS = false;
+  List<SQLPrimaryKey> primaryKeys;
+  List<SQLForeignKey> foreignKeys;
 
   public CreateTableDesc() {
   }
@@ -101,13 +105,14 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       String storageHandler,
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
-      boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues) {
+      boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
 
     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);
+        tblProps, ifNotExists, skewedColNames, skewedColValues, primaryKeys, foreignKeys);
 
     this.databaseName = databaseName;
   }
@@ -122,12 +127,12 @@ 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) {
+                         boolean isCTAS, List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
     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);
+            tblProps, ifNotExists, skewedColNames, skewedColValues, primaryKeys, foreignKeys);
     this.isCTAS = isCTAS;
 
   }
@@ -142,7 +147,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
       String storageHandler,
       Map<String, String> serdeProps,
       Map<String, String> tblProps,
-      boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues) {
+      boolean ifNotExists, List<String> skewedColNames, List<List<String>> skewedColValues,
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys) {
     this.tableName = tableName;
     this.isExternal = isExternal;
     this.isTemporary = isTemporary;
@@ -167,6 +173,16 @@ 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);
+    }
   }
 
   private static <T> List<T> copyList(List<T> copy) {
@@ -221,6 +237,22 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     this.partCols = partCols;
   }
 
+  public List<SQLPrimaryKey> getPrimaryKeys() {
+    return primaryKeys;
+  }
+
+  public void setPrimaryKeys(ArrayList<SQLPrimaryKey> primaryKeys) {
+    this.primaryKeys = primaryKeys;
+  }
+
+  public List<SQLForeignKey> getForeignKeys() {
+    return foreignKeys;
+  }
+
+  public void setForeignKeys(ArrayList<SQLForeignKey> foreignKeys) {
+    this.foreignKeys = foreignKeys;
+  }
+
   @Explain(displayName = "bucket columns")
   public List<String> getBucketCols() {
     return bucketCols;
@@ -634,6 +666,7 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
     if (getPartCols() != null) {
       tbl.setPartCols(getPartCols());
     }
+
     if (getNumBuckets() != -1) {
       tbl.setNumBuckets(getNumBuckets());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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 7d5a414..013b3e8 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
@@ -105,7 +105,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 specification"));
+      driver.getErrorMsg().contains("missing ) at ',' near ',' in column name or primary key or foreign key"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..b6b30c5
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_with_constraints_duplicate_name.q
@@ -0,0 +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);

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..4ce3cbc
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_with_constraints_enable.q
@@ -0,0 +1 @@
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) enable novalidate);

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/ql/src/test/queries/clientnegative/create_with_constraints_validate.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/create_with_constraints_validate.q b/ql/src/test/queries/clientnegative/create_with_constraints_validate.q
new file mode 100644
index 0000000..6601235
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/create_with_constraints_validate.q
@@ -0,0 +1 @@
+CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable validate);

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..eef0c64
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/create_with_constraints.q
@@ -0,0 +1,12 @@
+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(b)  DISABLE NOVALIDATE); 
+CREATE TABLE table4 (x string, y string, PRIMARY KEY (x) disable novalidate, CONSTRAINT fk2 FOREIGN KEY (x) REFERENCES table2(b)  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(b)  DISABLE NOVALIDATE);
+CREATE TABLE table6 (x string, y string, PRIMARY KEY (x) disable novalidate, FOREIGN KEY (x) REFERENCES table2(b)  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);
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..20f5d30
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_with_constraints_duplicate_name.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: create table t1(x int, constraint pk1 primary key (x) disable novalidate)
+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: 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: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:For direct MetaStore DB connections, we don't support retries at the client level.)

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..f5dcf85
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_with_constraints_enable.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Primary Key syntax  ENABLE feature not supported yet

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..307e8ec
--- /dev/null
+++ b/ql/src/test/results/clientnegative/create_with_constraints_validate.q.out
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10326]: Invalid Primary Key syntax  VALIDATE feature not supported yet

http://git-wip-us.apache.org/repos/asf/hive/blob/53249a35/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
new file mode 100644
index 0000000..5cf8d83
--- /dev/null
+++ b/ql/src/test/results/clientpositive/create_with_constraints.q.out
@@ -0,0 +1,68 @@
+PREHOOK: query: CREATE TABLE table1 (a STRING, b STRING, primary key (a) disable novalidate)
+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: 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: 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: 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(b)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE, 
+CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE, 
+CONSTRAINT fk3 FOREIGN KEY (y) REFERENCES table2(a)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE,
+CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE)
+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(b)  DISABLE NOVALIDATE,
+CONSTRAINT fk4 FOREIGN KEY (y) REFERENCES table1(a)  DISABLE NOVALIDATE)
+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: 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: 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: 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: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table8