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

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

I feel like this code is still not quite right.  We can get two 
{{LeaseRenewer}} objects now, right?

1. beginFileLease calls into getLeaseRenewer, gets LeaseRenewer #1
2. LeaseRenewer#closeClient (for LeaseRenewer #1) removes itself from 
Factory.INSTANCE.
3. another thread calls beginFileLease.  There is no LeaseRenewer object in 
Factory.INSTANCE any more, so a new one is created (call it #2).
4. first thread calls put, adds the DFSClient to LeaseRenewer #1 and LR1 to 
Factory.INSTANCE
5. second thread calls put, adds the DFSClient to LeaseRenewer #2 and LR2 to 
Factory.INSTANCE.

Won't we end up with two {{LeaseRenewer}} objects after this point?

The problem is basically that if we allow the {{LeaseRenewer}} object to escape 
from LeaseRenewer.java, and we accept that these objects can "die", we have to 
accept that people can be using dead LeaseRenewer objects.

I'm not sure what the best way to fix this is... it is kind of a mess.  I guess 
maybe it's a pre-existing problem too?  If I'm understanding the situation 
correctly.

> Aborted DFSClient's impact on long running service like YARN
> ------------------------------------------------------------
>
>                 Key: HDFS-7314
>                 URL: https://issues.apache.org/jira/browse/HDFS-7314
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>            Reporter: Ming Ma
>            Assignee: Ming Ma
>         Attachments: HDFS-7314-2.patch, HDFS-7314-3.patch, HDFS-7314-4.patch, 
> HDFS-7314-5.patch, HDFS-7314-6.patch, HDFS-7314-7.patch, HDFS-7314.patch
>
>
> It happened in YARN nodemanger scenario. But it could happen to any long 
> running service that use cached instance of DistrbutedFileSystem.
> 1. Active NN is under heavy load. So it became unavailable for 10 minutes; 
> any DFSClient request will get ConnectTimeoutException.
> 2. YARN nodemanager use DFSClient for certain write operation such as log 
> aggregator or shared cache in YARN-1492. DFSClient used by YARN NM's 
> renewLease RPC got ConnectTimeoutException.
> {noformat}
> 2014-10-29 01:36:19,559 WARN org.apache.hadoop.hdfs.LeaseRenewer: Failed to 
> renew lease for [DFSClient_NONMAPREDUCE_-550838118_1] for 372 seconds.  
> Aborting ...
> {noformat}
> 3. After DFSClient is in Aborted state, YARN NM can't use that cached 
> instance of DistributedFileSystem.
> {noformat}
> 2014-10-29 20:26:23,991 INFO 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService:
>  Failed to download rsrc...
> java.io.IOException: Filesystem closed
>         at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:727)
>         at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1780)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1124)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1120)
>         at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1120)
>         at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:237)
>         at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:340)
>         at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:57)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We can make YARN or DFSClient more tolerant to temporary NN unavailability. 
> Given the callstack is YARN -> DistributedFileSystem -> DFSClient, this can 
> be addressed at different layers.
> * YARN closes the DistributedFileSystem object when it receives some well 
> defined exception. Then the next HDFS call will create a new instance of 
> DistributedFileSystem. We have to fix all the places in YARN. Plus other HDFS 
> applications need to address this as well.
> * DistributedFileSystem detects Aborted DFSClient and create a new instance 
> of DFSClient. We will need to fix all the places DistributedFileSystem calls 
> DFSClient.
> * After DFSClient gets into Aborted state, it doesn't have to reject all 
> requests , instead it can retry. If NN is available again it can transition 
> to healthy state.
> Comments?



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

Reply via email to