[ 
https://issues.apache.org/jira/browse/HDFS-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13848765#comment-13848765
 ] 

Colin Patrick McCabe commented on HDFS-5431:
--------------------------------------------

bq. With the new EnumSet, I also added a version of add/modify directive 
without a flags argument, since it's really annoying to put 
EnumSet.noneOf(CacheFlag.class everywhere.

Yeah, I agree.  That is the one thing I never liked about EnumSets.  I suppose 
we could declare a constant insider DFS like NONE, but it's probably not worth 
it.

bq. Tweaked the kick behavior somewhat. We now try to do read-after-write 
consistency with the dirty bit. add/modify with force is essentially a read and 
a write.

Great.

bq. I don't like putting bytesOverlimit in CachePoolEntry because it feels 
inconsistent with CachePoolStats. Left it as is, unless you feel strongly on 
this one.

I think it's fine as-is.  Avoiding duplication is good, but I think you are 
right that putting it into entry would add too much complexity.

{code}
     this.cachedBlocks = cacheManager.getCachedBlocks();
     this.intervalMs = intervalMs;
+    this.needsRefresh = true;
   }
{code}
We don't need this here, since it's initialized in {{run}}.

{code}
    synchronized (this) {
      needsRefresh = true;
      this.notifyAll();
    }
{code}
I would move {{needsRefresh}} out of the synchronized block.  Since it's 
volatile, you don't need to protect it (and it's nice to make that clear).

+1 once those changes are addressed.

> support cachepool-based limit management in path-based caching
> --------------------------------------------------------------
>
>                 Key: HDFS-5431
>                 URL: https://issues.apache.org/jira/browse/HDFS-5431
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: datanode, namenode
>    Affects Versions: 3.0.0
>            Reporter: Colin Patrick McCabe
>            Assignee: Andrew Wang
>         Attachments: hdfs-5431-1.patch, hdfs-5431-2.patch, hdfs-5431-3.patch, 
> hdfs-5431-4.patch, hdfs-5431-5.patch
>
>
> We should support cachepool-based limit management in path-based caching.



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)

Reply via email to