You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "SourabhBadhya (via GitHub)" <gi...@apache.org> on 2023/03/06 13:58:28 UTC

[GitHub] [hive] SourabhBadhya opened a new pull request, #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor th…

SourabhBadhya opened a new pull request, #4098:
URL: https://github.com/apache/hive/pull/4098

   …reads
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Use Caffeine for caching metadata objects in Compactor threads
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Caffeine is a better library than Guava for caching, hence better to use it.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Existing tests


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] akshat0395 commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "akshat0395 (via GitHub)" <gi...@apache.org>.
akshat0395 commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127398769


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -133,6 +150,24 @@ public static List<Partition> getPartitionsByNames(HiveConf conf, String dbName,
     }
   }
 
+  public static Database resolveDatabase(HiveConf conf, String dbName) throws MetaException, NoSuchObjectException {
+    try {
+      return getMSForConf(conf).getDatabase(MetaStoreUtils.getDefaultCatalog(conf), dbName);
+    } catch (NoSuchObjectException e) {
+      LOG.error("Unable to find database {}, {}", dbName, e.getMessage());
+      throw e;
+    }
+  }
+
+  public static Table resolveTable(HiveConf conf, String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(MetaStoreUtils.getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find database {}, {}", dbName, e.getMessage());

Review Comment:
   This method is resolving for table, This log should be `Unable to find Table {}, {}`, tableName



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -74,6 +77,20 @@ static Runnable unchecked(ThrowingRunnable<?> r) {
     }
   }
 
+  public interface ThrowingFunction<T, R, E extends Exception> {

Review Comment:
   Can we please add comments to this method to reduce the cognitive complexity of this method



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127915943


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -66,12 +66,7 @@ public void init(AtomicBoolean stop) throws Exception {
   }
 
   @Override Table resolveTable(CompactionInfo ci) throws MetaException {
-    try {
-      return getMSForConf(conf).getTable(getDefaultCatalog(conf), ci.dbname, ci.tableName);
-    } catch (MetaException e) {
-      LOG.error("Unable to find table " + ci.getFullTableName() + ", " + e.getMessage());
-      throw e;
-    }
+    return CompactorUtil.resolveTable(conf, ci.dbname, ci.tableName);

Review Comment:
   Its also a utility method used across Initiator and worker. To reduce common code, I made it use the static function in `CompactorUtil`. Also removing this and using `CompactorUtil.resolveTable`, will make tests use static mocking which is undesirable.
   
   



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127455767


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -133,6 +150,24 @@ public static List<Partition> getPartitionsByNames(HiveConf conf, String dbName,
     }
   }
 
+  public static Database resolveDatabase(HiveConf conf, String dbName) throws MetaException, NoSuchObjectException {
+    try {
+      return getMSForConf(conf).getDatabase(MetaStoreUtils.getDefaultCatalog(conf), dbName);
+    } catch (NoSuchObjectException e) {
+      LOG.error("Unable to find database {}, {}", dbName, e.getMessage());
+      throw e;
+    }
+  }
+
+  public static Table resolveTable(HiveConf conf, String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(MetaStoreUtils.getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find database {}, {}", dbName, e.getMessage());

Review Comment:
   Nice catch. Modified accordingly. Done.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127901922


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {

Review Comment:
   This was part of previous refactor wherein it was suggested to name it as `MetadataCache` since it only caches database and table objects currently.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127595407


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java:
##########
@@ -58,12 +55,7 @@ public abstract class TaskHandler {
   public abstract List<Runnable> getTasks() throws MetaException;
 
   protected Table resolveTable(String dbName, String tableName) throws MetaException {
-    try {
-      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
-    } catch (MetaException e) {
-      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
-      throw e;
-    }
+    return CompactorUtil.resolveTable(conf, dbName, tableName);

Review Comment:
   I think it's better to keep a utility method for fetching Table objects in the generic class TaskHandler. Provides an easy way to test things out.
   
   The cache is created in MetastoreCompactorThread and it's passed around to all required TaskHandlers so that it can be used.
   
   See this and its usages -https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1457835955

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [5 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] akshat0395 commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "akshat0395 (via GitHub)" <gi...@apache.org>.
akshat0395 commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1457701537

   LGTM +1


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] veghlaci05 commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "veghlaci05 (via GitHub)" <gi...@apache.org>.
veghlaci05 commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127815441


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -66,12 +66,7 @@ public void init(AtomicBoolean stop) throws Exception {
   }
 
   @Override Table resolveTable(CompactionInfo ci) throws MetaException {
-    try {
-      return getMSForConf(conf).getTable(getDefaultCatalog(conf), ci.dbname, ci.tableName);
-    } catch (MetaException e) {
-      LOG.error("Unable to find table " + ci.getFullTableName() + ", " + e.getMessage());
-      throw e;
-    }
+    return CompactorUtil.resolveTable(conf, ci.dbname, ci.tableName);

Review Comment:
   resolveTable can be removed from here and CompactorUtil.resolveTable can be called directly



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {

Review Comment:
   Both MetadataCache type and tableCacheOn param could have a more generic name, as this class is not specific to Metadata/tables at all.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {
     if (tableCacheOn) {
-      metaCache = CacheBuilder.newBuilder().softValues().build();
+      metaCache = Caffeine.newBuilder().softValues().build();
     }
   }
 
-  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Function<? super String, ? extends TBase> function) throws Exception {
     if (metaCache != null) {
-      try {
-        return (T) metaCache.get(key, callable);
-      } catch (ExecutionException e) {
-        throw (Exception) e.getCause();
-      }

Review Comment:
   Why did you replace the exception handling with a Functional interface? Is there any benefit from it, because now the client code must wrap each call.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java:
##########
@@ -58,12 +55,7 @@ public abstract class TaskHandler {
   public abstract List<Runnable> getTasks() throws MetaException;
 
   protected Table resolveTable(String dbName, String tableName) throws MetaException {
-    try {
-      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
-    } catch (MetaException e) {
-      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
-      throw e;
-    }
+    return CompactorUtil.resolveTable(conf, dbName, tableName);

Review Comment:
   For now it is fine to leave it there. But this method is only required because CompactorUtil methods are static. We need to get rid of these static utility methods, so we will be able to mock `CompactorUtil.resolveTable` directly and remove this method.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -74,6 +77,40 @@ static Runnable unchecked(ThrowingRunnable<?> r) {
     }
   }
 
+  /**
+   * An interface to create {@link Function} which can throw exceptions.
+   * This is especially useful when we pass lambda-expressions which embed functions which throw exceptions
+   * @param <T> the type of the input to the function
+   * @param <R> the type of the result of the function
+   * @param <E> the exception that is being thrown by the function
+   */
+  public interface ThrowingFunction<T, R, E extends Exception> {

Review Comment:
   I'm not sure this interface is needed. `R apply(T t)` is called only from `unchecked`, and all `unchecked` calls are done for `org.apache.hadoop.hive.ql.txn.compactor.MetadataCache#computeIfAbsent` from which it was extracted. I don't get the point why is this better, it just makes the computeIfAbsent call more complicated.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127894227


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {
     if (tableCacheOn) {
-      metaCache = CacheBuilder.newBuilder().softValues().build();
+      metaCache = Caffeine.newBuilder().softValues().build();
     }
   }
 
-  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Function<? super String, ? extends TBase> function) throws Exception {
     if (metaCache != null) {
-      try {
-        return (T) metaCache.get(key, callable);
-      } catch (ExecutionException e) {
-        throw (Exception) e.getCause();
-      }

Review Comment:
   This is required as an input parameter for caching in Caffeine is - 
   `metaCache.get(key, an instance of Function).`
   
   Since the API itself is using functional interface, I modified all callables to functional instances. Yes there is a problem of exception handling when using functional interface which is why I wrote the `ThrowingFunction` interface.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127455288


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -74,6 +77,20 @@ static Runnable unchecked(ThrowingRunnable<?> r) {
     }
   }
 
+  public interface ThrowingFunction<T, R, E extends Exception> {

Review Comment:
   Added Javadoc for the same. Done.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1459569757

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127897811


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -74,6 +77,40 @@ static Runnable unchecked(ThrowingRunnable<?> r) {
     }
   }
 
+  /**
+   * An interface to create {@link Function} which can throw exceptions.
+   * This is especially useful when we pass lambda-expressions which embed functions which throw exceptions
+   * @param <T> the type of the input to the function
+   * @param <R> the type of the result of the function
+   * @param <E> the exception that is being thrown by the function
+   */
+  public interface ThrowingFunction<T, R, E extends Exception> {

Review Comment:
   See above comment



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1458614686

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [4 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] rkirtir commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "rkirtir (via GitHub)" <gi...@apache.org>.
rkirtir commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127580345


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java:
##########
@@ -58,12 +55,7 @@ public abstract class TaskHandler {
   public abstract List<Runnable> getTasks() throws MetaException;
 
   protected Table resolveTable(String dbName, String tableName) throws MetaException {
-    try {
-      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
-    } catch (MetaException e) {
-      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
-      throw e;
-    }
+    return CompactorUtil.resolveTable(conf, dbName, tableName);

Review Comment:
   How about calling CompactorUtil.resolveTable(conf, dbName, tableName) from initiator directly. 
   does taskhandler responsible for all caffein cache related
    tasks?



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] veghlaci05 merged pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "veghlaci05 (via GitHub)" <gi...@apache.org>.
veghlaci05 merged PR #4098:
URL: https://github.com/apache/hive/pull/4098


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1457508154

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [5 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1128026739


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {
     if (tableCacheOn) {
-      metaCache = CacheBuilder.newBuilder().softValues().build();
+      metaCache = Caffeine.newBuilder().softValues().build();
     }
   }
 
-  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Function<? super String, ? extends TBase> function) throws Exception {
     if (metaCache != null) {
-      try {
-        return (T) metaCache.get(key, callable);
-      } catch (ExecutionException e) {
-        throw (Exception) e.getCause();
-      }

Review Comment:
   ThrowingFunction is no more required. Added the definition of the function within `computeIfAbsent`.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1127894227


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -17,32 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.thrift.TBase;
 
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 
 public class MetadataCache {
 
   private Cache<String, TBase> metaCache;
 
   public MetadataCache(boolean tableCacheOn) {
     if (tableCacheOn) {
-      metaCache = CacheBuilder.newBuilder().softValues().build();
+      metaCache = Caffeine.newBuilder().softValues().build();
     }
   }
 
-  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Function<? super String, ? extends TBase> function) throws Exception {
     if (metaCache != null) {
-      try {
-        return (T) metaCache.get(key, callable);
-      } catch (ExecutionException e) {
-        throw (Exception) e.getCause();
-      }

Review Comment:
   This is required as an input parameter for caching in Caffeine - 
   `metaCache.get(key, an instance of Function).`
   
   Since the API itself is using functional interface, I modified all callables to functional instances. Yes there is a problem of exception handling when using functional interface which is why I wrote the `ThrowingFunction` interface.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4098:
URL: https://github.com/apache/hive/pull/4098#discussion_r1128028232


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java:
##########
@@ -74,6 +77,40 @@ static Runnable unchecked(ThrowingRunnable<?> r) {
     }
   }
 
+  /**
+   * An interface to create {@link Function} which can throw exceptions.
+   * This is especially useful when we pass lambda-expressions which embed functions which throw exceptions
+   * @param <T> the type of the input to the function
+   * @param <R> the type of the result of the function
+   * @param <E> the exception that is being thrown by the function
+   */
+  public interface ThrowingFunction<T, R, E extends Exception> {

Review Comment:
   Removed `ThrowingFunction`. Done.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1459473844

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1456904823

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [5 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4098: HIVE-27122: Use Caffeine for caching metadata objects in Compactor threads

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4098:
URL: https://github.com/apache/hive/pull/4098#issuecomment-1456281279

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4098)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4098&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4098&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4098&metric=duplicated_lines_density&view=list) No Duplication information
   
   


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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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