-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-27997 Enhance prefetch executor to record region prefetch infor… #5339
Changes from 2 commits
b069f1f
90d1df6
eb6b155
28cc475
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,12 +17,15 @@ | |
*/ | ||
package org.apache.hadoop.hbase.io.hfile; | ||
|
||
import com.google.errorprone.annotations.RestrictedApi; | ||
import java.io.File; | ||
import java.io.FileInputStream; | ||
import java.io.FileOutputStream; | ||
import java.io.IOException; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ConcurrentSkipListMap; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.RejectedExecutionException; | ||
|
@@ -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<>(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updated the code as per the suggestion. Replaced HashMap and ConcurrentHashMap to Map. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How about using Pair<String,Long> as the map value? We don't really need a map as the value here. You can use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch Wellington, Thanks for pointing it out. Made the change in the updated patch. |
||
/** Executor pool shared among all HFiles for block prefetch */ | ||
private static final ScheduledExecutorService prefetchExecutorPool; | ||
/** Delay before beginning prefetch */ | ||
|
@@ -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()) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: no need for this check, since we are not doing any computation on the debug message. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Remove the check. |
||
LOG.debug("Removing file {} for region {}", hFileName, regionEncodedName); | ||
} | ||
regionPrefetchSizeMap.computeIfPresent(regionEncodedName, | ||
(rn, pf) -> pf - filePrefetchedSize); | ||
// If all the blocks for a region are evicted from the cache, remove the entry for that region | ||
if ( | ||
regionPrefetchSizeMap.containsKey(regionEncodedName) | ||
&& regionPrefetchSizeMap.get(regionEncodedName) == 0 | ||
) { | ||
regionPrefetchSizeMap.remove(regionEncodedName); | ||
} | ||
} | ||
prefetchCompleted.remove(hFileName); | ||
} | ||
|
||
public static void complete(final String regionName, Path path, long size) { | ||
prefetchFutures.remove(path); | ||
prefetchCompleted.put(path.getName(), true); | ||
Map<String, Long> tmpMap = new HashMap<>(); | ||
tmpMap.put(regionName, size); | ||
prefetchCompleted.put(path.getName(), tmpMap); | ||
regionPrefetchSizeMap.merge(regionName, size, (oldpf, fileSize) -> oldpf + fileSize); | ||
LOG.debug("Prefetch completed for {}", path.getName()); | ||
} | ||
|
||
|
@@ -173,11 +203,28 @@ public static void retrieveFromFile(String path) throws IOException { | |
try (FileInputStream fis = deleteFileOnClose(prefetchPersistenceFile)) { | ||
PersistentPrefetchProtos.PrefetchedHfileName proto = | ||
PersistentPrefetchProtos.PrefetchedHfileName.parseDelimitedFrom(fis); | ||
Map<String, Boolean> protoPrefetchedFilesMap = proto.getPrefetchedFilesMap(); | ||
prefetchCompleted.putAll(protoPrefetchedFilesMap); | ||
Map<String, PersistentPrefetchProtos.RegionFileSizeMap> protoPrefetchedFilesMap = | ||
proto.getPrefetchedFilesMap(); | ||
prefetchCompleted.putAll(PrefetchProtoUtils.fromPB(protoPrefetchedFilesMap)); | ||
updateRegionSizeMapWhileRetrievingFromFile(); | ||
} | ||
} | ||
|
||
private static void updateRegionSizeMapWhileRetrievingFromFile() { | ||
// Update the regionPrefetchedSizeMap with the region size while restarting the region server | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Updating region size map after retrieving prefetch file list"); | ||
} | ||
prefetchCompleted.forEach((hFileName, hFileSize) -> { | ||
// Get the region name for each file | ||
Map.Entry<String, Long> regionEntry = hFileSize.entrySet().iterator().next(); | ||
String regionEncodedName = regionEntry.getKey(); | ||
long filePrefetchSize = regionEntry.getValue(); | ||
regionPrefetchSizeMap.merge(regionEncodedName, filePrefetchSize, | ||
(oldpf, fileSize) -> oldpf + fileSize); | ||
}); | ||
} | ||
|
||
private static FileInputStream deleteFileOnClose(final File file) throws IOException { | ||
return new FileInputStream(file) { | ||
private File myFile; | ||
|
@@ -203,13 +250,24 @@ public void close() throws IOException { | |
} | ||
|
||
public static void removePrefetchedFileWhileEvict(String hfileName) { | ||
prefetchCompleted.remove(hfileName); | ||
removeFileFromPrefetch(hfileName); | ||
} | ||
|
||
public static boolean isFilePrefetched(String hfileName) { | ||
return prefetchCompleted.containsKey(hfileName); | ||
} | ||
|
||
public static Map<String, Long> getRegionPrefetchInfo() { | ||
return Collections.unmodifiableMap(regionPrefetchSizeMap); | ||
} | ||
|
||
@RestrictedApi(explanation = "Should only be called in tests", link = "", | ||
allowedOnPath = ".*(/src/test/.*|PrefetchExecutor).java") | ||
public static void reset() { | ||
prefetchCompleted = new HashMap<>(); | ||
regionPrefetchSizeMap = new ConcurrentHashMap<>(); | ||
} | ||
|
||
private PrefetchExecutor() { | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OOP good practices: "Program to interfaces"
Map<String, Long> regionPrefetchSizeMap = new ConcurrentHashMap<>();
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.