You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/08/28 18:27:41 UTC

[GitHub] [hive] sankarh commented on a change in pull request #1419: HIVE-22782: Consolidate metastore call to fetch constraints

sankarh commented on a change in pull request #1419:
URL: https://github.com/apache/hive/pull/1419#discussion_r479451533



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2811,6 +2811,26 @@ public GetFieldsResponse getFieldsRequest(GetFieldsRequest req)
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
+  @Override
+  public SQLAllTableConstraints getAllTableConstraints(AllTableConstraintsRequest req)
+      throws MetaException, NoSuchObjectException, TException {
+    long t1 = System.currentTimeMillis();
+
+    try {
+      if (!req.isSetCatName()) {
+        req.setCatName(getDefaultCatalog(conf));
+      }
+
+      return client.get_all_table_constraints(req).getAllTableConstraints();
+    } finally {
+      long diff = System.currentTimeMillis() - t1;

Review comment:
       I think, it is redundant as HMS also logs the time taken by get_all_table_constraints api. 

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
##########
@@ -1485,6 +1485,16 @@ void getFileMetadataByExpr(List<Long> fileIds, FileMetadataExprType type, byte[]
   List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name,
                                                    String tbl_name) throws MetaException;
 
+  /**
+   *  Get all constraints of the table
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @return all constraints for this table
+   * @throws MetaException error accessing the RDBMS
+   */
+  SQLAllTableConstraints getAllTableConstraints(String catName, String db_name, String tbl_name)

Review comment:
       Follow uniform naming style for arguments.

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
##########
@@ -1014,6 +1015,11 @@ public FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type) {
     return null;
   }
 
+  @Override public SQLAllTableConstraints getAllTableConstraints(String catName, String db_name, String tbl_name)

Review comment:
       Nit: Annotation and api signature can be in separate lines.

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2811,6 +2811,26 @@ public GetFieldsResponse getFieldsRequest(GetFieldsRequest req)
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
+  @Override
+  public SQLAllTableConstraints getAllTableConstraints(AllTableConstraintsRequest req)
+      throws MetaException, NoSuchObjectException, TException {
+    long t1 = System.currentTimeMillis();
+
+    try {
+      if (!req.isSetCatName()) {

Review comment:
       HMS api handles this default catalog name flow. Shall remove it here.

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetAllTableConstraints.java
##########
@@ -0,0 +1,382 @@
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AllTableConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.SQLAllTableConstraints;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+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;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLCheckConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLDefaultConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLUniqueConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestGetAllTableConstraints
+    extends MetaStoreClientTest {
+  private static final String OTHER_DATABASE = "test_constraints_other_database";
+  private static final String OTHER_CATALOG = "test_constraints_other_catalog";
+  private static final String DATABASE_IN_OTHER_CATALOG = "test_constraints_database_in_other_catalog";
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+  private Table[] testTables = new Table[3];
+  private Database inOtherCatalog;
+
+  public TestGetAllTableConstraints(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+    // Clean up the database
+    client.dropDatabase(OTHER_DATABASE, true, true, true);
+    // Drop every table in the default database
+    for (String tableName : client.getAllTables(DEFAULT_DATABASE_NAME)) {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
+    }
+
+    client.dropDatabase(OTHER_CATALOG, DATABASE_IN_OTHER_CATALOG, true, true, true);
+    try {
+      client.dropCatalog(OTHER_CATALOG);
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Clean up trash
+    metaStore.cleanWarehouseDirs();
+
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
+
+    Catalog cat =
+        new CatalogBuilder().setName(OTHER_CATALOG).setLocation(MetaStoreTestUtils.getTestWarehouseDir(OTHER_CATALOG))
+            .build();
+    client.createCatalog(cat);
+
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    inOtherCatalog = new DatabaseBuilder().setName(DATABASE_IN_OTHER_CATALOG).setCatalogName(OTHER_CATALOG)
+        .create(client, metaStore.getConf());
+
+    testTables[0] = new TableBuilder().setTableName("test_table_1").addCol("col1", "int").addCol("col2", "int")
+        .addCol("col3", "boolean").addCol("col4", "int").addCol("col5", "varchar(32)")
+        .create(client, metaStore.getConf());
+
+    testTables[1] = new TableBuilder().setDbName(OTHER_DATABASE).setTableName("test_table_2").addCol("col1", "int")
+        .addCol("col2", "varchar(32)").create(client, metaStore.getConf());
+
+    testTables[2] = new TableBuilder().inDb(inOtherCatalog).setTableName("test_table_3").addCol("col1", "int")
+        .addCol("col2", "varchar(32)").create(client, metaStore.getConf());
+
+    // Reload tables from the MetaStore
+    for (int i = 0; i < testTables.length; i++) {
+      testTables[i] =
+          client.getTable(testTables[i].getCatName(), testTables[i].getDbName(), testTables[i].getTableName());
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        try {
+          client.close();
+        } catch (Exception e) {
+          // HIVE-19729: Shallow the exceptions based on the discussion in the Jira
+        }
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void noConstraints() throws TException {
+    Table table = testTables[0];
+
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    Assert.assertTrue(fetched.getCheckConstraints().isEmpty());
+    Assert.assertTrue(fetched.getForeignKeys().isEmpty());
+    Assert.assertTrue(fetched.getDefaultConstraints().isEmpty());
+    Assert.assertTrue(fetched.getNotNullConstraints().isEmpty());
+    Assert.assertTrue(fetched.getPrimaryKeys().isEmpty());
+    Assert.assertTrue(fetched.getUniqueConstraints().isEmpty());
+  }
+
+  @Test
+  public void fewPresentWithMultipleConstraints() throws TException {
+    Table table = testTables[0];
+
+    // Set col1 as primary key Constraint in default catalog and database
+    String pkConstraintName = "col1_pk";
+    List<SQLPrimaryKey> pk =
+        new SQLPrimaryKeyBuilder().onTable(table).addColumn("col1").setConstraintName(pkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    // Set col2 with Unique Constraint in default catalog and database
+    String uniqueConstraintName = "col2_unique";
+    List<SQLUniqueConstraint> uc =
+        new SQLUniqueConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(uniqueConstraintName)
+            .build(metaStore.getConf());
+    client.addUniqueConstraint(uc);
+
+    // Set col3 with default Constraint in default catalog and database
+    String defaultConstraintName = "col3_default";
+    List<SQLDefaultConstraint> dv =
+        new SQLDefaultConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(defaultConstraintName)
+            .setDefaultVal(false).build(metaStore.getConf());
+    client.addDefaultConstraint(dv);
+
+    // Set col2 with not null constraint in default catalog and database;
+    String nnCol2ConstraintName = "col2_not_null";
+    List<SQLNotNullConstraint> nnCol2 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(nnCol2ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol2);
+
+    // Set col3 with not null constraint in default catalog and database;
+    String nnCol3ConstraintName = "col3_not_null";
+    List<SQLNotNullConstraint> nnCol3 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(nnCol3ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol3);
+
+    // Fetch all constraints for the table in default catalog and database
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    // Assert primary key constraint
+    Assert.assertEquals(1, fetched.getPrimaryKeysSize());
+    Assert.assertEquals(table.getDbName(), fetched.getPrimaryKeys().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getPrimaryKeys().get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.getPrimaryKeys().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getPrimaryKeys().get(0).getKey_seq());
+    Assert.assertEquals(pkConstraintName, fetched.getPrimaryKeys().get(0).getPk_name());
+    Assert.assertTrue(fetched.getPrimaryKeys().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getPrimaryKeys().get(0).getCatName());
+
+    // Assert unique constraint
+    Assert.assertEquals(1, fetched.getUniqueConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getUniqueConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getUniqueConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getUniqueConstraints().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getUniqueConstraints().get(0).getKey_seq());
+    Assert.assertEquals(uniqueConstraintName, fetched.getUniqueConstraints().get(0).getUk_name());
+    Assert.assertTrue(fetched.getUniqueConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getUniqueConstraints().get(0).getCatName());
+
+    // Assert Default constraint
+    Assert.assertEquals(1, fetched.getDefaultConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getDefaultConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getDefaultConstraints().get(0).getTable_name());
+    Assert.assertEquals("col3", fetched.getDefaultConstraints().get(0).getColumn_name());
+    Assert.assertEquals("false", fetched.getDefaultConstraints().get(0).getDefault_value());
+    Assert.assertEquals(defaultConstraintName, fetched.getDefaultConstraints().get(0).getDc_name());
+    Assert.assertTrue(fetched.getDefaultConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getDefaultConstraints().get(0).getCatName());
+
+    // Assert Not Null constraint
+    Assert.assertEquals(2, fetched.getNotNullConstraintsSize());
+
+    // Assert Not Null constraint for col2
+    Assert.assertEquals(table.getDbName(), fetched.getNotNullConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getNotNullConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getNotNullConstraints().get(0).getColumn_name());
+    Assert.assertEquals(nnCol2ConstraintName, fetched.getNotNullConstraints().get(0).getNn_name());
+    Assert.assertTrue(fetched.getNotNullConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getNotNullConstraints().get(0).getCatName());
+
+    // Assert Not Null constraint for col3
+    Assert.assertEquals(table.getDbName(), fetched.getNotNullConstraints().get(1).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getNotNullConstraints().get(1).getTable_name());
+    Assert.assertEquals("col3", fetched.getNotNullConstraints().get(1).getColumn_name());
+    Assert.assertEquals(nnCol3ConstraintName, fetched.getNotNullConstraints().get(1).getNn_name());
+    Assert.assertTrue(fetched.getNotNullConstraints().get(1).isEnable_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(1).isValidate_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(1).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getNotNullConstraints().get(1).getCatName());
+
+    // Check constraints which is not present in table
+    Assert.assertTrue(fetched.getCheckConstraints().isEmpty());
+    Assert.assertTrue(fetched.getForeignKeys().isEmpty());
+
+  }
+
+  @Test
+  public void allConstraintsPresent() throws TException {
+    Table table = testTables[0];
+    Table parentTable = testTables[1];
+
+    // Set col1 as primary key Constraint in default catalog and database
+    String pkConstraintName = "col1_pk";
+    List<SQLPrimaryKey> pk =
+        new SQLPrimaryKeyBuilder().onTable(table).addColumn("col1").setConstraintName(pkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    // Set col2 with Unique Constraint in default catalog and database
+    String uniqueConstraintName = "col2_unique";
+    List<SQLUniqueConstraint> uc =
+        new SQLUniqueConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(uniqueConstraintName)
+            .build(metaStore.getConf());
+    client.addUniqueConstraint(uc);
+
+    // Set col3 with default Constraint in default catalog and database
+    String defaultConstraintName = "col3_default";
+    List<SQLDefaultConstraint> dv =
+        new SQLDefaultConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(defaultConstraintName)
+            .setDefaultVal(false).build(metaStore.getConf());
+    client.addDefaultConstraint(dv);
+
+    // Set col3 with not null constraint in default catalog and database;
+    String nnCol3ConstraintName = "col3_not_null";
+    List<SQLNotNullConstraint> nnCol2 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(nnCol3ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol2);
+
+    // Set col2 with not check constraint in default catalog and database;
+    String ccCol2ConstraintName = "col2_check";
+    List<SQLCheckConstraint> cc =
+        new SQLCheckConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(ccCol2ConstraintName)
+            .setCheckExpression("= 5").build(metaStore.getConf());
+    client.addCheckConstraint(cc);
+
+    // Set col1 of parent table to PK and Set Col4 of table to FK
+    String parentPkConstraintName = "parentpk";
+    List<SQLPrimaryKey> parentPk =
+        new SQLPrimaryKeyBuilder().onTable(parentTable).addColumn("col1").setConstraintName(parentPkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(parentPk);
+    String fkConstraintName = "fk";
+    List<SQLForeignKey> fk =
+        new SQLForeignKeyBuilder().fromPrimaryKey(parentPk).onTable(table).setConstraintName(fkConstraintName)
+            .addColumn("col4").build(metaStore.getConf());
+    client.addForeignKey(fk);
+
+    // Fetch all constraints for the table in default catalog and database
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    // Assert primary key constraint
+    Assert.assertEquals(1, fetched.getPrimaryKeysSize());
+    Assert.assertEquals(table.getDbName(), fetched.getPrimaryKeys().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getPrimaryKeys().get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.getPrimaryKeys().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getPrimaryKeys().get(0).getKey_seq());
+    Assert.assertEquals(pkConstraintName, fetched.getPrimaryKeys().get(0).getPk_name());
+    Assert.assertTrue(fetched.getPrimaryKeys().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getPrimaryKeys().get(0).getCatName());
+
+    // Assert unique constraint
+    Assert.assertEquals(1, fetched.getUniqueConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getUniqueConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getUniqueConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getUniqueConstraints().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getUniqueConstraints().get(0).getKey_seq());
+    Assert.assertEquals(uniqueConstraintName, fetched.getUniqueConstraints().get(0).getUk_name());
+    Assert.assertTrue(fetched.getUniqueConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getUniqueConstraints().get(0).getCatName());
+
+    // Assert Default constraint
+    Assert.assertEquals(1, fetched.getDefaultConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getDefaultConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getDefaultConstraints().get(0).getTable_name());
+    Assert.assertEquals("col3", fetched.getDefaultConstraints().get(0).getColumn_name());
+    Assert.assertEquals("false", fetched.getDefaultConstraints().get(0).getDefault_value());
+    Assert.assertEquals(defaultConstraintName, fetched.getDefaultConstraints().get(0).getDc_name());
+    Assert.assertTrue(fetched.getDefaultConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getDefaultConstraints().get(0).getCatName());
+
+    // Assert Not Null constraint
+    Assert.assertEquals(1, fetched.getNotNullConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getNotNullConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getNotNullConstraints().get(0).getTable_name());
+    Assert.assertEquals("col3", fetched.getNotNullConstraints().get(0).getColumn_name());
+    Assert.assertEquals(nnCol3ConstraintName, fetched.getNotNullConstraints().get(0).getNn_name());
+    Assert.assertTrue(fetched.getNotNullConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getNotNullConstraints().get(0).getCatName());
+
+    // Assert check constraint
+    Assert.assertEquals(1, fetched.getNotNullConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getCheckConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getCheckConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getCheckConstraints().get(0).getColumn_name());
+    Assert.assertEquals("= 5", fetched.getCheckConstraints().get(0).getCheck_expression());
+    Assert.assertEquals(ccCol2ConstraintName, fetched.getCheckConstraints().get(0).getDc_name());
+    Assert.assertTrue(fetched.getCheckConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getCheckConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getCheckConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getCheckConstraints().get(0).getCatName());
+
+    // Assert foreign key
+    Assert.assertEquals(1, fetched.getForeignKeysSize());
+    Assert.assertEquals(table.getDbName(), fetched.getForeignKeys().get(0).getFktable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getForeignKeys().get(0).getFktable_name());
+    Assert.assertEquals("col4", fetched.getForeignKeys().get(0).getFkcolumn_name());
+    Assert.assertEquals(parentTable.getDbName(), fetched.getForeignKeys().get(0).getPktable_db());
+    Assert.assertEquals(parentTable.getTableName(), fetched.getForeignKeys().get(0).getPktable_name());
+    Assert.assertEquals("col1", fetched.getForeignKeys().get(0).getPkcolumn_name());
+    Assert.assertEquals(1, fetched.getForeignKeys().get(0).getKey_seq());
+    Assert.assertEquals(parentPkConstraintName, fetched.getForeignKeys().get(0).getPk_name());
+    Assert.assertEquals(fkConstraintName, fetched.getForeignKeys().get(0).getFk_name());
+    Assert.assertTrue(fetched.getForeignKeys().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getForeignKeys().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getForeignKeys().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getForeignKeys().get(0).getCatName());
+
+  }

Review comment:
       Add a negative scenario where db invalid or table invalid etc and check if we get NoSuchObjectException.

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetAllTableConstraints.java
##########
@@ -0,0 +1,382 @@
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AllTableConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.SQLAllTableConstraints;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+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;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLCheckConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLDefaultConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLUniqueConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)

Review comment:
       Will these tests run in our PR build pipeline?

##########
File path: standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
##########
@@ -720,6 +729,15 @@ struct CheckConstraintsResponse {
   1: required list<SQLCheckConstraint> checkConstraints
 }
 
+struct AllTableConstraintsRequest {
+  1: required string db_name,

Review comment:
       Nit: Naming can be dbName, tblName and catName.

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2811,6 +2811,26 @@ public GetFieldsResponse getFieldsRequest(GetFieldsRequest req)
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
+  @Override
+  public SQLAllTableConstraints getAllTableConstraints(AllTableConstraintsRequest req)
+      throws MetaException, NoSuchObjectException, TException {
+    long t1 = System.currentTimeMillis();

Review comment:
       This can be kept under isDebugEnabled flag.

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -2811,6 +2811,26 @@ public GetFieldsResponse getFieldsRequest(GetFieldsRequest req)
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
+  @Override
+  public SQLAllTableConstraints getAllTableConstraints(AllTableConstraintsRequest req)

Review comment:
       Use this api in query planner instead of multiple calls.

##########
File path: standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
##########
@@ -122,6 +122,15 @@ struct SQLCheckConstraint {
   9: bool rely_cstr      // Rely/No Rely
 }
 
+struct SQLAllTableConstraints {
+  1: optional list<SQLPrimaryKey> primaryKeys,
+  2: optional list<SQLForeignKey> foreignKeys,
+  3: optional list<SQLUniqueConstraint> uniqueConstraints,
+  4: optional list<SQLNotNullConstraint> notNullConstraints,
+  5: optional list<SQLDefaultConstraint> defaultConstraints,
+  6: optional list<SQLCheckConstraint> checkConstraints,

Review comment:
       Remove comma "," in last column.

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetAllTableConstraints.java
##########
@@ -0,0 +1,382 @@
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AllTableConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.SQLAllTableConstraints;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+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;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLCheckConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLDefaultConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLUniqueConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestGetAllTableConstraints
+    extends MetaStoreClientTest {
+  private static final String OTHER_DATABASE = "test_constraints_other_database";
+  private static final String OTHER_CATALOG = "test_constraints_other_catalog";
+  private static final String DATABASE_IN_OTHER_CATALOG = "test_constraints_database_in_other_catalog";
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+  private Table[] testTables = new Table[3];
+  private Database inOtherCatalog;
+
+  public TestGetAllTableConstraints(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+    // Clean up the database
+    client.dropDatabase(OTHER_DATABASE, true, true, true);
+    // Drop every table in the default database
+    for (String tableName : client.getAllTables(DEFAULT_DATABASE_NAME)) {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
+    }
+
+    client.dropDatabase(OTHER_CATALOG, DATABASE_IN_OTHER_CATALOG, true, true, true);
+    try {
+      client.dropCatalog(OTHER_CATALOG);
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Clean up trash
+    metaStore.cleanWarehouseDirs();
+
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
+
+    Catalog cat =
+        new CatalogBuilder().setName(OTHER_CATALOG).setLocation(MetaStoreTestUtils.getTestWarehouseDir(OTHER_CATALOG))
+            .build();
+    client.createCatalog(cat);
+
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    inOtherCatalog = new DatabaseBuilder().setName(DATABASE_IN_OTHER_CATALOG).setCatalogName(OTHER_CATALOG)
+        .create(client, metaStore.getConf());
+
+    testTables[0] = new TableBuilder().setTableName("test_table_1").addCol("col1", "int").addCol("col2", "int")
+        .addCol("col3", "boolean").addCol("col4", "int").addCol("col5", "varchar(32)")
+        .create(client, metaStore.getConf());
+
+    testTables[1] = new TableBuilder().setDbName(OTHER_DATABASE).setTableName("test_table_2").addCol("col1", "int")
+        .addCol("col2", "varchar(32)").create(client, metaStore.getConf());
+
+    testTables[2] = new TableBuilder().inDb(inOtherCatalog).setTableName("test_table_3").addCol("col1", "int")
+        .addCol("col2", "varchar(32)").create(client, metaStore.getConf());
+
+    // Reload tables from the MetaStore
+    for (int i = 0; i < testTables.length; i++) {
+      testTables[i] =
+          client.getTable(testTables[i].getCatName(), testTables[i].getDbName(), testTables[i].getTableName());
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        try {
+          client.close();
+        } catch (Exception e) {
+          // HIVE-19729: Shallow the exceptions based on the discussion in the Jira
+        }
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void noConstraints() throws TException {
+    Table table = testTables[0];
+
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    Assert.assertTrue(fetched.getCheckConstraints().isEmpty());
+    Assert.assertTrue(fetched.getForeignKeys().isEmpty());
+    Assert.assertTrue(fetched.getDefaultConstraints().isEmpty());
+    Assert.assertTrue(fetched.getNotNullConstraints().isEmpty());
+    Assert.assertTrue(fetched.getPrimaryKeys().isEmpty());
+    Assert.assertTrue(fetched.getUniqueConstraints().isEmpty());
+  }
+
+  @Test
+  public void fewPresentWithMultipleConstraints() throws TException {
+    Table table = testTables[0];
+
+    // Set col1 as primary key Constraint in default catalog and database
+    String pkConstraintName = "col1_pk";
+    List<SQLPrimaryKey> pk =
+        new SQLPrimaryKeyBuilder().onTable(table).addColumn("col1").setConstraintName(pkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    // Set col2 with Unique Constraint in default catalog and database
+    String uniqueConstraintName = "col2_unique";
+    List<SQLUniqueConstraint> uc =
+        new SQLUniqueConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(uniqueConstraintName)
+            .build(metaStore.getConf());
+    client.addUniqueConstraint(uc);
+
+    // Set col3 with default Constraint in default catalog and database
+    String defaultConstraintName = "col3_default";
+    List<SQLDefaultConstraint> dv =
+        new SQLDefaultConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(defaultConstraintName)
+            .setDefaultVal(false).build(metaStore.getConf());
+    client.addDefaultConstraint(dv);
+
+    // Set col2 with not null constraint in default catalog and database;
+    String nnCol2ConstraintName = "col2_not_null";
+    List<SQLNotNullConstraint> nnCol2 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(nnCol2ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol2);
+
+    // Set col3 with not null constraint in default catalog and database;
+    String nnCol3ConstraintName = "col3_not_null";
+    List<SQLNotNullConstraint> nnCol3 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(nnCol3ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol3);
+
+    // Fetch all constraints for the table in default catalog and database
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    // Assert primary key constraint
+    Assert.assertEquals(1, fetched.getPrimaryKeysSize());
+    Assert.assertEquals(table.getDbName(), fetched.getPrimaryKeys().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getPrimaryKeys().get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.getPrimaryKeys().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getPrimaryKeys().get(0).getKey_seq());
+    Assert.assertEquals(pkConstraintName, fetched.getPrimaryKeys().get(0).getPk_name());
+    Assert.assertTrue(fetched.getPrimaryKeys().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getPrimaryKeys().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getPrimaryKeys().get(0).getCatName());
+
+    // Assert unique constraint
+    Assert.assertEquals(1, fetched.getUniqueConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getUniqueConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getUniqueConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getUniqueConstraints().get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.getUniqueConstraints().get(0).getKey_seq());
+    Assert.assertEquals(uniqueConstraintName, fetched.getUniqueConstraints().get(0).getUk_name());
+    Assert.assertTrue(fetched.getUniqueConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getUniqueConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getUniqueConstraints().get(0).getCatName());
+
+    // Assert Default constraint
+    Assert.assertEquals(1, fetched.getDefaultConstraintsSize());
+    Assert.assertEquals(table.getDbName(), fetched.getDefaultConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getDefaultConstraints().get(0).getTable_name());
+    Assert.assertEquals("col3", fetched.getDefaultConstraints().get(0).getColumn_name());
+    Assert.assertEquals("false", fetched.getDefaultConstraints().get(0).getDefault_value());
+    Assert.assertEquals(defaultConstraintName, fetched.getDefaultConstraints().get(0).getDc_name());
+    Assert.assertTrue(fetched.getDefaultConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getDefaultConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getDefaultConstraints().get(0).getCatName());
+
+    // Assert Not Null constraint
+    Assert.assertEquals(2, fetched.getNotNullConstraintsSize());
+
+    // Assert Not Null constraint for col2
+    Assert.assertEquals(table.getDbName(), fetched.getNotNullConstraints().get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getNotNullConstraints().get(0).getTable_name());
+    Assert.assertEquals("col2", fetched.getNotNullConstraints().get(0).getColumn_name());
+    Assert.assertEquals(nnCol2ConstraintName, fetched.getNotNullConstraints().get(0).getNn_name());
+    Assert.assertTrue(fetched.getNotNullConstraints().get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getNotNullConstraints().get(0).getCatName());
+
+    // Assert Not Null constraint for col3
+    Assert.assertEquals(table.getDbName(), fetched.getNotNullConstraints().get(1).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.getNotNullConstraints().get(1).getTable_name());
+    Assert.assertEquals("col3", fetched.getNotNullConstraints().get(1).getColumn_name());
+    Assert.assertEquals(nnCol3ConstraintName, fetched.getNotNullConstraints().get(1).getNn_name());
+    Assert.assertTrue(fetched.getNotNullConstraints().get(1).isEnable_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(1).isValidate_cstr());
+    Assert.assertFalse(fetched.getNotNullConstraints().get(1).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.getNotNullConstraints().get(1).getCatName());
+
+    // Check constraints which is not present in table
+    Assert.assertTrue(fetched.getCheckConstraints().isEmpty());
+    Assert.assertTrue(fetched.getForeignKeys().isEmpty());
+
+  }
+
+  @Test
+  public void allConstraintsPresent() throws TException {
+    Table table = testTables[0];
+    Table parentTable = testTables[1];
+
+    // Set col1 as primary key Constraint in default catalog and database
+    String pkConstraintName = "col1_pk";
+    List<SQLPrimaryKey> pk =
+        new SQLPrimaryKeyBuilder().onTable(table).addColumn("col1").setConstraintName(pkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    // Set col2 with Unique Constraint in default catalog and database
+    String uniqueConstraintName = "col2_unique";
+    List<SQLUniqueConstraint> uc =
+        new SQLUniqueConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(uniqueConstraintName)
+            .build(metaStore.getConf());
+    client.addUniqueConstraint(uc);
+
+    // Set col3 with default Constraint in default catalog and database
+    String defaultConstraintName = "col3_default";
+    List<SQLDefaultConstraint> dv =
+        new SQLDefaultConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(defaultConstraintName)
+            .setDefaultVal(false).build(metaStore.getConf());
+    client.addDefaultConstraint(dv);
+
+    // Set col3 with not null constraint in default catalog and database;
+    String nnCol3ConstraintName = "col3_not_null";
+    List<SQLNotNullConstraint> nnCol2 =
+        new SQLNotNullConstraintBuilder().onTable(table).addColumn("col3").setConstraintName(nnCol3ConstraintName)
+            .build(metaStore.getConf());
+    client.addNotNullConstraint(nnCol2);
+
+    // Set col2 with not check constraint in default catalog and database;
+    String ccCol2ConstraintName = "col2_check";
+    List<SQLCheckConstraint> cc =
+        new SQLCheckConstraintBuilder().onTable(table).addColumn("col2").setConstraintName(ccCol2ConstraintName)
+            .setCheckExpression("= 5").build(metaStore.getConf());
+    client.addCheckConstraint(cc);
+
+    // Set col1 of parent table to PK and Set Col4 of table to FK
+    String parentPkConstraintName = "parentpk";
+    List<SQLPrimaryKey> parentPk =
+        new SQLPrimaryKeyBuilder().onTable(parentTable).addColumn("col1").setConstraintName(parentPkConstraintName)
+            .build(metaStore.getConf());
+    client.addPrimaryKey(parentPk);
+    String fkConstraintName = "fk";
+    List<SQLForeignKey> fk =
+        new SQLForeignKeyBuilder().fromPrimaryKey(parentPk).onTable(table).setConstraintName(fkConstraintName)
+            .addColumn("col4").build(metaStore.getConf());
+    client.addForeignKey(fk);
+
+    // Fetch all constraints for the table in default catalog and database
+    AllTableConstraintsRequest request = new AllTableConstraintsRequest(table.getDbName(), table.getTableName());
+    request.setCatName(table.getCatName());
+    SQLAllTableConstraints fetched = client.getAllTableConstraints(request);
+
+    // Assert primary key constraint

Review comment:
       Too much of duplicate code. Can write common methods for the asserts and use it in all tests.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org