You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2020/02/15 13:04:00 UTC

[impala] branch master updated: IMPALA-9256: Refactor constraint information into a separate class.

This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git


The following commit(s) were added to refs/heads/master by this push:
     new d75562a  IMPALA-9256: Refactor constraint information into a separate class.
d75562a is described below

commit d75562a7e331b7e5b2509bdc3652c557a24a04d0
Author: Anurag Mantripragada <an...@cloudera.com>
AuthorDate: Mon Jan 27 16:54:55 2020 -0800

    IMPALA-9256: Refactor constraint information into a separate class.
    
    This change refactors the primary keys and foreign keys into a
    SqlConstraints class since they are almost always used together. This
    work also helps extend the constraints class to include other
    constraints we may support in the future. (Ex: Unique constraints.)
    
    This patch also:
    - Fixes a bug in the MetadataOp.getPrimaryKeys() and getForeignKeys()
      which returned incorrect results. The tests did not catch this
      before beacuse we did not have tests to verify individual resultset
      rows. The patch modifies these tests.
    - Fixes a bug in foreign key constraint name generation that was
      causing foreign keys corresponding to a composite primary key get
      different foreign key constraint names instead of the same name.
    - Introduces a canonical representation for foreign keys to prevent
      bugs like IMPALA-9372 which can occur due to HMS returning results
      in inconsistent ways.
    
    Testing:
    - Fixed the tests to work with the new behavior.
    - Ran all the PK/FK tests.
    
    Change-Id: I3f1c441c24df84d2d0791ffe94dff60d039a3341
    Reviewed-on: http://gerrit.cloudera.org:8080/15213
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/generated-sources/gen-cpp/CMakeLists.txt        |   2 +
 common/thrift/CMakeLists.txt                       |   1 +
 common/thrift/CatalogObjects.thrift                |   6 +-
 common/thrift/CatalogService.thrift                |   9 +-
 common/thrift/SqlConstraints.thrift                |  31 +++
 .../java/org/apache/impala/analysis/TableDef.java  |  20 +-
 .../java/org/apache/impala/catalog/FeFsTable.java  |  13 +-
 .../java/org/apache/impala/catalog/FeTable.java    |  11 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  |  47 ++---
 .../org/apache/impala/catalog/SqlConstraints.java  |  99 +++++++++
 .../main/java/org/apache/impala/catalog/Table.java |  21 +-
 .../impala/catalog/local/CatalogdMetaProvider.java |  23 +--
 .../impala/catalog/local/DirectMetaProvider.java   |  21 +-
 .../apache/impala/catalog/local/LocalFsTable.java  |  44 +---
 .../apache/impala/catalog/local/LocalTable.java    |  15 +-
 .../apache/impala/catalog/local/MetaProvider.java  |   8 +-
 .../java/org/apache/impala/service/Frontend.java   |   9 +-
 .../java/org/apache/impala/service/MetadataOp.java |   5 +-
 .../java/org/apache/impala/util/MetaStoreUtil.java |  16 --
 .../org/apache/impala/catalog/CatalogTest.java     |  22 +-
 .../impala/catalog/PartialCatalogInfoTest.java     |  28 ++-
 .../impala/catalog/local/LocalCatalogTest.java     |  21 +-
 .../java/org/apache/impala/service/JdbcTest.java   |  57 +++--
 tests/hs2/test_hs2.py                              | 230 +++++++++------------
 24 files changed, 405 insertions(+), 354 deletions(-)

