hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Colin Patrick McCabe (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-5431) support cachepool-based limit management in path-based caching
Date Mon, 16 Dec 2013 03:16:08 GMT

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

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

I looked at this again, and I think I found a bug.

{code}
        startTimeMs = curTimeMs;
        mark = !mark;
        rescan();
        curTimeMs = Time.monotonicNow();  <======= POINT 1
        needsRefresh = false;
        endTimeMs.updateTime(curTimeMs);
        LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
            scannedBlocks + " block(s) in " + (curTimeMs - startTimeMs) + " " +
            "millisecond(s).");
      }
{code}

What if the CRM thread gets to "point 1" right before {{waitForRescan}} is called?  Perhaps
{{waitForRescan}} will set the volatile boolean to true, but it will be immediately overwritten
by the CRM thread.  And then {{waitForRescan}} will have to wait a full 5 minutes (or whatever)
for the next update.

I think the best solution here is using an {{AtomicBoolean}}.  Then the original check in
the loop can be a compare-and-swap which atomically replaces true with false, avoiding the
lost updates problem.

> 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)

Mime
View raw message