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 2021/02/11 08:25:13 UTC

[GitHub] [hive] sankarh commented on a change in pull request #1610: HIVE-24259: [CachedStore] Optimise get constraints call by removing redundant table check

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
##########
@@ -1325,6 +1204,7 @@ private void validateTableType(Table tbl) {
     }
     validateTableType(tbl);
     sharedCache.addTableToCache(catName, dbName, tblName, tbl);
+    sharedCache.addTableConstraintsToCache(catName,dbName,tblName,new SQLAllTableConstraints());

Review comment:
       nit: Add space after comma.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -506,10 +512,18 @@ boolean cacheCheckConstraints(List<SQLCheckConstraint> checkConstraints, boolean
       return cacheConstraints(checkConstraints, fromPrewarm, MemberName.CHECK_CONSTRAINT_CACHE);
     }
 
+    boolean cacheConstraints(SQLAllTableConstraints constraints, boolean fromPrewarm) {
+      this.isConstraintsValid =
+          cacheConstraints(constraints.getPrimaryKeys(), fromPrewarm, MemberName.PRIMARY_KEY_CACHE) && cacheConstraints(
+              constraints.getForeignKeys(), fromPrewarm, MemberName.FOREIGN_KEY_CACHE) && cacheConstraints(constraints.getDefaultConstraints(), fromPrewarm, MemberName.DEFAULT_CONSTRAINT_CACHE)
+              && cacheConstraints(constraints.getUniqueConstraints(), fromPrewarm, MemberName.UNIQUE_CONSTRAINT_CACHE)
+              && cacheConstraints(constraints.getNotNullConstraints(), fromPrewarm, MemberName.NOTNULL_CONSTRAINT_CACHE)
+              && cacheConstraints(constraints.getCheckConstraints(), fromPrewarm, MemberName.CHECK_CONSTRAINT_CACHE);
+      return this.isConstraintsValid;
+    }
+
     // Common method to cache constraints
