-
Couldn't load subscription status.
- Fork 3.4k
HBASE-28468: Integration of time-based priority caching in eviction logic. #5826
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -76,6 +76,7 @@ | |
| import org.apache.hadoop.hbase.nio.ByteBuff; | ||
| import org.apache.hadoop.hbase.nio.RefCnt; | ||
| import org.apache.hadoop.hbase.protobuf.ProtobufMagic; | ||
| import org.apache.hadoop.hbase.regionserver.DataTieringManager; | ||
| import org.apache.hadoop.hbase.util.Bytes; | ||
| import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; | ||
| import org.apache.hadoop.hbase.util.IdReadWriteLock; | ||
|
|
@@ -141,8 +142,8 @@ public class BucketCache implements BlockCache, HeapSize { | |
| /** Statistics thread */ | ||
| private static final int statThreadPeriod = 5 * 60; | ||
|
|
||
| final static int DEFAULT_WRITER_THREADS = 3; | ||
| final static int DEFAULT_WRITER_QUEUE_ITEMS = 64; | ||
| public final static int DEFAULT_WRITER_THREADS = 3; | ||
| public final static int DEFAULT_WRITER_QUEUE_ITEMS = 64; | ||
|
|
||
| // Store/read block data | ||
| transient final IOEngine ioEngine; | ||
|
|
@@ -935,6 +936,16 @@ void freeSpace(final String why) { | |
| } | ||
| try { | ||
| freeInProgress = true; | ||
|
|
||
| // Check the list of files to determine the cold files which can be readily evicted. | ||
| Set<String> coldFiles = | ||
| DataTieringManager.getInstance().getColdDataFiles(backingMap.keySet()); | ||
| if (coldFiles != null) { | ||
| for(String fileName : coldFiles) { | ||
| evictBlocksByHfileName(fileName); | ||
| } | ||
| } | ||
|
|
||
|
Comment on lines
+939
to
+948
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. We can do like this here, but then can we think if we can modify DataTieringManager somehow to track when we have no CF with TIME_RANGE type? That way we could avoid this extra loops through the whole set of block keys unnecessarily. Alternatively, we could follow the BucketEntryGroup logic starting from #987. We could define a COLD priority group and add that as the first group in the priority queue. That way we don't need this extra loop over the whole block sets to find out which blocks are cold. We would leverage the already existing loop from line #996 for that. |
||
| long bytesToFreeWithoutExtra = 0; | ||
| // Calculate free byte for each bucketSizeinfo | ||
| StringBuilder msgBuffer = LOG.isDebugEnabled() ? new StringBuilder() : null; | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,22 +25,28 @@ | |
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentSkipListSet; | ||
| import java.util.function.Function; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.hbase.io.ByteBuffAllocator; | ||
| import org.apache.hadoop.hbase.io.ByteBuffAllocator.Recycler; | ||
| import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; | ||
| import org.apache.hadoop.hbase.io.hfile.BlockPriority; | ||
| import org.apache.hadoop.hbase.io.hfile.BlockType; | ||
| import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager; | ||
| import org.apache.hadoop.hbase.io.hfile.HFileBlock; | ||
| import org.apache.hadoop.hbase.regionserver.DataTieringManager; | ||
| import org.apache.hadoop.hbase.util.Pair; | ||
| import org.apache.yetus.audience.InterfaceAudience; | ||
|
|
||
| import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; | ||
|
|
||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.BucketCacheProtos; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| @InterfaceAudience.Private | ||
| final class BucketProtoUtils { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(BucketProtoUtils.class); | ||
| private BucketProtoUtils() { | ||
|
|
||
| } | ||
|
|
@@ -130,10 +136,30 @@ static Pair<ConcurrentHashMap<BlockCacheKey, BucketEntry>, NavigableSet<BlockCac | |
| ConcurrentHashMap<BlockCacheKey, BucketEntry> result = new ConcurrentHashMap<>(); | ||
| NavigableSet<BlockCacheKey> resultSet = new ConcurrentSkipListSet<>(Comparator | ||
| .comparing(BlockCacheKey::getHfileName).thenComparingLong(BlockCacheKey::getOffset)); | ||
|
|
||
| Map<String, Path> allFilePaths = null; | ||
| DataTieringManager dataTieringManager; | ||
| try { | ||
| dataTieringManager = DataTieringManager.getInstance(); | ||
| allFilePaths = dataTieringManager.getAllFilesList(); | ||
| } catch (IllegalStateException e) { | ||
| // Data-Tiering manager has not been set up. | ||
| // Ignore the error and proceed with the normal flow. | ||
| LOG.error("Error while getting DataTieringManager instance: {}", e.getMessage()); | ||
|
Comment on lines
+146
to
+148
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. Is this to be expected often? If so, let's lower the log level to DEBUG. If not, it seems we can still continue RS normal functioning, so it should rather be a WARNING than an ERROR. 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. ack! |
||
| } | ||
|
|
||
| for (BucketCacheProtos.BackingMapEntry entry : backingMap.getEntryList()) { | ||
| BucketCacheProtos.BlockCacheKey protoKey = entry.getKey(); | ||
| BlockCacheKey key = new BlockCacheKey(protoKey.getHfilename(), protoKey.getOffset(), | ||
| protoKey.getPrimaryReplicaBlock(), fromPb(protoKey.getBlockType())); | ||
|
|
||
| BlockCacheKey key; | ||
|
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. Need handling with and without paths in the BlockCacheKey creation |
||
|
|
||
| if(allFilePaths != null && allFilePaths.containsKey(protoKey.getHfilename())) { | ||
| key = new BlockCacheKey(allFilePaths.get(protoKey.getHfilename()), protoKey.getOffset(), | ||
| protoKey.getPrimaryReplicaBlock(), fromPb(protoKey.getBlockType())); | ||
| } else { | ||
| key = new BlockCacheKey(protoKey.getHfilename(), protoKey.getOffset(), | ||
| protoKey.getPrimaryReplicaBlock(), fromPb(protoKey.getBlockType())); | ||
| } | ||
| BucketCacheProtos.BucketEntry protoValue = entry.getValue(); | ||
| // TODO:We use ByteBuffAllocator.HEAP here, because we could not get the ByteBuffAllocator | ||
| // which created by RpcServer elegantly. | ||
|
|
||
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.
How do you know this is a Data block? At this point, I don't think this can be assumed.
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.
The constructor used earlier(BlockCacheKey(name, offset);) sets these parameters internally. Hence, I used these parameters:
public BlockCacheKey(String hfileName, long offset) {
this(hfileName, offset, true, BlockType.DATA);
}