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

ASF GitHub Bot commented on HDFS-16855:
---------------------------------------

dingshun3016 commented on PR #5170:
URL: https://github.com/apache/hadoop/pull/5170#issuecomment-1333710803

   according to the situation discussed so far, it seems that there are several 
ways to solve this problem
   -  remove the BLOCK_POOl level write lock in #addBlockPool
        > but worry about having replica consistency problem
   - forbid refresh() when ReplicaCachingGetSpaceUsed#init() at first time
        > it will cause the value of dfsUsage to be 0 until the next time 
refresh()
   - use du or df command instead at first time
        > du is very expensive and slow
        > df is inaccurate when the disk sharing by other servers
            reference 
[HDFS-14313](https://issues.apache.org/jira/browse/HDFS-14313)
   
   Now that, this case only happen when invoke addBlockPool() and 
CachingGetSpaceUsed#used < 0,  I have an idea,  is it possible to add a switch, 
not add lock when ReplicaCachingGetSpaceUsed#init() at first time , and add it 
at other times
   
   do you think it's possible?@MingXiangLi 




> Remove the redundant write lock in addBlockPool
> -----------------------------------------------
>
>                 Key: HDFS-16855
>                 URL: https://issues.apache.org/jira/browse/HDFS-16855
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: datanode
>            Reporter: dingshun
>            Priority: Major
>              Labels: pull-request-available
>
> When patching the datanode's fine-grained lock, we found that the datanode 
> couldn't start,maybe happened deadlock,when addBlockPool, so we can remove it.
> {code:java}
> // getspaceused classname
>   <property>
>     <name>fs.getspaceused.classname</name>
>     
> <value>org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed</value>
>   </property> {code}
> {code:java}
> // 
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl#addBlockPool
>  
> // get writeLock
> @Override
> public void addBlockPool(String bpid, Configuration conf)
>     throws IOException {
>   LOG.info("Adding block pool " + bpid);
>   AddBlockPoolException volumeExceptions = new AddBlockPoolException();
>   try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, 
> bpid)) {
>     try {
>       volumes.addBlockPool(bpid, conf);
>     } catch (AddBlockPoolException e) {
>       volumeExceptions.mergeException(e);
>     }
>     volumeMap.initBlockPool(bpid);
>     Set<String> vols = storageMap.keySet();
>     for (String v : vols) {
>       lockManager.addLock(LockLevel.VOLUME, bpid, v);
>     }
>   }
>  
> } {code}
> {code:java}
> // 
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl#deepCopyReplica
> // need readLock
> void replicas(String bpid, Consumer<Iterator<ReplicaInfo>> consumer) {
>   LightWeightResizableGSet<Block, ReplicaInfo> m = null;
>   try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, 
> bpid)) {
>     m = map.get(bpid);
>     if (m !=null) {
>       m.getIterator(consumer);
>     }
>   }
> } {code}
>  
> because it is not the same thread, so the write lock cannot be downgraded to 
> a read lock
> {code:java}
> void addBlockPool(final String bpid, final Configuration conf) throws 
> IOException {
>   long totalStartTime = Time.monotonicNow();
>   final Map<FsVolumeSpi, IOException> unhealthyDataDirs =
>       new ConcurrentHashMap<FsVolumeSpi, IOException>();
>   List<Thread> blockPoolAddingThreads = new ArrayList<Thread>();
>   for (final FsVolumeImpl v : volumes) {
>     Thread t = new Thread() {
>       public void run() {
>         try (FsVolumeReference ref = v.obtainReference()) {
>           FsDatasetImpl.LOG.info("Scanning block pool " + bpid +
>               " on volume " + v + "...");
>           long startTime = Time.monotonicNow();
>           v.addBlockPool(bpid, conf);
>           long timeTaken = Time.monotonicNow() - startTime;
>           FsDatasetImpl.LOG.info("Time taken to scan block pool " + bpid +
>               " on " + v + ": " + timeTaken + "ms");
>         } catch (IOException ioe) {
>           FsDatasetImpl.LOG.info("Caught exception while scanning " + v +
>               ". Will throw later.", ioe);
>           unhealthyDataDirs.put(v, ioe);
>         }
>       }
>     };
>     blockPoolAddingThreads.add(t);
>     t.start();
>   }
>   for (Thread t : blockPoolAddingThreads) {
>     try {
>       t.join();
>     } catch (InterruptedException ie) {
>       throw new IOException(ie);
>     }
>   }
> } {code}
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to