jcamachor commented on a change in pull request #1317:
URL: https://github.com/apache/hive/pull/1317#discussion_r461147163



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1631,6 +1633,9 @@ public Table 
apply(org.apache.hadoop.hive.metastore.api.Table table) {
       }
     } catch (Exception e) {
       throw new HiveException(e);
+    } finally {
+      long diff = System.nanoTime() - t1;

Review comment:
       We should use `PerfLogger` to record this information.
   ```
       PerfLogger perfLogger = SessionState.getPerfLogger();
   ```
   There are already a few calls to PerfLogger in the same file.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {

Review comment:
       Can we move this static block into an `initialize` method that is called 
from `HiveServer2.init` and `CliDriver.run`? You will also be able to pass the 
`conf` from those, where you should have access to these variables. 
Additionally, you could initialize `cachedEnabled` to false and change it 
whenever we call that method.

##########
File path: 
ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientListPartitionsTempTable.java
##########
@@ -22,12 +22,8 @@
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TestMetastoreExpr;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.*;

Review comment:
       Can you expand this import?

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()
+            .build();
+
+    cacheObjName = 
mscLocalCache.toString().substring(mscLocalCache.toString().indexOf("Cache@"));
+  }
+
+  /**
+   * This method is used to load the cache by calling relevant APIs, depending 
on the type of the request.
+   *
+   * @param cacheKey key of the cache, containing an identifier and a request 
object
+   * @return Result object / null
+   * @throws TException
+   */
+  private Object getResultObject(CacheKey cacheKey) throws TException {
+    Object result = null;
+
+    switch (cacheKey.IDENTIFIER) {
+      case PARTITIONS_BY_EXPR:
+        result = 
super.getPartitionsByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      case PARTITIONS_SPEC_BY_EXPR:
+        result = 
super.getPartitionsSpecByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      default:
+        break;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected PartitionsByExprResult 
getPartitionsByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load); // get either the 
result or an Exception
+
+        if (val instanceof PartitionsByExprResult) {
+          r = (PartitionsByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());

Review comment:
       Wrap with `if (LOG.isDebugEnabled())` (prevents concatenation that is 
discarded).

##########
File path: standalone-metastore/pom.xml
##########
@@ -115,6 +115,14 @@
 
   </properties>
 
+  <dependencies>
+    <dependency>
+      <groupId>com.github.ben-manes.caffeine</groupId>
+      <artifactId>caffeine</artifactId>
+      <version>2.8.4</version>

Review comment:
       Can we add version to `<!-- Dependency versions -->` list in pom file 
and use the reference?
   ```
       <caffeine.version>2.8.4</caffeine.version>
   ```

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);

Review comment:
       Do we need this line? Iiuc this is going to create the estimator for the 
class and objects referenced by the class, which does not seem needed.

##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -487,6 +487,10 @@ private static void populateLlapDaemonVarsSet(Set<String> 
llapDaemonVarsSetLocal
    * in the underlying Hadoop configuration.
    */
   public static enum ConfVars {
+    MSC_CACHE_ENABLED("hive.metastore.client.cache.enabled", true,
+            "This property enables a Caffeiene LoadingCache for Metastore 
client"),
+    MSC_CACHE_MAX_SIZE("hive.metastore.client.cache.maxSize", 1_000_000_000,

Review comment:
       Can we also set the default lower for the tests? In particular, you can 
do that by adding `hive.metastore.client.cache.maxSize` property with value of 
`10MB` to `data/conf/hive-site.xml`, `data/conf/llap/hive-site.xml` and 
`data/conf/rlist/hive-site.xml`.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()
+            .build();
+
+    cacheObjName = 
mscLocalCache.toString().substring(mscLocalCache.toString().indexOf("Cache@"));
+  }
+
+  /**
+   * This method is used to load the cache by calling relevant APIs, depending 
on the type of the request.
+   *
+   * @param cacheKey key of the cache, containing an identifier and a request 
object
+   * @return Result object / null
+   * @throws TException
+   */
+  private Object getResultObject(CacheKey cacheKey) throws TException {
+    Object result = null;
+
+    switch (cacheKey.IDENTIFIER) {
+      case PARTITIONS_BY_EXPR:
+        result = 
super.getPartitionsByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      case PARTITIONS_SPEC_BY_EXPR:
+        result = 
super.getPartitionsSpecByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      default:
+        break;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected PartitionsByExprResult 
getPartitionsByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load); // get either the 
result or an Exception
+
+        if (val instanceof PartitionsByExprResult) {
+          r = (PartitionsByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {
+          mscLocalCache.invalidate(cacheKey);
+          throw (Exception)val;
+        }
+      } catch (MetaException me) {
+        throw me;
+      } catch (Exception e) {
+        LOG.error("Exception in MSC local cache: {}", e.toString());
+        throw new TException(e.getMessage());
+      }
+    } else {
+         r = client.get_partitions_by_expr(req);
+    }
+
+    return r;
+  }
+
+  @Override
+  protected PartitionsSpecByExprResult 
getPartitionsSpecByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsSpecByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_SPEC_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load);
+
+        if (val instanceof PartitionsSpecByExprResult) {
+          r = (PartitionsSpecByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {

Review comment:
       Same as with comment above, let's avoid this pattern.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;

Review comment:
       This is not the best pattern. Can we simply remove the catch and throw 
the exception?

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
##########
@@ -50,13 +50,7 @@
 import org.apache.hadoop.hive.metastore.ColumnType;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.WMPoolSchedulingPolicy;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.api.*;

Review comment:
       Please expand.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;

Review comment:
       nit. In Hive, we always use expanded form for this statements, e.g.:
   ```
           if (this == o) {
             return true;
           }
   ```
   Can you change that?

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()

Review comment:
       Can we add a config variable to enable/disable recording of stats?

##########
File path: standalone-metastore/versionmap.txt
##########
@@ -0,0 +1 @@
+9257984804222f0fae06e5cca98e59c6

Review comment:
       Can we exclude this file?

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()
+            .build();
+
+    cacheObjName = 
mscLocalCache.toString().substring(mscLocalCache.toString().indexOf("Cache@"));
+  }
+
+  /**
+   * This method is used to load the cache by calling relevant APIs, depending 
on the type of the request.
+   *
+   * @param cacheKey key of the cache, containing an identifier and a request 
object
+   * @return Result object / null
+   * @throws TException
+   */
+  private Object getResultObject(CacheKey cacheKey) throws TException {
+    Object result = null;
+
+    switch (cacheKey.IDENTIFIER) {
+      case PARTITIONS_BY_EXPR:
+        result = 
super.getPartitionsByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      case PARTITIONS_SPEC_BY_EXPR:
+        result = 
super.getPartitionsSpecByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      default:
+        break;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected PartitionsByExprResult 
getPartitionsByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load); // get either the 
result or an Exception
+
+        if (val instanceof PartitionsByExprResult) {
+          r = (PartitionsByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {
+          mscLocalCache.invalidate(cacheKey);
+          throw (Exception)val;
+        }
+      } catch (MetaException me) {
+        throw me;
+      } catch (Exception e) {
+        LOG.error("Exception in MSC local cache: {}", e.toString());

Review comment:
       Change to :
   ```
            LOG.error("Exception in MSC local cache", e);
   ```
   That way we will log the complete stacktrace.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);

Review comment:
       `LOG.isDebugEnabled()`

##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -487,6 +487,10 @@ private static void populateLlapDaemonVarsSet(Set<String> 
llapDaemonVarsSetLocal
    * in the underlying Hadoop configuration.
    */
   public static enum ConfVars {
+    MSC_CACHE_ENABLED("hive.metastore.client.cache.enabled", true,
+            "This property enables a Caffeiene LoadingCache for Metastore 
client"),
+    MSC_CACHE_MAX_SIZE("hive.metastore.client.cache.maxSize", 1_000_000_000,

Review comment:
       Can you use a size validator?
   ```
   "hive.metastore.client.cache.maxSize", "1Gb", new SizeValidator(),
   ```

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()
+            .build();
+
+    cacheObjName = 
mscLocalCache.toString().substring(mscLocalCache.toString().indexOf("Cache@"));
+  }
+
+  /**
+   * This method is used to load the cache by calling relevant APIs, depending 
on the type of the request.
+   *
+   * @param cacheKey key of the cache, containing an identifier and a request 
object
+   * @return Result object / null
+   * @throws TException
+   */
+  private Object getResultObject(CacheKey cacheKey) throws TException {
+    Object result = null;
+
+    switch (cacheKey.IDENTIFIER) {
+      case PARTITIONS_BY_EXPR:
+        result = 
super.getPartitionsByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      case PARTITIONS_SPEC_BY_EXPR:
+        result = 
super.getPartitionsSpecByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      default:
+        break;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected PartitionsByExprResult 
getPartitionsByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load); // get either the 
result or an Exception
+
+        if (val instanceof PartitionsByExprResult) {
+          r = (PartitionsByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {

Review comment:
       Once we throw the Exception in the code above, we should be able to get 
rid of this _if_ branch.

##########
File path: 
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientWithLocalCache.java
##########
@@ -0,0 +1,246 @@
+package org.apache.hadoop.hive.metastore;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
+import org.apache.hadoop.hive.metastore.api.PartitionsSpecByExprResult;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator;
+import 
org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator;
+import org.apache.thrift.TException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Objects;
+
+public class HiveMetaStoreClientWithLocalCache extends HiveMetaStoreClient {
+
+  private static Cache<CacheKey, Object> mscLocalCache;
+  //TODO: initialize in the static block
+  private static final boolean IS_CACHE_ENABLED = true;
+  private static final int MAX_SIZE;
+  private static HashMap<Class<?>, ObjectEstimator> sizeEstimator = null;
+  private static String cacheObjName = null;
+
+  static {
+    LOG.debug("Initializing local cache in HiveMetaStoreClient...");
+    MAX_SIZE = MetastoreConf.getIntVar(MetastoreConf.newMetastoreConf(), 
MetastoreConf.ConfVars.MSC_CACHE_MAX_SIZE);
+//    IS_CACHE_ENABLED = 
MetastoreConf.getBoolVar(MetastoreConf.newMetastoreConf(),
+//                        MetastoreConf.ConfVars.MSC_CACHE_ENABLED);
+    initSizeEstimator();
+    initCache();
+    LOG.debug("Local cache initialized in HiveMetaStoreClient: " + 
mscLocalCache);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf) throws 
MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientWithLocalCache(Configuration conf, 
HiveMetaHookLoader hookLoader, Boolean allowEmbedded) throws MetaException {
+    super(conf, hookLoader, allowEmbedded);
+  }
+
+  private static void initSizeEstimator() {
+    sizeEstimator = 
IncrementalObjectSizeEstimator.createEstimators(HiveMetaStoreClientWithLocalCache.class);
+    IncrementalObjectSizeEstimator.createEstimators(CacheKey.class, 
sizeEstimator);
+    Arrays.stream(KeyType.values()).forEach(e -> {
+      IncrementalObjectSizeEstimator.createEstimators(e.keyClass, 
sizeEstimator);
+      IncrementalObjectSizeEstimator.createEstimators(e.valueClass, 
sizeEstimator);}
+    );
+  }
+
+  /**
+   * KeyType is used to differentiate the request types. More types can be 
added in future.
+   */
+  public enum KeyType {
+    PARTITIONS_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsByExprResult.class),
+    PARTITIONS_SPEC_BY_EXPR(PartitionsByExprRequest.class, 
PartitionsSpecByExprResult.class);
+
+    private final Class<?> keyClass;
+    private final Class<?> valueClass;
+
+    KeyType(Class<?> keyClass, Class<?> valueClass) {
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+    }
+  }
+
+  /**
+   * CacheKey objects are used as key for the cache.
+   */
+  public static class CacheKey{
+    KeyType IDENTIFIER;
+    Object obj;
+
+    public CacheKey(KeyType IDENTIFIER, Object obj) {
+      this.IDENTIFIER = IDENTIFIER;
+      this.obj = obj;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      CacheKey cacheKey = (CacheKey) o;
+      return IDENTIFIER == cacheKey.IDENTIFIER &&
+              Objects.equals(obj, cacheKey.obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(IDENTIFIER, obj);
+    }
+  }
+
+  private static int getWeight(CacheKey key, Object val) {
+    if (val instanceof Exception) return 0;
+    ObjectEstimator keySizeEstimator = sizeEstimator.get(key.getClass());
+    ObjectEstimator valSizeEstimator = 
sizeEstimator.get(key.IDENTIFIER.valueClass);
+    int keySize = keySizeEstimator.estimate(key, sizeEstimator);
+    int valSize = valSizeEstimator.estimate(val, sizeEstimator);
+    LOG.debug("Cache entry weight - key: {}, value: {}, total: {}", keySize, 
valSize, keySize+valSize);
+    return keySize + valSize;
+  }
+
+  private Object load(CacheKey key) {
+    Object val;
+    try {
+      val = getResultObject(key);
+    } catch (Exception e) {
+      LOG.debug("Exception in MSC local cache: {}", e.toString());
+      if (e instanceof MetaException) {
+        val = e;
+      } else {
+        val = new Exception(e.getMessage());
+      }
+    }
+    return val;
+  }
+
+/**
+ * Initializes the cache
+ */
+  private static synchronized void initCache() {
+    int initSize = 100;
+    mscLocalCache = Caffeine.newBuilder()
+            .initialCapacity(initSize)
+            .maximumWeight(MAX_SIZE)
+            .weigher(HiveMetaStoreClientWithLocalCache::getWeight)
+            .removalListener((key, val, cause) ->
+                    LOG.debug("Caffeine - ({}, {}) was removed ({})", key, 
val, cause))
+            .recordStats()
+            .build();
+
+    cacheObjName = 
mscLocalCache.toString().substring(mscLocalCache.toString().indexOf("Cache@"));
+  }
+
+  /**
+   * This method is used to load the cache by calling relevant APIs, depending 
on the type of the request.
+   *
+   * @param cacheKey key of the cache, containing an identifier and a request 
object
+   * @return Result object / null
+   * @throws TException
+   */
+  private Object getResultObject(CacheKey cacheKey) throws TException {
+    Object result = null;
+
+    switch (cacheKey.IDENTIFIER) {
+      case PARTITIONS_BY_EXPR:
+        result = 
super.getPartitionsByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      case PARTITIONS_SPEC_BY_EXPR:
+        result = 
super.getPartitionsSpecByExprResult((PartitionsByExprRequest)cacheKey.obj);
+        break;
+      default:
+        break;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected PartitionsByExprResult 
getPartitionsByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load); // get either the 
result or an Exception
+
+        if (val instanceof PartitionsByExprResult) {
+          r = (PartitionsByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {
+          mscLocalCache.invalidate(cacheKey);
+          throw (Exception)val;
+        }
+      } catch (MetaException me) {
+        throw me;
+      } catch (Exception e) {
+        LOG.error("Exception in MSC local cache: {}", e.toString());
+        throw new TException(e.getMessage());
+      }
+    } else {
+         r = client.get_partitions_by_expr(req);
+    }
+
+    return r;
+  }
+
+  @Override
+  protected PartitionsSpecByExprResult 
getPartitionsSpecByExprResult(PartitionsByExprRequest req) throws TException {
+    PartitionsSpecByExprResult r = null;
+
+    // table should be transactional to get responses from the cache
+    if (IS_CACHE_ENABLED && isRequestCachable(req)) {
+      CacheKey cacheKey = new CacheKey(KeyType.PARTITIONS_SPEC_BY_EXPR, req);
+      Object val;
+      try {
+        val = mscLocalCache.get(cacheKey, this::load);
+
+        if (val instanceof PartitionsSpecByExprResult) {
+          r = (PartitionsSpecByExprResult) val;
+          LOG.debug(cacheObjName + ": " + mscLocalCache.stats().toString());
+        } else if (val instanceof Exception) {
+          mscLocalCache.invalidate(cacheKey);
+          throw (Exception)val;
+        }
+      } catch (MetaException me) {
+        throw me;
+      } catch (Exception e) {
+        LOG.error("Exception in MSC local cache: {}", e.toString());
+        throw new TException(e.getMessage());
+      }
+    } else {
+        r = client.get_partitions_spec_by_expr(req);
+    }
+
+    return r;
+  }
+
+  /**
+   * This method determines if the request should be cached.
+   * @param request Request object
+   * @return boolean
+   */
+  private boolean isRequestCachable(Object request) {

Review comment:
       Can we pass the `KeyType` as parameter too and check based on switch 
statement on the enum?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1631,6 +1633,9 @@ public Table 
apply(org.apache.hadoop.hive.metastore.api.Table table) {
       }
     } catch (Exception e) {
       throw new HiveException(e);
+    } finally {
+      long diff = System.nanoTime() - t1;
+      LOG.debug(String.format(logString, "getTablesByType", diff, dbName, ""));

Review comment:
       Fwiw the perf logger already has a check on the logging level?

##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -487,6 +487,10 @@ private static void populateLlapDaemonVarsSet(Set<String> 
llapDaemonVarsSetLocal
    * in the underlying Hadoop configuration.
    */
   public static enum ConfVars {
+    MSC_CACHE_ENABLED("hive.metastore.client.cache.enabled", true,
+            "This property enables a Caffeiene LoadingCache for Metastore 
client"),
+    MSC_CACHE_MAX_SIZE("hive.metastore.client.cache.maxSize", 1_000_000_000,

Review comment:
       1. You can add 10MB indeed.
   2. I am not sure about the initialization process. I think it should be 
taken from those `hive-site.xml` but you can certainly check that the value is 
correctly set.




----------------------------------------------------------------
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:
us...@infra.apache.org



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

Reply via email to