hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Manjunath (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-11182) Update DataNode to use DatasetVolumeChecker
Date Sat, 03 Dec 2016 15:14:58 GMT

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

Manjunath commented on HDFS-11182:
----------------------------------

[~arpitagarwal] I am little confused about the reason for the change in DatasetVolumeChecker.checkAllVolume
methods to use Semaphore and CoutDownLatch instead of the earlier approach of using CountDownLatch.

The confusion is whether the CountDownLatch countdown is correctly called after all threads
scheduled for completing the volume check finish their jobs.

The below code acquires all references.size()-1 permits .
{code}
final Semaphore semaphore = new Semaphore(-references.size() + 1);
{code}

The invokeCallback releases permit in Semaphore and then does a tryAcquire which would allow
it to aquire the permit it just released and hence would go and call the callback.call method
which would release the latch by the countDown.

So it looks like the first thread which completed the volume check will release the latch
which would mean we havent waited for maxAllowedTimeForCheckMs by the await call made on the
latch.

{code}
private void invokeCallback() {
      try {
        semaphore.release();
        if (callback != null && semaphore.tryAcquire()) {
          callback.call(healthyVolumes, failedVolumes);
        }
      } catch(Exception e) {
        // Propagating this exception is unlikely to be helpful.
        LOG.warn("Unexpected exception", e);
      }
    }


Futures.addCallback(future, new ResultHandler(
 -          reference, healthyVolumes, failedVolumes, resultsLatch, null));		 +          reference,
healthyVolumes, failedVolumes, semaphore, new Callback() {
 +        @Override
 +        public void call(Set<StorageLocation> ignored1,
 +                         Set<StorageLocation> ignored2) {
 +          latch.countDown();
 +        }
 +      }));
{code}

Please suggest on this.

> Update DataNode to use DatasetVolumeChecker
> -------------------------------------------
>
>                 Key: HDFS-11182
>                 URL: https://issues.apache.org/jira/browse/HDFS-11182
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: datanode
>            Reporter: Arpit Agarwal
>            Assignee: Arpit Agarwal
>
> Update DataNode to use the DatasetVolumeChecker class introduced in HDFS-11149 to parallelize
disk checks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org


Mime
View raw message