[ https://issues.apache.org/jira/browse/HDFS-14355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16805884#comment-16805884 ]
Anoop Sam John commented on HDFS-14355: --------------------------------------- getBlockInputStreamWithCheckingPmemCache -> Can be private method public PmemVolumeManager getPmemVolumeManager -> Why being exposed? For tests? If so can this be package private? And also mark it with @VisibleForTesting I think the afterCache() thing is an unwanted indirection {code} FsDatasetCache try { 411 mappableBlock = cacheLoader.load(length, blockIn, metaIn, 412 blockFileName, key); 413 } catch (ChecksumException e) { 414 // Exception message is bogus since this wasn't caused by a file read 418 LOG.warn("Failed to cache the block [key=" + key + "]!", e); 419 return; 420 } 421 mappableBlock.afterCache(); PmemMappedBlock @Override 58 public void afterCache() { 59 pmemVolumeManager.afterCache(key, volumeIndex); 60 } PmemVolumeManager public void afterCache(ExtendedBlockId key, Byte volumeIndex) { 299 blockKeyToVolume.put(key, volumeIndex); 300 } {code} Actually in PmemMappableBlockLoader#load, once the load is successful (mappableBlock != null), we can do this pmemVolumeManager work right? {code} public void close() { 64 pmemVolumeManager.afterUncache(key); ... 68 FsDatasetUtil.deleteMappedFile(cacheFilePath); {code} Call afterUncache() after delete the file public PmemVolumeManager(DNConf dnConf) Can we only pass pmemVolumes and maxLockedPmem? That is cleaner IMO getVolumeByIndex -> can this be package private getCacheFilePath(ExtendedBlockId key) -> Better name would be getCachedPath(ExtendedBlockId) dfs.datanode.cache.pmem.capacity -> Am not sure any naming convention u follow in HDFS. But as a user I would prefer a name dfs.datanode.pmem.cache.capacity. Ditto for dfs.datanode.cache.pmem.dirs > Implement HDFS cache on SCM by using pure java mapped byte buffer > ----------------------------------------------------------------- > > Key: HDFS-14355 > URL: https://issues.apache.org/jira/browse/HDFS-14355 > Project: Hadoop HDFS > Issue Type: Sub-task > Components: caching, datanode > Reporter: Feilong He > Assignee: Feilong He > Priority: Major > Attachments: HDFS-14355.000.patch, HDFS-14355.001.patch, > HDFS-14355.002.patch, HDFS-14355.003.patch, HDFS-14355.004.patch, > HDFS-14355.005.patch, HDFS-14355.006.patch, HDFS-14355.007.patch, > HDFS-14355.008.patch > > > This task is to implement the caching to persistent memory using pure > {{java.nio.MappedByteBuffer}}, which could be useful in case native support > isn't available or convenient in some environments or platforms. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org