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

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

HIVE-18726 : Implement DEFAULT constraint (Vineet Garg, reviewed by Ashutosh Chauhan, Jesús Camacho Rodríguez)


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

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

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


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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/4047befe/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
new file mode 100644
index 0000000..59df3da
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultConstraint.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.metadata;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+
+/**
+ * DefaultConstraintInfo is a metadata structure containing the default constraints
+ * associated with a table.
+ */
+@SuppressWarnings("serial")
+public class DefaultConstraint implements Serializable {
+
+  public class DefaultConstraintCol {
+    public String colName;
+    public String defaultVal;
+
+    public DefaultConstraintCol(String colName, String defaultVal) {
+      this.colName = colName;
+      this.defaultVal = defaultVal;
+    }
+  }
+
+  // Mapping from constraint name to list of default constraints
+  Map<String, List<DefaultConstraintCol>> defaultConstraints;
+
+  // Mapping from column name to default value
+  Map<String, String> colNameToDefaultValueMap;
+  String tableName;
+  String databaseName;
+
+  public DefaultConstraint() {}
+
+  public DefaultConstraint(List<SQLDefaultConstraint> defaultConstraintList, String tableName, String databaseName) {
+    this.tableName = tableName;
+    this.databaseName = databaseName;
+    defaultConstraints = new TreeMap<String, List<DefaultConstraintCol>>();
+    colNameToDefaultValueMap = new TreeMap<String, String>();
+    if (defaultConstraintList == null) {
+      return;
+    }
+    for (SQLDefaultConstraint uk : defaultConstraintList) {
+      if (uk.getTable_db().equalsIgnoreCase(databaseName) &&
+          uk.getTable_name().equalsIgnoreCase(tableName)) {
+        String colName = uk.getColumn_name();
+        String defVal = uk.getDefault_value();
+        colNameToDefaultValueMap.put(colName, defVal);
+        DefaultConstraintCol currCol = new DefaultConstraintCol(
+                colName, defVal);
+        String constraintName = uk.getDc_name();
+        if (defaultConstraints.containsKey(constraintName)) {
+          defaultConstraints.get(constraintName).add(currCol);
+        } else {
+          List<DefaultConstraintCol> currList = new ArrayList<DefaultConstraintCol>();
+          currList.add(currCol);
+          defaultConstraints.put(constraintName, currList);
+        }
+      }
+    }
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public Map<String, List<DefaultConstraintCol>> getDefaultConstraints() {
+    return defaultConstraints;
+  }
+  public Map<String, String> getColNameToDefaultValueMap() {
+    return colNameToDefaultValueMap;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Default Constraints for " + databaseName + "." + tableName + ":");
+    sb.append("[");
+    if (defaultConstraints != null && defaultConstraints.size() > 0) {
+      for (Map.Entry<String, List<DefaultConstraintCol>> me : defaultConstraints.entrySet()) {
+        sb.append(" {Constraint Name: " + me.getKey() + ",");
+        List<DefaultConstraintCol> currCol = me.getValue();
+        if (currCol != null && currCol.size() > 0) {
+          for (DefaultConstraintCol ukc : currCol) {
+            sb.append (" (Column Name: " + ukc.colName + ", Default Value: " + ukc.defaultVal + "),");
+          }
+          sb.setLength(sb.length()-1);
+        }
+        sb.append("},");
+      }
+      sb.setLength(sb.length()-1);
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+}

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

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

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

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

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

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

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

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

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

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

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