diff --git a/be/generated-sources/gen-cpp/CMakeLists.txt b/be/generated-sources/gen-cpp/CMakeLists.txt
index 90b4930..56093f4 100644
--- a/be/generated-sources/gen-cpp/CMakeLists.txt
+++ b/be/generated-sources/gen-cpp/CMakeLists.txt
@@ -89,6 +89,8 @@ set(SRC_FILES
   ResourceProfile_types.cpp
   RuntimeProfile_constants.cpp
   RuntimeProfile_types.cpp
+  SqlConstraints_constants.cpp
+  SqlConstraints_types.cpp
   StatestoreService.cpp
   StatestoreService_constants.cpp
   StatestoreService_types.cpp
diff --git a/common/thrift/CMakeLists.txt b/common/thrift/CMakeLists.txt
index eb9489d..d5a277c 100644
--- a/common/thrift/CMakeLists.txt
+++ b/common/thrift/CMakeLists.txt
@@ -231,6 +231,7 @@ set (SRC_FILES
   ResourceProfile.thrift
   Results.thrift
   RuntimeProfile.thrift
+  SqlConstraints.thrift
   StatestoreService.thrift
   Zip.thrift
   ${TCLI_SERVICE_THRIFT}
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index a67ee9b..ee06b39 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -22,6 +22,7 @@ include "Exprs.thrift"
 include "Status.thrift"
 include "Types.thrift"
 include "hive_metastore.thrift"
+include "SqlConstraints.thrift"
 
 // Types used to represent catalog objects.
 
@@ -368,10 +369,7 @@ struct THdfsTable {
   7: optional list<Types.TNetworkAddress> network_addresses,
 
   // Primary Keys information for HDFS Tables
-  11: optional list<hive_metastore.SQLPrimaryKey> primary_keys,
-
-  // Foreign Keys information for HDFS Tables
-  12: optional list<hive_metastore.SQLForeignKey> foreign_keys
+  11: optional SqlConstraints.TSqlConstraints sql_constraints
 }
 
 struct THBaseTable {
diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift
index 1b809a8..5694f3e 100644
--- a/common/thrift/CatalogService.thrift
+++ b/common/thrift/CatalogService.thrift
@@ -24,6 +24,7 @@ include "Types.thrift"
 include "Status.thrift"
 include "Results.thrift"
 include "hive_metastore.thrift"
+include "SqlConstraints.thrift"
 
 // CatalogServer service API and related structs.
 
@@ -380,13 +381,9 @@ struct TPartialTableInfo {
   // Only used when partition files are fetched.
   7: optional list<Types.TNetworkAddress> network_addresses
 
-  // List of primary key constraints, small enough that we can
+  // SqlConstraints for the table, small enough that we can
   // return them wholesale.
-  8: optional list<hive_metastore.SQLPrimaryKey> primary_keys
-
-  // List of foreign key constraints, small enough that we can
-  // return them wholesale.
-  9: optional list<hive_metastore.SQLForeignKey> foreign_keys
+  8: optional SqlConstraints.TSqlConstraints sql_constraints
 }
 
 // Selector for partial information about a Database.
diff --git a/common/thrift/SqlConstraints.thrift b/common/thrift/SqlConstraints.thrift
new file mode 100644
index 0000000..ee4a9cf
--- /dev/null
+++ b/common/thrift/SqlConstraints.thrift
@@ -0,0 +1,31 @@
+// 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.
+
+
+namespace cpp impala
+namespace java org.apache.impala.thrift
+
+include "hive_metastore.thrift"
+
+// Represents the SQL Constraints for a table.
+struct TSqlConstraints {
+  // Primary Keys information for the given table.
+  1: required list<hive_metastore.SQLPrimaryKey> primary_keys,
+
+  // Foreign Keys information for the given table.
+  2: required list<hive_metastore.SQLForeignKey> foreign_keys
+}
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/analysis/TableDef.java b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
index 0f1afa7..40bae14 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TableDef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
@@ -255,7 +255,7 @@ class TableDef {
     final List<String> foreignKeyColNames;
 
     // Name of fk
-    final String fkConstraintName;
+    String fkConstraintName;
 
     // Fully qualified pk name. Set during analysis.
     TableName fullyQualifiedPkTableName;
@@ -293,6 +293,10 @@ class TableDef {
       return fkConstraintName;
     }
 
+    public void setConstraintName(String constraintName) {
+      fkConstraintName = constraintName;
+    }
+
     public TableName getFullyQualifiedPkTableName() {
       return fullyQualifiedPkTableName;
     }
@@ -570,22 +574,18 @@ class TableDef {
         throw new AnalysisException("VALIDATE feature is not supported yet.");
       }
 
-      String constraintName = null;
+      if (fk.getFkConstraintName() == null) {
+        fk.setConstraintName(generateConstraintName());
+      }
+
       for (int i = 0; i < fk.getForeignKeyColNames().size(); i++) {
-        if (fk.getFkConstraintName() == null) {
-          if (i == 0){
-            constraintName = generateConstraintName();
-          }
-        } else {
-          constraintName = fk.getFkConstraintName();
-        }
         SQLForeignKey sqlForeignKey = new SQLForeignKey();
         sqlForeignKey.setPktable_db(parentDb);
         sqlForeignKey.setPktable_name(fk.getPkTableName().getTbl());
         sqlForeignKey.setFktable_db(getTblName().getDb());
         sqlForeignKey.setFktable_name(getTbl());
         sqlForeignKey.setPkcolumn_name(fk.getPrimaryKeyColNames().get(i).toLowerCase());
-        sqlForeignKey.setFk_name(constraintName);
+        sqlForeignKey.setFk_name(fk.getFkConstraintName());
         sqlForeignKey.setKey_seq(i+1);
         sqlForeignKey.setFkcolumn_name(fk.getForeignKeyColNames().get(i).toLowerCase());
         sqlForeignKey.setRely_cstr(fk.isRelyCstr());
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
index b024da1..23839a9 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
@@ -184,14 +184,9 @@ public interface FeFsTable extends FeTable {
   List<? extends FeFsPartition> loadPartitions(Collection<Long> ids);
 
   /**
-   * @return: Primary keys information.
+   * @return: SQL Constraints Information.
    */
-  List<SQLPrimaryKey> getPrimaryKeys();
-
-  /**
-   * @return Foreign keys information.
-   */
-  List<SQLForeignKey> getForeignKeys();
+  SqlConstraints getSqlConstraints();
 
   /**
    * @return  List of primary keys column names, useful for toSqlUtils. In local
@@ -199,7 +194,7 @@ public interface FeFsTable extends FeTable {
    */
   default List<String> getPrimaryKeyColumnNames() throws TException {
     List<String> primaryKeyColNames = new ArrayList<>();
-    List<SQLPrimaryKey> primaryKeys = getPrimaryKeys();
+    List<SQLPrimaryKey> primaryKeys = getSqlConstraints().getPrimaryKeys();
     if (!primaryKeys.isEmpty()) {
       primaryKeys.stream().forEach(p -> primaryKeyColNames.add(p.getColumn_name()));
     }
@@ -217,7 +212,7 @@ public interface FeFsTable extends FeTable {
     // and each foreign key may contain multiple columns. The outerloop collects
     // information common to a foreign key (pk table information). The inner
     // loop collects column information.
-    List<SQLForeignKey> foreignKeys = getForeignKeys();
+    List<SQLForeignKey> foreignKeys = getSqlConstraints().getForeignKeys();
     for (int i = 0; i < foreignKeys.size(); i++) {
       String pkTableDb = foreignKeys.get(i).getPktable_db();
       String pkTableName = foreignKeys.get(i).getPktable_name();
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeTable.java b/fe/src/main/java/org/apache/impala/catalog/FeTable.java
index 3d069c2..b22e11d 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeTable.java
@@ -20,8 +20,6 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.thrift.TCatalogObjectType;
@@ -93,14 +91,9 @@ public interface FeTable {
   List<String> getColumnNames();
 
   /**
-   * @return the list of primary keys for this table.
+   * @return SQL constraints for the table.
    */
-  List<SQLPrimaryKey> getPrimaryKeys();
-
-  /**
-   * @return the list of foreign keys for this table.
-   */
-  List<SQLForeignKey> getForeignKeys();
+  SqlConstraints getSqlConstraints();
 
   /**
    * @return an unmodifiable list of all partition columns.
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 8ed52b2..c1decbb 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -47,8 +47,6 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.impala.analysis.Expr;
@@ -78,6 +76,7 @@ import org.apache.impala.thrift.TPartialPartitionInfo;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TResultSet;
 import org.apache.impala.thrift.TResultSetMetadata;
+import org.apache.impala.thrift.TSqlConstraints;
 import org.apache.impala.thrift.TTable;
 import org.apache.impala.thrift.TTableDescriptor;
 import org.apache.impala.thrift.TTableType;
@@ -245,10 +244,9 @@ public class HdfsTable extends Table implements FeFsTable {
   // for setAvroSchema().
   private boolean isSchemaLoaded_ = false;
 
-  // Primary Key and Foreign Key information. Set in load() method. An empty list could
-  // mean either the table does not have any keys or the table is not loaded.
-  private final List<SQLPrimaryKey> primaryKeys_ = new ArrayList<>();
-  private final List<SQLForeignKey> foreignKeys_ = new ArrayList<>();
+  // SQL constraints information for the table. Set in load() method.
+  private SqlConstraints sqlConstraints_ = new SqlConstraints(new ArrayList<>(),
+      new ArrayList<>());
 
   // Represents a set of storage-related statistics aggregated at the table or partition
   // level.
@@ -1051,16 +1049,10 @@ public class HdfsTable extends Table implements FeFsTable {
   private void loadConstraintsInfo(IMetaStoreClient client,
       org.apache.hadoop.hive.metastore.api.Table msTbl) throws TableLoadingException{
     try {
-      // Reset and add primary keys info and foreign keys info.
-      primaryKeys_.clear();
-      foreignKeys_.clear();
-      primaryKeys_.addAll(client.getPrimaryKeys(
-          new PrimaryKeysRequest(msTbl.getDbName(), msTbl.getTableName())));
-      foreignKeys_.addAll(client.getForeignKeys(new ForeignKeysRequest(null, null,
+      sqlConstraints_ = new SqlConstraints(client.getPrimaryKeys(
+          new PrimaryKeysRequest(msTbl.getDbName(), msTbl.getTableName())),
+          client.getForeignKeys(new ForeignKeysRequest(null, null,
           msTbl.getDbName(), msTbl.getTableName())));
-
-      // Store primary keys in a canonical order.
-      primaryKeys_.sort(new MetaStoreUtil.SqlPrimaryKeyComparator());
     } catch (Exception e) {
       throw new TableLoadingException("Failed to load primary keys/foreign keys for "
           + "table: " + getFullName(), e);
@@ -1452,10 +1444,7 @@ public class HdfsTable extends Table implements FeFsTable {
     nullColumnValue_ = hdfsTable.nullColumnValue;
     nullPartitionKeyValue_ = hdfsTable.nullPartitionKeyValue;
     hostIndex_.populate(hdfsTable.getNetwork_addresses());
-    primaryKeys_.clear();
-    primaryKeys_.addAll(hdfsTable.getPrimary_keys());
-    foreignKeys_.clear();
-    foreignKeys_.addAll(hdfsTable.getForeign_keys());
+    sqlConstraints_ =  SqlConstraints.fromThrift(hdfsTable.getSql_constraints());
     resetPartitions();
     try {
       for (Map.Entry<Long, THdfsPartition> part: hdfsTable.getPartitions().entrySet()) {
@@ -1557,10 +1546,10 @@ public class HdfsTable extends Table implements FeFsTable {
     }
 
     if (req.table_info_selector.want_table_constraints) {
-      List<SQLPrimaryKey> primaryKeys = new ArrayList<>(primaryKeys_);
-      List<SQLForeignKey> foreignKeys = new ArrayList<>(foreignKeys_);
-      resp.table_info.setPrimary_keys(primaryKeys);
-      resp.table_info.setForeign_keys(foreignKeys);
+      TSqlConstraints sqlConstraints =
+          new TSqlConstraints(sqlConstraints_.getPrimaryKeys(),
+          sqlConstraints_.getForeignKeys());
+      resp.table_info.setSql_constraints(sqlConstraints);
     }
     return resp;
   }
@@ -1623,8 +1612,7 @@ public class HdfsTable extends Table implements FeFsTable {
     THdfsTable hdfsTable = new THdfsTable(hdfsBaseDir_, getColumnNames(),
         nullPartitionKeyValue_, nullColumnValue_, idToPartition, prototypePartition);
     hdfsTable.setAvroSchema(avroSchema_);
-    hdfsTable.setPrimary_keys(primaryKeys_);
-    hdfsTable.setForeign_keys(foreignKeys_);
+    hdfsTable.setSql_constraints(sqlConstraints_.toThrift());
     if (type == ThriftObjectType.FULL) {
       // Network addresses are used only by THdfsFileBlocks which are inside
       // THdfsFileDesc, so include network addreses only when including THdfsFileDesc.
@@ -1652,13 +1640,8 @@ public class HdfsTable extends Table implements FeFsTable {
   public ListMap<TNetworkAddress> getHostIndex() { return hostIndex_; }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys() {
-    return ImmutableList.copyOf(primaryKeys_);
-  }
-
-  @Override
-  public List<SQLForeignKey> getForeignKeys() {
-    return ImmutableList.copyOf(foreignKeys_);
+  public SqlConstraints getSqlConstraints() {
+    return sqlConstraints_;
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/catalog/SqlConstraints.java b/fe/src/main/java/org/apache/impala/catalog/SqlConstraints.java
new file mode 100644
index 0000000..9ecbe60
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/SqlConstraints.java
@@ -0,0 +1,99 @@
+// 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.impala.catalog;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.impala.thrift.TSqlConstraints;
+
+/**
+ * This class encapsulates all the SQL constraints for a given table.
+ */
+public class SqlConstraints {
+
+  // List of primary keys and foreign keys for a table. An empty list could
+  // mean either the table does not have any keys or the table is not loaded.
+  private final List<SQLPrimaryKey> primaryKeys_;
+  private final List<SQLForeignKey> foreignKeys_;
+
+  public SqlConstraints(List<SQLPrimaryKey> primaryKeys,
+      List<SQLForeignKey> foreignKeys) {
+    this.primaryKeys_ = primaryKeys == null ? new ArrayList<>() : primaryKeys;
+    this.foreignKeys_ = foreignKeys == null ? new ArrayList<>() : foreignKeys;
+    sortSqlConstraints();
+  }
+
+  /**
+   * Utility method to sort SQL constraints. This is done to prevent flaky tests if HMS
+   * returns the constraints in inconsistent orders.
+   */
+  private void sortSqlConstraints() {
+    // We sort SQLPrimaryKeys in alphabetical order of the
+    // primary key name. If the primary key names are same (composite primary key), we
+    // will sort in increasing order of key_seq.
+    primaryKeys_.sort((pk1, pk2) -> {
+      int keyNameComp = pk1.getPk_name().compareTo(pk2.getPk_name());
+      if (keyNameComp == 0) {
+        return Integer.compare(pk1.getKey_seq(), pk2.getKey_seq());
+      }
+      return keyNameComp;
+    });
+
+    // We sort SQLForeignKeys in alphabetical order of the
+    // parent table name. If the parent table names are same (composite primary key in
+    // parent table), we will sort in increasing order of key_seq.
+    foreignKeys_.sort((fk1, fk2) -> {
+      int parentTableNameComp = fk1.getPktable_name().compareTo(fk2.getPktable_name());
+      if (parentTableNameComp == 0) {
+        return Integer.compare(fk1.getKey_seq(), fk2.getKey_seq());
+      }
+      return parentTableNameComp;
+    });
+  }
+
+  public List<SQLPrimaryKey> getPrimaryKeys() {
+    Preconditions.checkNotNull(primaryKeys_);
+    return ImmutableList.copyOf(primaryKeys_);
+  }
+
+  public List<SQLForeignKey> getForeignKeys() {
+    Preconditions.checkNotNull(foreignKeys_);
+    return ImmutableList.copyOf(foreignKeys_);
+  }
+
+  /**
+   * @return the thirft SqlConstraints for this table.
+   */
+  public TSqlConstraints toThrift()  {
+    Preconditions.checkNotNull(primaryKeys_);
+    Preconditions.checkNotNull(foreignKeys_);
+    TSqlConstraints tSqlConstraints = new TSqlConstraints();
+    tSqlConstraints.setPrimary_keys(primaryKeys_);
+    tSqlConstraints.setForeign_keys(foreignKeys_);
+    return tSqlConstraints;
+  }
+
+  public static SqlConstraints fromThrift(TSqlConstraints tSqlConstraints) {
+    return new SqlConstraints(tSqlConstraints.getPrimary_keys(),
+        tSqlConstraints.getForeign_keys());
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index 7df8d18..68802a9 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -33,8 +33,6 @@ import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.catalog.events.InFlightEvents;
 import org.apache.impala.common.ImpalaRuntimeException;
@@ -116,11 +114,9 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
   // map from lowercase column name to Column object.
   private final Map<String, Column> colsByName_ = new HashMap<>();
 
-  // List of primary keys associated with the table.
-  protected final List<SQLPrimaryKey> primaryKeys_ = new ArrayList<>();
-
-  // List of foreign keys associated with the table.
-  protected final List<SQLForeignKey> foreignKeys_ = new ArrayList<>();
+  // List of SQL constraints associated with the table.
+  private final SqlConstraints sqlConstraints_ = new SqlConstraints(new ArrayList<>(),
+      new ArrayList<>());
 
   // Type of this table (array of struct) that mirrors the columns. Useful for analysis.
   protected final ArrayType type_ = new ArrayType(new StructType());
@@ -661,16 +657,7 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
   public List<Column> getColumns() { return colsByPos_; }
 
   @Override // FeTable
-  public List<SQLPrimaryKey> getPrimaryKeys() {
-    // Prevent clients from modifying the primary keys list.
-    return ImmutableList.copyOf(primaryKeys_);
-  }
-
-  @Override // FeTable
-  public List<SQLForeignKey> getForeignKeys() {
-    // Prevent clients from modifying the foreign keys list.
-    return ImmutableList.copyOf(foreignKeys_);
-  }
+  public SqlConstraints getSqlConstraints()  { return sqlConstraints_; }
 
   @Override // FeTable
   public List<String> getColumnNames() { return Column.toColumnNames(colsByPos_); }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/CatalogdMetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/CatalogdMetaProvider.java
index d9c8b6c..b43c2c2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/CatalogdMetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/CatalogdMetaProvider.java
@@ -55,6 +55,7 @@ import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.catalog.ImpaladCatalog.ObjectUpdateSequencer;
 import org.apache.impala.catalog.Principal;
 import org.apache.impala.catalog.PrincipalPrivilege;
+import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.Pair;
 import org.apache.impala.service.FeSupport;
@@ -696,9 +697,13 @@ public class CatalogdMetaProvider implements MetaProvider {
                 req, "missing expected HMS table");
             addTableMetadatStorageLoadTimeToProfile(
                 resp.table_info.storage_metadata_load_time_ns);
+            List<SQLPrimaryKey> primaryKeys = resp.table_info.sql_constraints == null ?
+                new ArrayList<>() : resp.table_info.sql_constraints.getPrimary_keys();
+            List<SQLForeignKey> foreignKeys = resp.table_info.sql_constraints == null ?
+                new ArrayList<>() : resp.table_info.sql_constraints.getForeign_keys();
             return new TableMetaRefImpl(
                 dbName, tableName, resp.table_info.hms_table, resp.object_version_number,
-                resp.table_info.primary_keys, resp.table_info.foreign_keys);
+                new SqlConstraints(primaryKeys, foreignKeys));
            }
       });
     return Pair.create(ref.msTable_, (TableMetaRef)ref);
@@ -797,12 +802,9 @@ public class CatalogdMetaProvider implements MetaProvider {
   }
 
   @Override
-  public Pair<List<SQLPrimaryKey>, List<SQLForeignKey>> loadConstraints(
+  public SqlConstraints loadConstraints(
       final TableMetaRef table, Table msTbl) {
-     Pair<List<SQLPrimaryKey>, List<SQLForeignKey>> pair =
-         new Pair<>(((TableMetaRefImpl) table).primaryKeys_,
-         ((TableMetaRefImpl) table).foreignKeys_);
-     return pair;
+    return ((TableMetaRefImpl) table).sqlConstraints_;
   }
 
   @Override
@@ -1394,8 +1396,7 @@ public class CatalogdMetaProvider implements MetaProvider {
     private final String dbName_;
     private final String tableName_;
     // SQL constraints for the table, populated during loadTable().
-    private final List<SQLPrimaryKey> primaryKeys_;
-    private final List<SQLForeignKey> foreignKeys_;
+    private final SqlConstraints sqlConstraints_;
 
     /**
      * Stash the HMS Table object since we need this in order to handle some strange
@@ -1411,14 +1412,12 @@ public class CatalogdMetaProvider implements MetaProvider {
     private final long catalogVersion_;
 
     public TableMetaRefImpl(String dbName, String tableName,
-        Table msTable, long catalogVersion, List<SQLPrimaryKey> primaryKeys,
-        List<SQLForeignKey> foreignKeys) {
+        Table msTable, long catalogVersion, SqlConstraints sqlConstraints) {
       this.dbName_ = dbName;
       this.tableName_ = tableName;
       this.msTable_ = msTable;
       this.catalogVersion_ = catalogVersion;
-      this.primaryKeys_ = primaryKeys;
-      this.foreignKeys_ = foreignKeys;
+      this.sqlConstraints_ = sqlConstraints;
     }
 
     @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
index 620c966..0fc44ae 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
@@ -19,7 +19,6 @@ package org.apache.impala.catalog.local;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -33,8 +32,6 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.impala.authorization.AuthorizationPolicy;
@@ -43,6 +40,7 @@ import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.catalog.MetaStoreClientPool;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
+import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.common.Pair;
 import org.apache.impala.compat.MetastoreShim;
 import org.apache.impala.service.BackendConfig;
@@ -164,21 +162,16 @@ class DirectMetaProvider implements MetaProvider {
   }
 
   @Override
-  public Pair<List<SQLPrimaryKey>, List<SQLForeignKey>> loadConstraints(
+  public SqlConstraints loadConstraints(
       TableMetaRef table, Table msTbl) throws TException {
-    List<SQLPrimaryKey> primaryKeys = new ArrayList<>();
-    List<SQLForeignKey> foreignKeys = new ArrayList<>();
-
+    SqlConstraints sqlConstraints;
     try (MetaStoreClient c = msClientPool_.getClient()) {
-      primaryKeys.addAll(c.getHiveClient().getPrimaryKeys(
-          new PrimaryKeysRequest(msTbl.getDbName(), msTbl.getTableName())));
-      foreignKeys.addAll(c.getHiveClient().getForeignKeys(new ForeignKeysRequest(null,
+      sqlConstraints = new SqlConstraints(c.getHiveClient().getPrimaryKeys(
+          new PrimaryKeysRequest(msTbl.getDbName(), msTbl.getTableName())),
+          c.getHiveClient().getForeignKeys(new ForeignKeysRequest(null,
           null, msTbl.getDbName(), msTbl.getTableName())));
-
-      // Store primary keys in a canonical order.
-      primaryKeys.sort(new MetaStoreUtil.SqlPrimaryKeyComparator());
     }
-    return new Pair<>(primaryKeys, foreignKeys);
+    return sqlConstraints;
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
index d9cf712..b464f47 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
@@ -31,8 +31,6 @@ import org.apache.avro.Schema;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -48,12 +46,12 @@ import org.apache.impala.catalog.FeFsTable;
 import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.PrunablePartition;
+import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.catalog.local.MetaProvider.PartitionMetadata;
 import org.apache.impala.catalog.local.MetaProvider.PartitionRef;
 import org.apache.impala.catalog.local.MetaProvider.TableMetaRef;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FileSystemUtil;
-import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.CatalogObjectsConstants;
 import org.apache.impala.thrift.THdfsPartition;
 import org.apache.impala.thrift.THdfsTable;
@@ -109,16 +107,9 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
 
 
   /**
-   * List of primary keys associated with the table. An empty list could
-   * mean either the table does not have any primary keys or the table is not loaded.
+   * SQL constraints associated with the table.
    */
-  private final List<SQLPrimaryKey> primaryKeys_ = new ArrayList<>();
-
-  /**
-   * List of foreign keys associated with the table. An empty list could
-   * mean either the table does not have any foreign keys or the table is not loaded.
-   */
-  private final List<SQLForeignKey> foreignKeys_ = new ArrayList<>();
+  private SqlConstraints sqlConstraints_;
 
   /**
    * The Avro schema for this table. Non-null if this table is an Avro table.
@@ -513,13 +504,7 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
    * Populate constraint information by making a request to MetaProvider.
    */
   private void loadConstraints() throws TException {
-    Pair<List<SQLPrimaryKey>, List<SQLForeignKey>> constraints =
-        db_.getCatalog().getMetaProvider().loadConstraints(ref_, msTable_);
-    // clear and load constraints.
-    primaryKeys_.clear();
-    foreignKeys_.clear();
-    primaryKeys_.addAll(constraints.first);
-    foreignKeys_.addAll(constraints.second);
+    sqlConstraints_ = db_.getCatalog().getMetaProvider().loadConstraints(ref_, msTable_);
   }
 
   /**
@@ -552,26 +537,13 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys() {
-    try {
-      loadConstraints();
-    } catch (TException e) {
-      throw new LocalCatalogException("Failed to load primary keys for table "
-          + getFullName(), e);
-    }
-    // Once loaded, clients must not be able to modify this list.
-    return ImmutableList.copyOf(primaryKeys_);
-  }
-
-  @Override
-  public List<SQLForeignKey> getForeignKeys() {
+  public SqlConstraints getSqlConstraints() {
     try {
       loadConstraints();
     } catch (TException e) {
-      throw new LocalCatalogException("Failed to load foreign keys for table "
-          + getFullName(), e);
+      throw new LocalCatalogException("Failed to load primary keys/foreign keys for "
+          + "table " + getFullName(), e);
     }
-    // Once loaded, clients must not be able to modify this list.
-    return ImmutableList.copyOf(foreignKeys_);
+    return sqlConstraints_;
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
index 68f406e..ccb05dc 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
@@ -18,15 +18,12 @@
 package org.apache.impala.catalog.local;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.catalog.ArrayType;
@@ -38,6 +35,7 @@ import org.apache.impala.catalog.FeTable;
 import org.apache.impala.catalog.HBaseTable;
 import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.KuduTable;
+import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.catalog.StructField;
 import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.TableLoadingException;
@@ -211,15 +209,8 @@ abstract class LocalTable implements FeTable {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys() {
-    // TODO: return primary keys after IMPALA-9158
-    return Collections.emptyList();
-  }
-
-  @Override
-  public List<SQLForeignKey> getForeignKeys() {
-    // TODO: return foreign keys after IMPALA-9158
-    return Collections.emptyList();
+  public SqlConstraints getSqlConstraints() {
+    return new SqlConstraints(new ArrayList<>(), new ArrayList<>());
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
index ef87c25..6b79567 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
@@ -25,13 +25,12 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.impala.authorization.AuthorizationPolicy;
 import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
+import org.apache.impala.catalog.SqlConstraints;
 import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TNetworkAddress;
 import org.apache.impala.util.ListMap;
@@ -80,7 +79,10 @@ public interface MetaProvider {
   List<PartitionRef> loadPartitionList(TableMetaRef table)
       throws MetaException, TException;
 
-  Pair<List<SQLPrimaryKey>, List<SQLForeignKey>> loadConstraints(TableMetaRef table,
+  /**
+   * Load the list of SQL constraints for the given table.
+   */
+  SqlConstraints loadConstraints(TableMetaRef table,
       Table msTbl) throws MetaException, TException;
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index fdce34f..3e1431d 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -933,7 +933,8 @@ public class Frontend {
       throws InternalException {
     Preconditions.checkNotNull(table);
     List<SQLPrimaryKey> pkList;
-    pkList = table.getPrimaryKeys();
+    pkList = table.getSqlConstraints().getPrimaryKeys();
+    Preconditions.checkNotNull(pkList);
     for (SQLPrimaryKey pk : pkList) {
       if (authzFactory_.getAuthorizationConfig().isEnabled()) {
         PrivilegeRequest privilegeRequest = new PrivilegeRequestBuilder(
@@ -974,7 +975,9 @@ public class Frontend {
     // but we return the "SQLFOreignKey" for col3.
     Set<String> omitList = new HashSet<>();
     List<SQLForeignKey> fkList = new ArrayList<>();
-    for (SQLForeignKey fk : table.getForeignKeys()) {
+    List<SQLForeignKey> foreignKeys = table.getSqlConstraints().getForeignKeys();
+    Preconditions.checkNotNull(foreignKeys);
+    for (SQLForeignKey fk : foreignKeys) {
       String fkName = fk.getFk_name();
       if (!omitList.contains(fkName)) {
         if (authzFactory_.getAuthorizationConfig().isEnabled()) {
@@ -999,7 +1002,7 @@ public class Frontend {
         }
       }
     }
-    for (SQLForeignKey fk : table.getForeignKeys()) {
+    for (SQLForeignKey fk : foreignKeys) {
       if (!omitList.contains(fk.getFk_name())) {
         fkList.add(fk);
       }
diff --git a/fe/src/main/java/org/apache/impala/service/MetadataOp.java b/fe/src/main/java/org/apache/impala/service/MetadataOp.java
index 4123072..78dcfb8 100644
--- a/fe/src/main/java/org/apache/impala/service/MetadataOp.java
+++ b/fe/src/main/java/org/apache/impala/service/MetadataOp.java
@@ -644,8 +644,8 @@ public class MetadataOp {
 
     for (int i = 0; i < dbsMetadata.dbs.size(); ++i) {
       for (int j = 0; j < dbsMetadata.tableNames.get(i).size(); ++j) {
-        TResultRow row = new TResultRow();
         for (SQLPrimaryKey pk : dbsMetadata.primaryKeys.get(i).get(j)) {
+          TResultRow row = new TResultRow();
           row.colVals = Lists.newArrayList();
           row.colVals.add(EMPTY_COL_VAL);
           row.colVals.add(createTColumnValue(pk.getTable_db()));
@@ -710,9 +710,8 @@ public class MetadataOp {
         List<SQLForeignKey> filteredForeignKeys =
             filterForeignKeys(dbsMetadata.foreignKeys.get(i).get(j), parentSchemaName,
                 parentTableName);
-
-        TResultRow row = new TResultRow();
         for (SQLForeignKey fk : filteredForeignKeys) {
+          TResultRow row = new TResultRow();
           row.colVals = Lists.newArrayList();
           row.colVals.add(EMPTY_COL_VAL); // PKTABLE_CAT
           row.colVals.add(createTColumnValue(fk.getPktable_db()));
diff --git a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
index 54af872..8c8c527 100644
--- a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
@@ -357,20 +357,4 @@ public class MetaStoreUtil {
     Preconditions.checkNotNull(msTbl);
     return msTbl.getSd().getNumBuckets() > 0;
   }
-
-  /**
-   * A custom comparator class to sort SQLPrimaryKeys in alphabetical order of the
-   * primary key name. If the primary key names are same(composite primary key), we
-   * will sort in increasing order of key_seq.
-   */
-  public static class SqlPrimaryKeyComparator implements Comparator<SQLPrimaryKey> {
-    @Override
-    public int compare(SQLPrimaryKey pk1, SQLPrimaryKey pk2) {
-      int keyNameComp = pk1.getPk_name().compareTo(pk2.getPk_name());
-      if (keyNameComp == 0) {
-        return Integer.compare(pk1.getKey_seq(), pk2.getKey_seq());
-      }
-      return keyNameComp;
-    }
-  }
 }
diff --git a/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
index 1fbc5e8..3a27e19 100644
--- a/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
@@ -445,8 +445,8 @@ public class CatalogTest {
         "test");
     assertNotNull(t);
     assertTrue(t instanceof FeFsTable);
-    List<SQLPrimaryKey> primaryKeys = t.getPrimaryKeys();
-    List<SQLForeignKey> foreignKeys = t.getForeignKeys();
+    List<SQLPrimaryKey> primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    List<SQLForeignKey> foreignKeys = t.getSqlConstraints().getForeignKeys();
     assertEquals(2, primaryKeys.size());
     assertEquals(0, foreignKeys.size());
     for (SQLPrimaryKey pk: primaryKeys) {
@@ -462,8 +462,8 @@ public class CatalogTest {
     t = (FeFsTable) catalog_.getOrLoadTable("functional", "child_table", "test");
     assertNotNull(t);
     assertTrue(t instanceof FeFsTable);
-    primaryKeys = t.getPrimaryKeys();
-    foreignKeys = t.getForeignKeys();
+    primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    foreignKeys = t.getSqlConstraints().getForeignKeys();
     assertEquals(1, primaryKeys.size());
     assertEquals(3, foreignKeys.size());
     assertEquals("functional", primaryKeys.get(0).getTable_db());
@@ -479,6 +479,20 @@ public class CatalogTest {
     assertEquals("id", foreignKeys.get(0).getPkcolumn_name());
     assertEquals("year", foreignKeys.get(1).getPkcolumn_name());
     assertEquals("a", foreignKeys.get(2).getPkcolumn_name());
+    // FK name for the composite primary key (id, year) should be equal.
+    assertEquals(foreignKeys.get(0).getFk_name(), foreignKeys.get(1).getFk_name());
+
+    // Check tables without constraints.
+    t = (FeFsTable) catalog_.getOrLoadTable("functional", "alltypes",
+        "test");
+    assertNotNull(t);
+    assertTrue(t instanceof FeFsTable);
+    primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    foreignKeys = t.getSqlConstraints().getForeignKeys();
+    assertNotNull(primaryKeys);
+    assertNotNull(foreignKeys);
+    assertEquals(0, primaryKeys.size());
+    assertEquals(0, foreignKeys.size());
   }
 
   public static void checkAllTypesPartitioning(FeFsTable table,
diff --git a/fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java b/fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
index 01599e7..ba677d1 100644
--- a/fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
@@ -20,6 +20,7 @@ package org.apache.impala.catalog;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
@@ -254,8 +255,8 @@ public class PartialCatalogInfoTest {
     req.table_info_selector.want_table_constraints = true;
 
     TGetPartialCatalogObjectResponse resp = sendRequest(req);
-    List<SQLPrimaryKey> primaryKeys = resp.table_info.primary_keys;
-    List<SQLForeignKey> foreignKeys = resp.table_info.foreign_keys;
+    List<SQLPrimaryKey> primaryKeys = resp.table_info.sql_constraints.primary_keys;
+    List<SQLForeignKey> foreignKeys = resp.table_info.sql_constraints.foreign_keys;
 
     assertEquals(2, primaryKeys.size());
     assertEquals(0, foreignKeys.size());
@@ -276,8 +277,8 @@ public class PartialCatalogInfoTest {
     req.table_info_selector.want_table_constraints = true;
 
     resp = sendRequest(req);
-    primaryKeys = resp.table_info.primary_keys;
-    foreignKeys = resp.table_info.foreign_keys;
+    primaryKeys = resp.table_info.sql_constraints.primary_keys;
+    foreignKeys = resp.table_info.sql_constraints.foreign_keys;
 
     assertEquals(1, primaryKeys.size());
     assertEquals(3, foreignKeys.size());
@@ -294,6 +295,25 @@ public class PartialCatalogInfoTest {
     assertEquals("id", foreignKeys.get(0).getPkcolumn_name());
     assertEquals("year", foreignKeys.get(1).getPkcolumn_name());
     assertEquals("a", foreignKeys.get(2).getPkcolumn_name());
+    // FK name for the composite primary key (id, year) should be equal.
+    assertEquals(foreignKeys.get(0).getFk_name(), foreignKeys.get(1).getFk_name());
+
+    // Check tables without constraints.
+    req = new TGetPartialCatalogObjectRequest();
+    req.object_desc = new TCatalogObject();
+    req.object_desc.setType(TCatalogObjectType.TABLE);
+    req.object_desc.table = new TTable("functional", "alltypes");
+    req.table_info_selector = new TTableInfoSelector();
+    req.table_info_selector.want_hms_table = true;
+    req.table_info_selector.want_table_constraints = true;
+
+    resp = sendRequest(req);
+    primaryKeys = resp.table_info.sql_constraints.primary_keys;
+    foreignKeys = resp.table_info.sql_constraints.foreign_keys;
+    assertNotNull(primaryKeys);
+    assertNotNull(foreignKeys);
+    assertEquals(0, primaryKeys.size());
+    assertEquals(0, foreignKeys.size());
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
index 67a36d9..b5f44c0 100644
--- a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
@@ -172,8 +172,8 @@ public class LocalCatalogTest {
     FeFsTable t = (FeFsTable) catalog_.getTable("functional", "parent_table");
     assertNotNull(t);
     assertTrue(t instanceof LocalFsTable);
-    List<SQLPrimaryKey> primaryKeys = t.getPrimaryKeys();
-    List<SQLForeignKey> foreignKeys = t.getForeignKeys();
+    List<SQLPrimaryKey> primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    List<SQLForeignKey> foreignKeys = t.getSqlConstraints().getForeignKeys();
     assertEquals(2, primaryKeys.size());
     assertEquals(0, foreignKeys.size());
     for (SQLPrimaryKey pk: primaryKeys) {
@@ -186,8 +186,8 @@ public class LocalCatalogTest {
     t = (FeFsTable) catalog_.getTable("functional", "child_table");
     assertNotNull(t);
     assertTrue(t instanceof LocalFsTable);
-    primaryKeys = t.getPrimaryKeys();
-    foreignKeys = t.getForeignKeys();
+    primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    foreignKeys = t.getSqlConstraints().getForeignKeys();
     assertEquals(1, primaryKeys.size());
     assertEquals(3, foreignKeys.size());
     assertEquals("functional", primaryKeys.get(0).getTable_db());
@@ -203,6 +203,19 @@ public class LocalCatalogTest {
     assertEquals("id", foreignKeys.get(0).getPkcolumn_name());
     assertEquals("year", foreignKeys.get(1).getPkcolumn_name());
     assertEquals("a", foreignKeys.get(2).getPkcolumn_name());
+    // FK name for the composite primary key (id, year) should be equal.
+    assertEquals(foreignKeys.get(0).getFk_name(), foreignKeys.get(1).getFk_name());
+
+    // Check tables without constraints.
+    t = (FeFsTable) catalog_.getTable("functional", "alltypes");
+    assertNotNull(t);
+    assertTrue(t instanceof FeFsTable);
+    primaryKeys = t.getSqlConstraints().getPrimaryKeys();
+    foreignKeys = t.getSqlConstraints().getForeignKeys();
+    assertNotNull(primaryKeys);
+    assertNotNull(foreignKeys);
+    assertEquals(0, primaryKeys.size());
+    assertEquals(0, foreignKeys.size());
   }
 
   /**
diff --git a/fe/src/test/java/org/apache/impala/service/JdbcTest.java b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
index 66b06ea..7d027ef 100644
--- a/fe/src/test/java/org/apache/impala/service/JdbcTest.java
+++ b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
@@ -428,19 +428,18 @@ public class JdbcTest extends JdbcTestBase {
     ResultSet rs = con_.getMetaData().getPrimaryKeys(null, "functional", "parent_table");
     ResultSetMetaData md = rs.getMetaData();
     assertEquals("Incorrect number of columns seen", 6, md.getColumnCount());
-    // TODO (IMPALA-9158): Remove this check.
-    if (!TestUtils.isCatalogV2Enabled("localhost", 25020)) {
-      int pkCount = 0;
-      while (rs.next()) {
-        pkCount++;
-        assertEquals("", rs.getString("TABLE_CAT"));
-        assertEquals("functional", rs.getString("TABLE_SCHEM"));
-        assertEquals("parent_table", rs.getString("TABLE_NAME"));
-        assertTrue(pkList.contains(rs.getString("COLUMN_NAME")));
-        assertTrue(rs.getString("PK_NAME").length() > 0);
-      }
-      assertEquals(2, pkCount);
+    int pkCount = 0;
+    while (rs.next()) {
+      pkCount++;
+      assertEquals("", rs.getString("TABLE_CAT"));
+      assertEquals("functional", rs.getString("TABLE_SCHEM"));
+      assertEquals("parent_table", rs.getString("TABLE_NAME"));
+      // functional.parent_table should always return only two primary keys.
+      assertTrue(pkCount <= 2);
+      assertEquals(pkList.get(pkCount - 1), rs.getString("COLUMN_NAME"));
+      assertTrue(rs.getString("PK_NAME").length() > 0);
     }
+    assertEquals(2, pkCount);
   }
 
   @Test
@@ -451,24 +450,24 @@ public class JdbcTest extends JdbcTestBase {
     ResultSetMetaData md = rs.getMetaData();
     assertEquals("Incorrect number of columns seen for primary key.",
         14, md.getColumnCount());
-    // TODO (IMPALA-9158): Remove this check.
-    if (!TestUtils.isCatalogV2Enabled("localhost", 25020)) {
-      List<String> colList = new ArrayList<>(Arrays.asList("id", "year"));
-      int fkCount = 0;
-      while (rs.next()) {
-        fkCount++;
-        assertEquals("", rs.getString("PKTABLE_CAT"));
-        assertEquals("functional", rs.getString("PKTABLE_SCHEM"));
-        assertEquals("parent_table", rs.getString("PKTABLE_NAME"));
-        assertTrue(colList.contains(rs.getString("PKCOLUMN_NAME")));
-        assertTrue(rs.getString("FK_NAME").length() > 0);
-        assertEquals("", rs.getString("FKTABLE_CAT"));
-        assertEquals("functional", rs.getString("FKTABLE_SCHEM"));
-        assertEquals("child_table", rs.getString("FKTABLE_NAME"));
-        assertTrue(colList.contains(rs.getString("FKCOLUMN_NAME")));
-      }
-      assertEquals(2, fkCount);
+    List<String> colList = new ArrayList<>(Arrays.asList("id", "year"));
+    int fkCount = 0;
+    while (rs.next()) {
+      fkCount++;
+      assertEquals("", rs.getString("PKTABLE_CAT"));
+      assertEquals("functional", rs.getString("PKTABLE_SCHEM"));
+      assertEquals("parent_table", rs.getString("PKTABLE_NAME"));
+      // functional.child_table should always return only two foreign keys for
+      // parent_table.
+      assertTrue(fkCount <= 2);
+      assertEquals(colList.get(fkCount - 1), rs.getString("PKCOLUMN_NAME"));
+      assertTrue(rs.getString("FK_NAME").length() > 0);
+      assertEquals("", rs.getString("FKTABLE_CAT"));
+      assertEquals("functional", rs.getString("FKTABLE_SCHEM"));
+      assertEquals("child_table", rs.getString("FKTABLE_NAME"));
+      assertTrue(colList.contains(rs.getString("FKCOLUMN_NAME")));
     }
+    assertEquals(2, fkCount);
   }
 
   @Test
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index 02b51a3..c0c4237 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -491,139 +491,115 @@ class TestHS2(HS2TestSuite):
     finally:
       self.execute_query("drop table {0}".format(table))
 
-  @needs_session_cluster_properties()
-  def test_get_primary_keys(self, cluster_properties, unique_database):
-    table = "pk"
-    self.execute_query("use {0}".format(unique_database))
-    self.execute_query("drop table if exists {0}".format(table))
-    self.execute_query("""
-        create table {0} (id int, year string, primary key(id, year))""".format(
-        table))
-    pks = ["id", "year"]
-    try:
-      req = TCLIService.TGetPrimaryKeysReq()
-      req.sessionHandle = self.session_handle
-      req.schemaName = unique_database
-      req.tableName = table
+  @needs_session()
+  def test_get_primary_keys(self):
+    req = TCLIService.TGetPrimaryKeysReq()
+    req.sessionHandle = self.session_handle
+    req.schemaName = 'functional'
+    req.tableName = 'parent_table'
 
-      get_primary_keys_resp = self.hs2_client.GetPrimaryKeys(req)
-      TestHS2.check_response(get_primary_keys_resp)
-
-      fetch_results_resp = self._fetch_results(
-          get_primary_keys_resp.operationHandle, 100)
-
-      if not cluster_properties.is_catalog_v2_cluster():
-        for i in range(len(pks)):
-          results = fetch_results_resp.results
-          table_cat = results.columns[0].stringVal.values[i]
-          table_schema = results.columns[1].stringVal.values[i]
-          table_name = results.columns[2].stringVal.values[i]
-          pk_col_name = results.columns[3].stringVal.values[i]
-          pk_name = results.columns[5].stringVal.values[i]
-          assert table_cat == ''
-          assert table_schema == unique_database
-          assert table_name == table
-          assert pk_col_name in pks
-          assert len(pk_name) > 0
+    get_primary_keys_resp = self.hs2_client.GetPrimaryKeys(req)
+    TestHS2.check_response(get_primary_keys_resp)
 
-    finally:
-      self.execute_query("drop table {0}".format(table))
+    fetch_results_resp = self._fetch_results(
+        get_primary_keys_resp.operationHandle, 100)
 
-  @needs_session_cluster_properties()
-  def test_get_cross_reference(self, cluster_properties, unique_database):
-    parent_table_1 = "pk_1"
-    parent_table_2 = "pk_2"
-    fk_table = "fk"
-    self.execute_query("use {0}".format(unique_database))
-    self.execute_query("drop table if exists {0}".format(parent_table_1))
-    self.execute_query("""
-        create table {0} (id int, year string, primary key(id, year))""".format(
-        parent_table_1))
-    self.execute_query("drop table if exists {0}".format(parent_table_2))
-    self.execute_query("""
-        create table {0} (a int, b string, primary key(a))""".format(
-        parent_table_2))
-    self.execute_query("drop table if exists {0}".format(fk_table))
-    self.execute_query("""
-        create table {0} (seq int, id int, year string, a int, primary key(seq)
-        , foreign key(id, year) references {1}(id, year), foreign key(a)
-        references {2}(a))
-        """.format(fk_table, parent_table_1, parent_table_2))
-    pk_column_names = ["id", "year", "a"]
-    try:
-      req = TCLIService.TGetCrossReferenceReq()
-      req.sessionHandle = self.session_handle
-      req.parentSchemaName = unique_database
-      req.foreignSchemaName = unique_database
-      req.parentTableName = parent_table_1
-      req.foreignTableName = fk_table
-
-      get_foreign_keys_resp = self.hs2_client.GetCrossReference(req)
-      TestHS2.check_response(get_foreign_keys_resp)
-
-      fetch_results_resp = self._fetch_results(
-          get_foreign_keys_resp.operationHandle, 100)
-
-      results = fetch_results_resp.results
-
-      if not cluster_properties.is_catalog_v2_cluster():
-        for i in range(2):
-          parent_table_cat = results.columns[0].stringVal.values[i]
-          parent_table_schema = results.columns[1].stringVal.values[i]
-          parent_table_name = results.columns[2].stringVal.values[i]
-          parent_col_name = results.columns[3].stringVal.values[i]
-          foreign_table_cat = results.columns[4].stringVal.values[i]
-          foreign_table_schema = results.columns[5].stringVal.values[i]
-          foreign_table_name = results.columns[6].stringVal.values[i]
-          foreign_col_name = results.columns[7].stringVal.values[i]
-
-          assert parent_table_cat == ''
-          assert parent_table_schema == unique_database
-          assert parent_table_name == parent_table_1
-          assert parent_col_name in pk_column_names
-          assert foreign_table_cat == ''
-          assert foreign_table_schema == unique_database
-          assert foreign_table_name == fk_table
-          assert foreign_col_name in pk_column_names
-
-        # Get all foreign keys from the FK side by not setting pkTableSchema
-        # and pkTable name in the request.
-        req = TCLIService.TGetCrossReferenceReq()
-        req.sessionHandle = self.session_handle
-        req.foreignSchemaName = unique_database
-        req.foreignTableName = fk_table
-
-        get_foreign_keys_resp = self.hs2_client.GetCrossReference(req)
-        TestHS2.check_response(get_foreign_keys_resp)
-
-        fetch_results_resp = self._fetch_results(
-            get_foreign_keys_resp.operationHandle, 100)
+    results = fetch_results_resp.results
+    for i in range(2):
+      table_cat = results.columns[0].stringVal.values[i]
+      table_schema = results.columns[1].stringVal.values[i]
+      table_name = results.columns[2].stringVal.values[i]
+      pk_name = results.columns[5].stringVal.values[i]
+      assert table_cat == ''
+      assert table_schema == 'functional'
+      assert table_name == 'parent_table'
+      assert len(pk_name) > 0
+
+    # Assert PK column names.
+    assert results.columns[3].stringVal.values[0] == 'id'
+    assert results.columns[3].stringVal.values[1] == 'year'
 
-        results = fetch_results_resp.results
+  @needs_session()
+  def test_get_cross_reference(self):
+    req = TCLIService.TGetCrossReferenceReq()
+    req.sessionHandle = self.session_handle
+    req.parentSchemaName = "functional"
+    req.foreignSchemaName = "functional"
+    req.parentTableName = "parent_table"
+    req.foreignTableName = "child_table"
 
-        pk_table_names = [parent_table_1, parent_table_2]
-        for i in range(len(pk_column_names)):
-          parent_table_cat = results.columns[0].stringVal.values[i]
-          parent_table_schema = results.columns[1].stringVal.values[i]
-          parent_table_name = results.columns[2].stringVal.values[i]
-          parent_col_name = results.columns[3].stringVal.values[i]
-          foreign_table_cat = results.columns[4].stringVal.values[i]
-          foreign_table_schema = results.columns[5].stringVal.values[i]
-          foreign_table_name = results.columns[6].stringVal.values[i]
-          foreign_col_name = results.columns[7].stringVal.values[i]
-          assert parent_table_cat == ''
-          assert parent_table_schema == unique_database
-          assert parent_table_name in pk_table_names
-          assert parent_col_name in pk_column_names
-          assert foreign_table_cat == ''
-          assert foreign_table_schema == unique_database
-          assert foreign_table_name == fk_table
-          assert foreign_col_name in pk_column_names
+    get_foreign_keys_resp = self.hs2_client.GetCrossReference(req)
+    TestHS2.check_response(get_foreign_keys_resp)
 
-    finally:
-      self.execute_query("drop table {0}".format(parent_table_1))
-      self.execute_query("drop table {0}".format(parent_table_2))
-      self.execute_query("drop table {0}".format(fk_table))
+    fetch_results_resp = self._fetch_results(
+        get_foreign_keys_resp.operationHandle, 100)
+
+    results = fetch_results_resp.results
+
+    for i in range(2):
+      parent_table_cat = results.columns[0].stringVal.values[i]
+      parent_table_schema = results.columns[1].stringVal.values[i]
+      parent_table_name = results.columns[2].stringVal.values[i]
+      foreign_table_cat = results.columns[4].stringVal.values[i]
+      foreign_table_schema = results.columns[5].stringVal.values[i]
+      foreign_table_name = results.columns[6].stringVal.values[i]
+      assert parent_table_cat == ''
+      assert parent_table_schema == 'functional'
+      assert parent_table_name == 'parent_table'
+      assert foreign_table_cat == ''
+      assert foreign_table_schema == 'functional'
+      assert foreign_table_name == 'child_table'
+
+    # Assert PK column names.
+    assert results.columns[3].stringVal.values[0] == 'id'
+    assert results.columns[3].stringVal.values[1] == 'year'
+
+    # Assert FK column names.
+    assert results.columns[7].stringVal.values[0] == 'id'
+    assert results.columns[7].stringVal.values[1] == 'year'
+
+    # Get all foreign keys from the FK side by not setting pkTableSchema
+    # and pkTable name in the request.
+    req = TCLIService.TGetCrossReferenceReq()
+    req.sessionHandle = self.session_handle
+    req.foreignSchemaName = 'functional'
+    req.foreignTableName = 'child_table'
+
+    get_foreign_keys_resp = self.hs2_client.GetCrossReference(req)
+    TestHS2.check_response(get_foreign_keys_resp)
+
+    fetch_results_resp = self._fetch_results(
+        get_foreign_keys_resp.operationHandle, 100)
+
+    results = fetch_results_resp.results
+
+    for i in range(3):
+      parent_table_cat = results.columns[0].stringVal.values[i]
+      parent_table_schema = results.columns[1].stringVal.values[i]
+      foreign_table_cat = results.columns[4].stringVal.values[i]
+      foreign_table_schema = results.columns[5].stringVal.values[i]
+      foreign_table_name = results.columns[6].stringVal.values[i]
+      assert parent_table_cat == ''
+      assert parent_table_schema == 'functional'
+      assert foreign_table_cat == ''
+      assert foreign_table_schema == 'functional'
+      assert foreign_table_name == 'child_table'
+
+    # First two FKs have 'parent_table' as PK table.
+    pk_table_names = ['parent_table', 'parent_table', 'parent_table_2']
+    for i in range(len(pk_table_names)):
+      parent_table_name = results.columns[2].stringVal.values[i]
+      parent_table_name == pk_table_names[i]
+
+      # Assert PK column names.
+    assert results.columns[3].stringVal.values[0] == 'id'
+    assert results.columns[3].stringVal.values[1] == 'year'
+    assert results.columns[3].stringVal.values[2] == 'a'
+
+    # Assert FK column names.
+    assert results.columns[7].stringVal.values[0] == 'id'
+    assert results.columns[7].stringVal.values[1] == 'year'
+    assert results.columns[7].stringVal.values[2] == 'a'
 
   @needs_session(conf_overlay={"idle_session_timeout": "5"})
   def test_get_operation_status_session_timeout(self):