You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "wchevreuil (via GitHub)" <gi...@apache.org> on 2023/08/02 09:24:47 UTC

[GitHub] [hbase] wchevreuil commented on a diff in pull request #5339: HBASE-27997 Enhance prefetch executor to record region prefetch infor…

wchevreuil commented on code in PR #5339:
URL: https://github.com/apache/hbase/pull/5339#discussion_r1281631574


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -53,7 +56,8 @@ public final class PrefetchExecutor {
   private static final Map<Path, Future<?>> prefetchFutures = new ConcurrentSkipListMap<>();
   /** Set of files for which prefetch is completed */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "MS_SHOULD_BE_FINAL")
-  private static HashMap<String, Boolean> prefetchCompleted = new HashMap<>();
+  private static ConcurrentHashMap<String, Long> regionPrefetchSizeMap = new ConcurrentHashMap<>();

Review Comment:
   OOP good practices: "Program to interfaces"`Map<String, Long> regionPrefetchSizeMap = new ConcurrentHashMap<>();`



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -53,7 +56,8 @@ public final class PrefetchExecutor {
   private static final Map<Path, Future<?>> prefetchFutures = new ConcurrentSkipListMap<>();
   /** Set of files for which prefetch is completed */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "MS_SHOULD_BE_FINAL")
-  private static HashMap<String, Boolean> prefetchCompleted = new HashMap<>();
+  private static ConcurrentHashMap<String, Long> regionPrefetchSizeMap = new ConcurrentHashMap<>();
+  private static HashMap<String, Map<String, Long>> prefetchCompleted = new HashMap<>();

Review Comment:
   Should use `Map<String,Pair<String,Long>> prefetchCompleted`. 



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -120,9 +124,35 @@ public static void request(Path path, Runnable runnable) {
     }
   }
 
-  public static void complete(Path path) {
+  private static void removeFileFromPrefetch(String hFileName) {
+    // Update the regionPrefetchedSizeMap before removing the file from prefetchCompleted
+    if (prefetchCompleted.containsKey(hFileName)) {
+      Map.Entry<String, Long> regionEntry =
+        prefetchCompleted.get(hFileName).entrySet().iterator().next();
+      String regionEncodedName = regionEntry.getKey();
+      long filePrefetchedSize = regionEntry.getValue();
+      if (LOG.isDebugEnabled()) {

Review Comment:
   Nit: no need for this check, since we are not doing any computation on the debug message.



-- 
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: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org