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/09/11 12:31:21 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_r486978810



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##########
@@ -1155,22 +1150,19 @@ void dumpConstraintMetadata(String dbName, String tblName, Path dbRoot, Hive hiv
       Path constraintsRoot = new Path(dbRoot, ReplUtils.CONSTRAINTS_ROOT_DIR_NAME);
       Path commonConstraintsFile = new Path(constraintsRoot, ConstraintFileType.COMMON.getPrefix() + tblName);
       Path fkConstraintsFile = new Path(constraintsRoot, ConstraintFileType.FOREIGNKEY.getPrefix() + tblName);
-      List<SQLPrimaryKey> pks = hiveDb.getPrimaryKeyList(dbName, tblName);
-      List<SQLForeignKey> fks = hiveDb.getForeignKeyList(dbName, tblName);
-      List<SQLUniqueConstraint> uks = hiveDb.getUniqueConstraintList(dbName, tblName);
-      List<SQLNotNullConstraint> nns = hiveDb.getNotNullConstraintList(dbName, tblName);
-      if ((pks != null && !pks.isEmpty()) || (uks != null && !uks.isEmpty())
-          || (nns != null && !nns.isEmpty())) {
+      SQLAllTableConstraints tableConstraints = hiveDb.getTableConstraints(dbName,tblName);
+      if ((tableConstraints.getPrimaryKeys() != null && !tableConstraints.getPrimaryKeys().isEmpty()) || (tableConstraints.getUniqueConstraints() != null && !tableConstraints.getUniqueConstraints().isEmpty())

Review comment:
       Can add utility method to check for null and empty of given list. Used multiple times. Also use local variables to reduce the code.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -5661,184 +5663,79 @@ public void dropConstraint(String dbName, String tableName, String constraintNam
     }
   }
 
-  public List<SQLDefaultConstraint> getDefaultConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
+  public SQLAllTableConstraints getTableConstraints(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
+      AllTableConstraintsRequest tableConstraintsRequest = new AllTableConstraintsRequest();
+      tableConstraintsRequest.setDbName(dbName);
+      tableConstraintsRequest.setTblName(tblName);
+      tableConstraintsRequest.setCatName(getDefaultCatalog(conf));
+      return getMSC().getAllTableConstraints(tableConstraintsRequest);
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
       throw new HiveException(e);
     }
   }
-
-  public List<SQLCheckConstraint> getCheckConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
-    try {
-      return getMSC().getCheckConstraints(new CheckConstraintsRequest(getDefaultCatalog(conf),
-          dbName, tblName));
-    } catch (NoSuchObjectException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new HiveException(e);
-    }
+  public TableConstraintsInfo getAllTableConstraints(String dbName, String tblName) throws HiveException {
+    return getTableConstraints(dbName, tblName, false, false);
   }
 
-  /**
-   * Get all primary key columns associated with the table.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Primary Key associated with the table.
-   * @throws HiveException
-   */
-  public PrimaryKeyInfo getPrimaryKeys(String dbName, String tblName) throws HiveException {
-    return getPrimaryKeys(dbName, tblName, false);
+  public TableConstraintsInfo getReliableAndEnableTableConstraints(String dbName, String tblName) throws HiveException {
+    return getTableConstraints(dbName, tblName, true, true);
   }
 
-  /**
-   * Get primary key columns associated with the table that are available for optimization.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Primary Key associated with the table.
-   * @throws HiveException
-   */
-  public PrimaryKeyInfo getReliablePrimaryKeys(String dbName, String tblName) throws HiveException {
-    return getPrimaryKeys(dbName, tblName, true);
-  }
-
-  private PrimaryKeyInfo getPrimaryKeys(String dbName, String tblName, boolean onlyReliable)
+  private TableConstraintsInfo getTableConstraints(String dbName, String tblName, boolean reliable, boolean enable)

Review comment:
       nit: Use "fetchReliable" and "fetchEnabled" instead of "reliable" and "enable" as it sound like flag to enable something.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
##########
@@ -116,22 +116,12 @@
   private transient Boolean outdatedForRewritingMaterializedView;
 
   /** Constraint related objects */
-  private transient PrimaryKeyInfo pki;
-  private transient ForeignKeyInfo fki;
-  private transient UniqueConstraint uki;
-  private transient NotNullConstraint nnc;
-  private transient DefaultConstraint dc;
-  private transient CheckConstraint cc;
+  private transient TableConstraintsInfo tableConstraintsInfo;
 
   /** Constraint related flags
    *  This is to track if constraints are retrieved from metastore or not
    */
-  private transient boolean isPKFetched=false;
-  private transient boolean isFKFetched=false;
-  private transient boolean isUniqueFetched=false;
-  private transient boolean isNotNullFetched=false;
-  private transient boolean isDefaultFetched=false;
-  private transient boolean isCheckFetched=false;
+  private transient boolean isTableConstraintsFetched=false;

Review comment:
       nit: Add space before and after "=".

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -3631,6 +3631,17 @@ boolean cacheFileMetadata(String dbName, String tableName, String partName,
   List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest request) throws MetaException,
       NoSuchObjectException, TException;
 
+  /**
+   * Get all constraints of given table
+   * @param request Request info
+   * @return all constraints of this table
+   * @throws MetaException
+   * @throws NoSuchObjectException
+   * @throws TException
+   */
+  SQLAllTableConstraints getAllTableConstraints(AllTableConstraintsRequest request)

Review comment:
       Method won't return NoSuchObjectException. Shall remove it here and from all implementations.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -5661,184 +5663,79 @@ public void dropConstraint(String dbName, String tableName, String constraintNam
     }
   }
 