-    private boolean cacheConstraints(List constraintsList,
-                             boolean fromPrewarm,
-                             MemberName mn) {
+    private boolean cacheConstraints(List constraintsList, boolean fromPrewarm, MemberName mn) {

Review comment:
       This method never returns false. It means, isConstraintsValid flag is always true. Is it by design? If yes, when it will be false?

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -2187,7 +2139,7 @@ public void addPrimaryKeysToCache(String catName, String dbName, String tblName,
       cacheLock.readLock().lock();
       String tblKey = CacheUtils.buildTableKey(catName, dbName, tblName);
       TableWrapper tblWrapper = tableCache.getIfPresent(tblKey);
-      if (tblWrapper != null) {
+      if (tblWrapper != null && tblWrapper.isConstraintsValid()) {

Review comment:
       Keep tblWrapper != null within ().

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -1788,47 +1771,16 @@ public boolean populateTableInCache(Table table, TableCacheObjects cacheObjects)
     tblWrapper.setMemberCacheUpdated(MemberName.PARTITION_COL_STATS_CACHE, false);
     tblWrapper.setMemberCacheUpdated(MemberName.AGGR_COL_STATS_CACHE, false);
 
-    if (CollectionUtils.isNotEmpty(constraints.getPrimaryKeys())) {
-      if (!tblWrapper.cachePrimaryKeys(constraints.getPrimaryKeys(), true)) {
-        return false;
-      }
-    }
-    tblWrapper.setMemberCacheUpdated(MemberName.PRIMARY_KEY_CACHE, false);
-
-    if (CollectionUtils.isNotEmpty(constraints.getForeignKeys())) {
-      if (!tblWrapper.cacheForeignKeys(constraints.getForeignKeys(), true)) {
-        return false;
-      }
-    }
-    tblWrapper.setMemberCacheUpdated(MemberName.FOREIGN_KEY_CACHE, false);
-
-    if (CollectionUtils.isNotEmpty(constraints.getNotNullConstraints())) {
-      if (!tblWrapper.cacheNotNullConstraints(constraints.getNotNullConstraints(), true)) {
-        return false;
-      }
-    }
-    tblWrapper.setMemberCacheUpdated(MemberName.NOTNULL_CONSTRAINT_CACHE, false);
-
-    if (CollectionUtils.isNotEmpty(constraints.getUniqueConstraints())) {
-      if (!tblWrapper.cacheUniqueConstraints(constraints.getUniqueConstraints(), true)) {
-        return false;
-      }
-    }
-    tblWrapper.setMemberCacheUpdated(MemberName.UNIQUE_CONSTRAINT_CACHE, false);
-
-    if (CollectionUtils.isNotEmpty(constraints.getDefaultConstraints())) {
-      if (!tblWrapper.cacheDefaultConstraints(constraints.getDefaultConstraints(), true)) {
-        return false;
-      }
-    }
-    tblWrapper.setMemberCacheUpdated(MemberName.DEFAULT_CONSTRAINT_CACHE, false);
-
-    if (CollectionUtils.isNotEmpty(constraints.getCheckConstraints())) {
-      if (!tblWrapper.cacheCheckConstraints(constraints.getCheckConstraints(), true)) {
-        return false;
-      }
+    if(tblWrapper.cacheConstraints(constraints,true)) {

Review comment:
       nit: Space before  ( and after comma

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
##########
@@ -1558,6 +1558,7 @@ public void testPrimaryKeys() {
     Table tbl = createUnpartitionedTableObject(db);
 
     sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, "db", tbl.getTableName(), tbl);
+    sharedCache.addTableConstraintsToCache(DEFAULT_CATALOG_NAME,"db", tbl.getTableName(), new SQLAllTableConstraints());

Review comment:
       nit: Need space after comma

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -2397,7 +2397,7 @@ public SQLAllTableConstraints listCachedAllTableConstraints(String catName, Stri
 
   public List<SQLForeignKey> listCachedForeignKeys(String catName, String foreignDbName, String foreignTblName,
                                                    String parentDbName, String parentTblName) {
-    List<SQLForeignKey> keys = new ArrayList<>();
+    List<SQLForeignKey> keys = null;

Review comment:
       Still keys can be referenced with null at line 2365.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -715,154 +747,105 @@ public void removeConstraint(String name) {
       }
     }
 
-    public void refreshPrimaryKeys(List<SQLPrimaryKey> keys) {
-      Map<String, SQLPrimaryKey> newKeys = new ConcurrentHashMap<>();
+    public void refreshAllTableConstraints(SQLAllTableConstraints constraints) {
       try {
         tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLPrimaryKey key : keys) {
-          if (compareAndSetMemberCacheUpdated(MemberName.PRIMARY_KEY_CACHE, true, false)) {
-            LOG.debug("Skipping primary key cache update for table: " + getTable().getTableName()
-                    + "; the primary keys are already refreshed.");
-            return;
-          }
-          String pkName = StringUtils.normalizeIdentifier(key.getPk_name());
-          key.setPk_name(pkName);
-          newKeys.put(pkName, key);
-          size += getObjectSize(SQLPrimaryKey.class, key);
-        }
-        primaryKeyCache = newKeys;
-        updateMemberSize(MemberName.PRIMARY_KEY_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Primary keys refresh in cache was successful for {}.{}.{}",
-            this.getTable().getCatName(), this.getTable().getDbName(), this.getTable().getTableName());
+        this.isConstraintsValid =
+            refreshConstraint(constraints.getPrimaryKeys(), MemberName.PRIMARY_KEY_CACHE) && refreshConstraint(
+                constraints.getForeignKeys(), MemberName.FOREIGN_KEY_CACHE) && refreshConstraint(
+                constraints.getUniqueConstraints(), MemberName.UNIQUE_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getDefaultConstraints(), MemberName.DEFAULT_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getNotNullConstraints(), MemberName.NOTNULL_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getCheckConstraints(), MemberName.CHECK_CONSTRAINT_CACHE);
       } finally {
         tableLock.writeLock().unlock();
       }
     }
 
-    public void refreshForeignKeys(List<SQLForeignKey> keys) {
-      Map<String, SQLForeignKey> newKeys = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLForeignKey key : keys) {
-          if (compareAndSetMemberCacheUpdated(MemberName.FOREIGN_KEY_CACHE, true, false)) {
-            LOG.debug("Skipping foreign key cache update for table: " + getTable().getTableName()
-                    + "; the foreign keys are already refreshed.");
-            return;
+    private boolean refreshConstraint(List constraints, MemberName mn) {
+      int size = 0;
+      switch (mn) {
+      case PRIMARY_KEY_CACHE:
+        Map<String, SQLPrimaryKey> newPk = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLPrimaryKey key : (List<SQLPrimaryKey>) constraints) {
+            String pkName = StringUtils.normalizeIdentifier(key.getPk_name());
+            key.setPk_name(pkName);
+            newPk.put(pkName, key);
+            size += getObjectSize(SQLPrimaryKey.class, key);
           }
-          String fkName = StringUtils.normalizeIdentifier(key.getFk_name());
-          key.setFk_name(fkName);
-          newKeys.put(fkName, key);
-          size += getObjectSize(SQLForeignKey.class, key);
         }
-        foreignKeyCache = newKeys;
-        updateMemberSize(MemberName.FOREIGN_KEY_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Foreign keys refresh in cache was successful for {}.{}.{}",
-            this.getTable().getCatName(), this.getTable().getDbName(), this.getTable().getTableName());
-      } finally {
-        tableLock.writeLock().unlock();
-      }
-    }
-
-    public void refreshNotNullConstraints(List<SQLNotNullConstraint> constraints) {
-      Map<String, SQLNotNullConstraint> newConstraints = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLNotNullConstraint constraint : constraints) {
-          if (compareAndSetMemberCacheUpdated(MemberName.NOTNULL_CONSTRAINT_CACHE, true, false)) {
-            LOG.debug("Skipping not null constraints cache update for table: " + getTable().getTableName()
-                    + "; the not null constraints are already refreshed.");
-            return;
+        primaryKeyCache = newPk;
+        break;
+      case FOREIGN_KEY_CACHE:
+        Map<String, SQLForeignKey> newFk = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLForeignKey key : (List<SQLForeignKey>) constraints) {
+            String fkName = StringUtils.normalizeIdentifier(key.getFk_name());
+            key.setFk_name(fkName);
+            newFk.put(fkName, key);
+            size += getObjectSize(SQLForeignKey.class, key);
           }
-          String nnName = StringUtils.normalizeIdentifier(constraint.getNn_name());
-          constraint.setNn_name(nnName);
-          newConstraints.put(nnName, constraint);
-          size += getObjectSize(SQLNotNullConstraint.class, constraint);
         }
-        notNullConstraintCache = newConstraints;
-        updateMemberSize(MemberName.NOTNULL_CONSTRAINT_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Not null constraints refresh in cache was successful for {}.{}.{}",
-            this.getTable().getCatName(), this.getTable().getDbName(), this.getTable().getTableName());
-      } finally {
-        tableLock.writeLock().unlock();
-      }
-    }
-
-    public void refreshUniqueConstraints(List<SQLUniqueConstraint> constraints) {
-      Map<String, SQLUniqueConstraint> newConstraints = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLUniqueConstraint constraint : constraints) {
-          if (compareAndSetMemberCacheUpdated(MemberName.UNIQUE_CONSTRAINT_CACHE, true, false)) {
-            LOG.debug("Skipping unique constraints cache update for table: " + getTable().getTableName()
-                    + "; the unique constraints are already refreshed.");
-            return;
+        foreignKeyCache = newFk;
+        break;
+      case UNIQUE_CONSTRAINT_CACHE:
+        Map<String, SQLUniqueConstraint> newUc = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLUniqueConstraint constraint : (List<SQLUniqueConstraint>) constraints) {
+            String ucName = StringUtils.normalizeIdentifier(constraint.getUk_name());
+            constraint.setUk_name(ucName);
+            newUc.put(ucName, constraint);
+            size += getObjectSize(SQLUniqueConstraint.class, constraint);
           }
-          String ucName = StringUtils.normalizeIdentifier(constraint.getUk_name());
-          constraint.setUk_name(ucName);
-          newConstraints.put(ucName, constraint);
-          size += getObjectSize(SQLUniqueConstraint.class, constraint);
         }
-        uniqueConstraintCache = newConstraints;
-        updateMemberSize(MemberName.UNIQUE_CONSTRAINT_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Unique constraints refresh in cache was successful for {}.{}.{}",
-            this.getTable().getCatName(), this.getTable().getDbName(), this.getTable().getTableName());
-      } finally {
-        tableLock.writeLock().unlock();
-      }
-    }
-
-    public void refreshDefaultConstraints(List<SQLDefaultConstraint> constraints) {
-      Map<String, SQLDefaultConstraint> newConstraints = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLDefaultConstraint constraint : constraints) {
-          if (compareAndSetMemberCacheUpdated(MemberName.DEFAULT_CONSTRAINT_CACHE, true, false)) {
-            LOG.debug("Skipping default constraint cache update for table: " + getTable().getTableName()
-                + "; the default constraint are already refreshed.");
-            return;
+        uniqueConstraintCache = newUc;
+        break;
+      case NOTNULL_CONSTRAINT_CACHE:
+        Map<String, SQLNotNullConstraint> newNn = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLNotNullConstraint constraint : (List<SQLNotNullConstraint>) constraints) {
+            String nnName = StringUtils.normalizeIdentifier(constraint.getNn_name());
+            constraint.setNn_name(nnName);
+            newNn.put(nnName, constraint);
+            size += getObjectSize(SQLNotNullConstraint.class, constraint);
           }
-          String dcName = StringUtils.normalizeIdentifier(constraint.getDc_name());
-          constraint.setDc_name(dcName);
-          newConstraints.put(dcName, constraint);
-          size += getObjectSize(SQLDefaultConstraint.class, constraint);
         }
-        defaultConstraintCache = newConstraints;
-        updateMemberSize(MemberName.DEFAULT_CONSTRAINT_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Default constraints refresh in cache was successful for {}.{}.{}", this.getTable().getCatName(),
-            this.getTable().getDbName(), this.getTable().getTableName());
-      } finally {
-        tableLock.writeLock().unlock();
-      }
-    }
-
-    public void refreshCheckConstraints(List<SQLCheckConstraint> constraints) {
-      Map<String, SQLCheckConstraint> newConstraints = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLCheckConstraint constraint : constraints) {
-          if (compareAndSetMemberCacheUpdated(MemberName.CHECK_CONSTRAINT_CACHE, true, false)) {
-            LOG.debug("Skipping check constraint cache update for table: " + getTable().getTableName()
-                + "; the check constraint are already refreshed.");
-            return;
+        notNullConstraintCache = newNn;
+        break;
+      case DEFAULT_CONSTRAINT_CACHE:
+        Map<String, SQLDefaultConstraint> newDc = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLDefaultConstraint constraint : (List<SQLDefaultConstraint>) constraints) {
+            String dcName = StringUtils.normalizeIdentifier(constraint.getDc_name());
+            constraint.setDc_name(dcName);
+            newDc.put(dcName, constraint);
+            size += getObjectSize(SQLDefaultConstraint.class, constraint);
           }
-          String ccName = StringUtils.normalizeIdentifier(constraint.getDc_name());
-          constraint.setDc_name(ccName);
-          newConstraints.put(ccName, constraint);
-          size += getObjectSize(SQLCheckConstraint.class, constraint);
         }
-        checkConstraintCache = newConstraints;
-        updateMemberSize(MemberName.CHECK_CONSTRAINT_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("check constraints refresh in cache was successful for {}.{}.{}", this.getTable().getCatName(),
-            this.getTable().getDbName(), this.getTable().getTableName());
-      } finally {
-        tableLock.writeLock().unlock();
-      }
+        defaultConstraintCache = newDc;
+        break;
+      case CHECK_CONSTRAINT_CACHE:
+        Map<String, SQLCheckConstraint> newCc = new ConcurrentHashMap<>();
+        if (!CollectionUtils.isEmpty(constraints)) {
+          for (SQLCheckConstraint constraint : (List<SQLCheckConstraint>) constraints) {
+            String ccName = StringUtils.normalizeIdentifier(constraint.getDc_name());
+            constraint.setDc_name(ccName);
+            newCc.put(ccName, constraint);
+            size += getObjectSize(SQLCheckConstraint.class, constraint);
+          }
+        }
+        checkConstraintCache = newCc;
+        break;
+      default:
+        LOG.error("Should not reach here");
+        break;
+      }
+      updateMemberSize(mn, size, SizeMode.Snapshot);
+      setMemberCacheUpdated(mn, false);

Review comment:
       Why do we set this flag to false?

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -2187,7 +2139,7 @@ public void addPrimaryKeysToCache(String catName, String dbName, String tblName,
       cacheLock.readLock().lock();
       String tblKey = CacheUtils.buildTableKey(catName, dbName, tblName);
       TableWrapper tblWrapper = tableCache.getIfPresent(tblKey);
-      if (tblWrapper != null) {
+      if (tblWrapper != null && tblWrapper.isConstraintsValid()) {

Review comment:
       Why do we update cache for each constraint individually? We should always cache a single snapshot of constraints of table. If not, isConstraintsValid flag wouldn't make sense.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
##########
@@ -715,154 +747,105 @@ public void removeConstraint(String name) {
       }
     }
 
-    public void refreshPrimaryKeys(List<SQLPrimaryKey> keys) {
-      Map<String, SQLPrimaryKey> newKeys = new ConcurrentHashMap<>();
+    public void refreshAllTableConstraints(SQLAllTableConstraints constraints) {
       try {
         tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLPrimaryKey key : keys) {
-          if (compareAndSetMemberCacheUpdated(MemberName.PRIMARY_KEY_CACHE, true, false)) {
-            LOG.debug("Skipping primary key cache update for table: " + getTable().getTableName()
-                    + "; the primary keys are already refreshed.");
-            return;
-          }
-          String pkName = StringUtils.normalizeIdentifier(key.getPk_name());
-          key.setPk_name(pkName);
-          newKeys.put(pkName, key);
-          size += getObjectSize(SQLPrimaryKey.class, key);
-        }
-        primaryKeyCache = newKeys;
-        updateMemberSize(MemberName.PRIMARY_KEY_CACHE, size, SizeMode.Snapshot);
-        LOG.debug("Primary keys refresh in cache was successful for {}.{}.{}",
-            this.getTable().getCatName(), this.getTable().getDbName(), this.getTable().getTableName());
+        this.isConstraintsValid =
+            refreshConstraint(constraints.getPrimaryKeys(), MemberName.PRIMARY_KEY_CACHE) && refreshConstraint(
+                constraints.getForeignKeys(), MemberName.FOREIGN_KEY_CACHE) && refreshConstraint(
+                constraints.getUniqueConstraints(), MemberName.UNIQUE_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getDefaultConstraints(), MemberName.DEFAULT_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getNotNullConstraints(), MemberName.NOTNULL_CONSTRAINT_CACHE) && refreshConstraint(
+                constraints.getCheckConstraints(), MemberName.CHECK_CONSTRAINT_CACHE);
       } finally {
         tableLock.writeLock().unlock();
       }
     }
 
-    public void refreshForeignKeys(List<SQLForeignKey> keys) {
-      Map<String, SQLForeignKey> newKeys = new ConcurrentHashMap<>();
-      try {
-        tableLock.writeLock().lock();
-        int size = 0;
-        for (SQLForeignKey key : keys) {
-          if (compareAndSetMemberCacheUpdated(MemberName.FOREIGN_KEY_CACHE, true, false)) {
-            LOG.debug("Skipping foreign key cache update for table: " + getTable().getTableName()
-                    + "; the foreign keys are already refreshed.");
-            return;
+    private boolean refreshConstraint(List constraints, MemberName mn) {

Review comment:
       This also always returns true and in tune make isConstraintsValid=true always.




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