-  public List<SQLDefaultConstraint> getDefaultConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
+  public SQLAllTableConstraints getTableConstraints(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
+      AllTableConstraintsRequest tableConstraintsRequest = new AllTableConstraintsRequest();
+      tableConstraintsRequest.setDbName(dbName);
+      tableConstraintsRequest.setTblName(tblName);
+      tableConstraintsRequest.setCatName(getDefaultCatalog(conf));
+      return getMSC().getAllTableConstraints(tableConstraintsRequest);
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
       throw new HiveException(e);
     }
   }
-
-  public List<SQLCheckConstraint> getCheckConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
-    try {
-      return getMSC().getCheckConstraints(new CheckConstraintsRequest(getDefaultCatalog(conf),
-          dbName, tblName));
-    } catch (NoSuchObjectException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new HiveException(e);
-    }
+  public TableConstraintsInfo getAllTableConstraints(String dbName, String tblName) throws HiveException {
+    return getTableConstraints(dbName, tblName, false, false);
   }
 
-  /**
-   * Get all primary key columns associated with the table.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Primary Key associated with the table.
-   * @throws HiveException
-   */
-  public PrimaryKeyInfo getPrimaryKeys(String dbName, String tblName) throws HiveException {
-    return getPrimaryKeys(dbName, tblName, false);
+  public TableConstraintsInfo getReliableAndEnableTableConstraints(String dbName, String tblName) throws HiveException {

Review comment:
       getReliableAndEnableTableConstraints() method name is confusing. It is used in only one place and hence, just expose getTableConstraints() itself.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##########
@@ -1155,22 +1150,19 @@ void dumpConstraintMetadata(String dbName, String tblName, Path dbRoot, Hive hiv
       Path constraintsRoot = new Path(dbRoot, ReplUtils.CONSTRAINTS_ROOT_DIR_NAME);
       Path commonConstraintsFile = new Path(constraintsRoot, ConstraintFileType.COMMON.getPrefix() + tblName);
       Path fkConstraintsFile = new Path(constraintsRoot, ConstraintFileType.FOREIGNKEY.getPrefix() + tblName);
-      List<SQLPrimaryKey> pks = hiveDb.getPrimaryKeyList(dbName, tblName);
-      List<SQLForeignKey> fks = hiveDb.getForeignKeyList(dbName, tblName);
-      List<SQLUniqueConstraint> uks = hiveDb.getUniqueConstraintList(dbName, tblName);
-      List<SQLNotNullConstraint> nns = hiveDb.getNotNullConstraintList(dbName, tblName);
-      if ((pks != null && !pks.isEmpty()) || (uks != null && !uks.isEmpty())
-          || (nns != null && !nns.isEmpty())) {
+      SQLAllTableConstraints tableConstraints = hiveDb.getTableConstraints(dbName,tblName);

Review comment:
       nit: Add space after comma between arguments.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
##########
@@ -1176,145 +1166,101 @@ public void setStatsStateLikeNewTable() {
    *  Note that set apis are used by DESCRIBE only, although get apis return RELY or ENABLE
    *  constraints DESCRIBE could set all type of constraints
    * */
-
-  /* This only return PK which are created with RELY */
-  public PrimaryKeyInfo getPrimaryKeyInfo() {
-    if(!this.isPKFetched) {
+  public TableConstraintsInfo getTableConstraintsInfo() {
+    if (!this.isTableConstraintsFetched) {
       try {
-        pki = Hive.get().getReliablePrimaryKeys(this.getDbName(), this.getTableName());
-        this.isPKFetched = true;
+        tableConstraintsInfo = Hive.get().getReliableAndEnableTableConstraints(this.getDbName(), this.getTableName());
+        this.isTableConstraintsFetched = true;
       } catch (HiveException e) {
-        LOG.warn("Cannot retrieve PK info for table : " + this.getTableName()
-            + " ignoring exception: " + e);
+        LOG.warn(
+            "Cannot retrieve table constraints info for table : " + this.getTableName() + " ignoring exception: " + e);

Review comment:
       Log dbName.tblName.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -5661,184 +5663,79 @@ public void dropConstraint(String dbName, String tableName, String constraintNam
     }
   }
 
-  public List<SQLDefaultConstraint> getDefaultConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
+  public SQLAllTableConstraints getTableConstraints(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
+      AllTableConstraintsRequest tableConstraintsRequest = new AllTableConstraintsRequest();
+      tableConstraintsRequest.setDbName(dbName);
+      tableConstraintsRequest.setTblName(tblName);
+      tableConstraintsRequest.setCatName(getDefaultCatalog(conf));
+      return getMSC().getAllTableConstraints(tableConstraintsRequest);
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
       throw new HiveException(e);
     }
   }
-
-  public List<SQLCheckConstraint> getCheckConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
-    try {
-      return getMSC().getCheckConstraints(new CheckConstraintsRequest(getDefaultCatalog(conf),
-          dbName, tblName));
-    } catch (NoSuchObjectException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new HiveException(e);
-    }
+  public TableConstraintsInfo getAllTableConstraints(String dbName, String tblName) throws HiveException {
+    return getTableConstraints(dbName, tblName, false, false);
   }
 
-  /**
-   * Get all primary key columns associated with the table.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Primary Key associated with the table.
-   * @throws HiveException
-   */
-  public PrimaryKeyInfo getPrimaryKeys(String dbName, String tblName) throws HiveException {
-    return getPrimaryKeys(dbName, tblName, false);
+  public TableConstraintsInfo getReliableAndEnableTableConstraints(String dbName, String tblName) throws HiveException {
+    return getTableConstraints(dbName, tblName, true, true);
   }
 
-  /**
-   * Get primary key columns associated with the table that are available for optimization.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Primary Key associated with the table.
-   * @throws HiveException
-   */
-  public PrimaryKeyInfo getReliablePrimaryKeys(String dbName, String tblName) throws HiveException {
-    return getPrimaryKeys(dbName, tblName, true);
-  }
-
-  private PrimaryKeyInfo getPrimaryKeys(String dbName, String tblName, boolean onlyReliable)
+  private TableConstraintsInfo getTableConstraints(String dbName, String tblName, boolean reliable, boolean enable)
       throws HiveException {
     PerfLogger perfLogger = SessionState.getPerfLogger();
-    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PK);
-    try {
-      List<SQLPrimaryKey> primaryKeys = getMSC().getPrimaryKeys(new PrimaryKeysRequest(dbName, tblName));
-      if (onlyReliable && primaryKeys != null && !primaryKeys.isEmpty()) {
-        primaryKeys = primaryKeys.stream()
-          .filter(pk -> pk.isRely_cstr())
-          .collect(Collectors.toList());
-      }
-
-      return new PrimaryKeyInfo(primaryKeys, tblName, dbName);
-    } catch (Exception e) {
-      throw new HiveException(e);
-    } finally {
-      perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PK, "HS2-cache");
-    }
-  }
-
-  /**
-   * Get all foreign keys associated with the table.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Foreign keys associated with the table.
-   * @throws HiveException
-   */
-  public ForeignKeyInfo getForeignKeys(String dbName, String tblName) throws HiveException {
-    return getForeignKeys(dbName, tblName, false);
-  }
-
-  /**
-   * Get foreign keys associated with the table that are available for optimization.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Foreign keys associated with the table.
-   * @throws HiveException
-   */
-  public ForeignKeyInfo getReliableForeignKeys(String dbName, String tblName) throws HiveException {
-    return getForeignKeys(dbName, tblName, true);
-  }
-
-  private ForeignKeyInfo getForeignKeys(String dbName, String tblName, boolean onlyReliable)
-      throws HiveException {
-    PerfLogger perfLogger = SessionState.getPerfLogger();
-    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_FK);
-    try {
-      List<SQLForeignKey> foreignKeys = getMSC().getForeignKeys(new ForeignKeysRequest(null, null, dbName, tblName));
-      if (onlyReliable && foreignKeys != null && !foreignKeys.isEmpty()) {
-        foreignKeys = foreignKeys.stream()
-          .filter(fk -> fk.isRely_cstr())
-          .collect(Collectors.toList());
+    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_TABLE_CONSTRAINTS);
+    try {
+      SQLAllTableConstraints tableConstraints =
+          getMSC().getAllTableConstraints(new AllTableConstraintsRequest(dbName, tblName));
+      if (reliable && tableConstraints != null) {
+        if (tableConstraints.getPrimaryKeys() != null && !tableConstraints.getPrimaryKeys().isEmpty()) {
+          tableConstraints.setPrimaryKeys(
+              tableConstraints.getPrimaryKeys().stream().filter(primaryKey -> primaryKey.isRely_cstr())
+                  .collect(Collectors.toList()));
+        }
+        if (tableConstraints.getForeignKeys() != null && !tableConstraints.getForeignKeys().isEmpty()) {
+          tableConstraints.setForeignKeys(
+              tableConstraints.getForeignKeys().stream().filter(foreignKey -> foreignKey.isRely_cstr())
+                  .collect(Collectors.toList()));
+        }
+        if (tableConstraints.getUniqueConstraints() != null && !tableConstraints.getUniqueConstraints().isEmpty()) {
+          tableConstraints.setUniqueConstraints(tableConstraints.getUniqueConstraints().stream()
+              .filter(uniqueConstraint -> uniqueConstraint.isRely_cstr()).collect(Collectors.toList()));
+        }
+        if (tableConstraints.getNotNullConstraints() != null && !tableConstraints.getNotNullConstraints().isEmpty()) {
+          tableConstraints.setNotNullConstraints(tableConstraints.getNotNullConstraints().stream()
+              .filter(notNullConstraint -> notNullConstraint.isRely_cstr()).collect(Collectors.toList()));
+        }
       }
 
-      return new ForeignKeyInfo(foreignKeys, tblName, dbName);
-    } catch (Exception e) {
-      throw new HiveException(e);
-    } finally {
-      perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_FK, "HS2-cache");
-    }
-  }
-
-  /**
-   * Get all unique constraints associated with the table.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Unique constraints associated with the table.
-   * @throws HiveException
-   */
-  public UniqueConstraint getUniqueConstraints(String dbName, String tblName) throws HiveException {
-    return getUniqueConstraints(dbName, tblName, false);
-  }
-
-  /**
-   * Get unique constraints associated with the table that are available for optimization.
-   *
-   * @param dbName Database Name
-   * @param tblName Table Name
-   * @return Unique constraints associated with the table.
-   * @throws HiveException
-   */
-  public UniqueConstraint getReliableUniqueConstraints(String dbName, String tblName) throws HiveException {
-    return getUniqueConstraints(dbName, tblName, true);
-  }
-
-  private UniqueConstraint getUniqueConstraints(String dbName, String tblName, boolean onlyReliable)
-      throws HiveException {
-    PerfLogger perfLogger = SessionState.getPerfLogger();
-    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_UNIQ_CONSTRAINT);
-    try {
-      List<SQLUniqueConstraint> uniqueConstraints = getMSC().getUniqueConstraints(
-              new UniqueConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
-      if (onlyReliable && uniqueConstraints != null && !uniqueConstraints.isEmpty()) {
-        uniqueConstraints = uniqueConstraints.stream()
-          .filter(uk -> uk.isRely_cstr())
-          .collect(Collectors.toList());
+      if (enable && tableConstraints != null) {

Review comment:
       Other constraints also needs to be filtered based on "enable" flag, right?

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
##########
@@ -9200,6 +9200,31 @@ public CheckConstraintsResponse get_check_constraints(CheckConstraintsRequest re
       return new CheckConstraintsResponse(ret);
     }
 
+    /**
+     * Api to fetch all table constraints at once
+     * @param request it consist of catalog name, database name and table name to identify the table in metastore
+     * @return all cnstraint attached to given table
+     * @throws TException
+     */
+    @Override
+    public AllTableConstraintsResponse get_all_table_constraints(AllTableConstraintsRequest request) throws TException {
+      String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf);
+      String dbName = request.getDbName();
+      String tblName = request.getTblName();
+      startTableFunction("get_all_table_constraints", catName, dbName, tblName);
+      SQLAllTableConstraints ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getAllTableConstraints(catName,dbName,tblName);

Review comment:
       nit: Add space after comma.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
##########
@@ -1176,145 +1166,101 @@ public void setStatsStateLikeNewTable() {
    *  Note that set apis are used by DESCRIBE only, although get apis return RELY or ENABLE
    *  constraints DESCRIBE could set all type of constraints
    * */
-
-  /* This only return PK which are created with RELY */
-  public PrimaryKeyInfo getPrimaryKeyInfo() {
-    if(!this.isPKFetched) {
+  public TableConstraintsInfo getTableConstraintsInfo() {
+    if (!this.isTableConstraintsFetched) {
       try {
-        pki = Hive.get().getReliablePrimaryKeys(this.getDbName(), this.getTableName());
-        this.isPKFetched = true;
+        tableConstraintsInfo = Hive.get().getReliableAndEnableTableConstraints(this.getDbName(), this.getTableName());
+        this.isTableConstraintsFetched = true;
       } catch (HiveException e) {
-        LOG.warn("Cannot retrieve PK info for table : " + this.getTableName()
-            + " ignoring exception: " + e);
+        LOG.warn(
+            "Cannot retrieve table constraints info for table : " + this.getTableName() + " ignoring exception: " + e);
       }
     }
-    return pki;
+    return tableConstraintsInfo;
   }
 
-  public void setPrimaryKeyInfo(PrimaryKeyInfo pki) {
-    this.pki = pki;
-    this.isPKFetched = true;
+  /**
+   * TableConstraintsInfo setter
+   * @param tableConstraintsInfo
+   */
+  public void setTableConstraintsInfo(TableConstraintsInfo tableConstraintsInfo) {
+    this.tableConstraintsInfo = tableConstraintsInfo;
+    this.isTableConstraintsFetched = true;
   }
 
-  /* This only return FK constraints which are created with RELY */
-  public ForeignKeyInfo getForeignKeyInfo() {
-    if(!isFKFetched) {
-      try {
-        fki = Hive.get().getReliableForeignKeys(this.getDbName(), this.getTableName());
-        this.isFKFetched = true;
-      } catch (HiveException e) {
-        LOG.warn(
-            "Cannot retrieve FK info for table : " + this.getTableName()
-                + " ignoring exception: " + e);
-      }
+  /**
+   * This only return PK which are created with RELY
+   * @return primary key constraint list
+   */
+  public PrimaryKeyInfo getPrimaryKeyInfo() {
+    if (!this.isTableConstraintsFetched) {
+      getTableConstraintsInfo();
     }
-    return fki;
+    return tableConstraintsInfo.getPrimaryKeyInfo();
   }
 
-  public void setForeignKeyInfo(ForeignKeyInfo fki) {
-    this.fki = fki;
-    this.isFKFetched = true;
+  /**
+   * This only return FK constraints which are created with RELY
+   * @return foreign key constraint list
+   */
+  public ForeignKeyInfo getForeignKeyInfo() {
+    if (!isTableConstraintsFetched) {
+      getTableConstraintsInfo();
+    }
+    return tableConstraintsInfo.getForeignKeyInfo();
   }
 
-  /* This only return UNIQUE constraint defined with RELY */
+  /**
+   * This only return UNIQUE constraint defined with RELY
+   * @return unique constraint list
+   */
   public UniqueConstraint getUniqueKeyInfo() {
-    if(!isUniqueFetched) {
-      try {
-        uki = Hive.get().getReliableUniqueConstraints(this.getDbName(), this.getTableName());
-        this.isUniqueFetched = true;
-      } catch (HiveException e) {
-        LOG.warn(
-            "Cannot retrieve Unique Key info for table : " + this.getTableName()
-                + " ignoring exception: " + e);
-      }
+    if (!isTableConstraintsFetched) {
+      getTableConstraintsInfo();
     }
-    return uki;
-  }
-
-  public void setUniqueKeyInfo(UniqueConstraint uki) {
-    this.uki = uki;
-    this.isUniqueFetched = true;
+    return tableConstraintsInfo.getUniqueConstraint();
   }
 
-  /* This only return NOT NULL constraint defined with RELY */
+  /**
+   * This only return NOT NULL constraint defined with RELY
+   * @return not null constraint list
+   */
   public NotNullConstraint getNotNullConstraint() {
-    if(!isNotNullFetched) {
-      try {
-        nnc = Hive.get().getReliableNotNullConstraints(this.getDbName(), this.getTableName());
-        this.isNotNullFetched = true;
-      } catch (HiveException e) {
-        LOG.warn("Cannot retrieve Not Null constraint info for table : "
-            + this.getTableName() + " ignoring exception: " + e);
-      }
+    if (!isTableConstraintsFetched) {
+      getTableConstraintsInfo();
     }
-    return nnc;
-  }
-
-  public void setNotNullConstraint(NotNullConstraint nnc) {
-    this.nnc = nnc;
-    this.isNotNullFetched = true;
+    return tableConstraintsInfo.getNotNullConstraint();
   }
 
-  /* This only return DEFAULT constraint defined with ENABLE */
+  /**
+   * This only return DEFAULT constraint defined with ENABLE
+   * @return default constraint list
+   */
   public DefaultConstraint getDefaultConstraint() {
-    if(!isDefaultFetched) {
-      try {
-        dc = Hive.get().getEnabledDefaultConstraints(this.getDbName(), this.getTableName());
-        this.isDefaultFetched = true;
-      } catch (HiveException e) {
-        LOG.warn("Cannot retrieve Default constraint info for table : "
-            + this.getTableName() + " ignoring exception: " + e);
-      }
+    if (!isTableConstraintsFetched) {
+      getTableConstraintsInfo();
     }
-    return dc;
+    return tableConstraintsInfo.getDefaultConstraint();
   }
 
-  public void setDefaultConstraint(DefaultConstraint dc) {
-    this.dc = dc;
-    this.isDefaultFetched = true;
-  }
-
-  /* This only return CHECK constraint defined with ENABLE */
+  /**
+   * This only return CHECK constraint defined with ENABLE
+   * @return check constraint list
+   */
   public CheckConstraint getCheckConstraint() {
-    if(!isCheckFetched) {
-      try{
-        cc = Hive.get().getEnabledCheckConstraints(this.getDbName(), this.getTableName());
-        this.isCheckFetched = true;
-      } catch (HiveException e) {
-        LOG.warn("Cannot retrieve Check constraint info for table : "
-            + this.getTableName() + " ignoring exception: " + e);
-      }
+    if (!isTableConstraintsFetched) {
+      getTableConstraintsInfo();
     }
-    return cc;
-  }
-
-  public void setCheckConstraint(CheckConstraint cc) {
-    this.cc = cc;
-    this.isCheckFetched = true;
+    return tableConstraintsInfo.getCheckConstraint();
   }
 
   /** This shouldn't use get apis because those api call metastore
    * to fetch constraints.
    * getMetaData only need to make a copy of existing constraints, even if those are not fetched
    */
   public void copyConstraints(final Table tbl) {
-    this.pki = tbl.pki;
-    this.isPKFetched = tbl.isPKFetched;
-
-    this.fki = tbl.fki;
-    this.isFKFetched = tbl.isFKFetched;
-
-    this.uki = tbl.uki;
-    this.isUniqueFetched = tbl.isUniqueFetched;
-
-    this.nnc = tbl.nnc;
-    this.isNotNullFetched = tbl.isNotNullFetched;
-
-    this.dc = tbl.dc;
-    this.isDefaultFetched = tbl.isDefaultFetched;
-
-    this.cc = tbl.cc;
-    this.isCheckFetched = tbl.isCheckFetched;
+    this.tableConstraintsInfo = tbl.tableConstraintsInfo;

Review comment:
       Just add a note that we don't make copy of constraints we just refer to same input constraints object.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/TableConstraintsInfo.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.hadoop.hive.ql.metadata;
+
+public class TableConstraintsInfo {
+  PrimaryKeyInfo primaryKeyInfo;
+  ForeignKeyInfo foreignKeyInfo;
+  UniqueConstraint uniqueConstraint;
+  DefaultConstraint defaultConstraint;
+  CheckConstraint checkConstraint;
+  NotNullConstraint notNullConstraint;
+
+  public TableConstraintsInfo() {
+  }
+
+  public TableConstraintsInfo(PrimaryKeyInfo primaryKeyInfo, ForeignKeyInfo foreignKeyInfo,
+      UniqueConstraint uniqueConstraint, DefaultConstraint defaultConstraint, CheckConstraint checkConstraint,
+      NotNullConstraint notNullConstraint) {
+    this.primaryKeyInfo = primaryKeyInfo;
+    this.foreignKeyInfo = foreignKeyInfo;
+    this.uniqueConstraint = uniqueConstraint;
+    this.defaultConstraint = defaultConstraint;
+    this.checkConstraint = checkConstraint;
+    this.notNullConstraint = notNullConstraint;
+  }
+
+  public PrimaryKeyInfo getPrimaryKeyInfo() {
+    return primaryKeyInfo;
+  }
+
+  public void setPrimaryKeyInfo(PrimaryKeyInfo primaryKeyInfo) {
+    this.primaryKeyInfo = primaryKeyInfo;
+  }
+
+  public ForeignKeyInfo getForeignKeyInfo() {
+    return foreignKeyInfo;
+  }
+
+  public void setForeignKeyInfo(ForeignKeyInfo foreignKeyInfo) {
+    this.foreignKeyInfo = foreignKeyInfo;
+  }
+
+  public UniqueConstraint getUniqueConstraint() {
+    return uniqueConstraint;
+  }
+
+  public void setUniqueConstraint(UniqueConstraint uniqueConstraint) {
+    this.uniqueConstraint = uniqueConstraint;
+  }
+
+  public DefaultConstraint getDefaultConstraint() {
+    return defaultConstraint;
+  }
+
+  public void setDefaultConstraint(DefaultConstraint defaultConstraint) {
+    this.defaultConstraint = defaultConstraint;
+  }
+
+  public CheckConstraint getCheckConstraint() {
+    return checkConstraint;
+  }
+
+  public void setCheckConstraint(CheckConstraint checkConstraint) {
+    this.checkConstraint = checkConstraint;
+  }
+
+  public NotNullConstraint getNotNullConstraint() {
+    return notNullConstraint;
+  }
+
+  public void setNotNullConstraint(NotNullConstraint notNullConstraint) {
+    this.notNullConstraint = notNullConstraint;
+  }
+
+  public static boolean isTableConstraintsInfoNotEmpty(TableConstraintsInfo info) {

Review comment:
       Why is it static method? In both the callers, we pass tbl.getTableConstraintsInfo(). This can be a public method without arguments inside "Table" class.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
##########
@@ -116,22 +116,12 @@
   private transient Boolean outdatedForRewritingMaterializedView;
 
   /** Constraint related objects */
-  private transient PrimaryKeyInfo pki;
-  private transient ForeignKeyInfo fki;
-  private transient UniqueConstraint uki;
-  private transient NotNullConstraint nnc;
-  private transient DefaultConstraint dc;
-  private transient CheckConstraint cc;
+  private transient TableConstraintsInfo tableConstraintsInfo;
 
   /** Constraint related flags
    *  This is to track if constraints are retrieved from metastore or not
    */
-  private transient boolean isPKFetched=false;
-  private transient boolean isFKFetched=false;
-  private transient boolean isUniqueFetched=false;
-  private transient boolean isNotNullFetched=false;
-  private transient boolean isDefaultFetched=false;
-  private transient boolean isCheckFetched=false;
+  private transient boolean isTableConstraintsFetched=false;

Review comment:
       Do we need an extra flag? Can't we just check null for tableConstraintsInfo?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/TableConstraintsInfo.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.hadoop.hive.ql.metadata;
+
+public class TableConstraintsInfo {
+  PrimaryKeyInfo primaryKeyInfo;

Review comment:
       Shall make all members private as we have get and set methods.




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