hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Arv Mistry" <...@kindsight.net>
Subject RE: File Descriptors not cleaned up
Date Thu, 31 Jul 2008 19:16:14 GMT
 
Raghu,

In the test program I see 3 fd's used when the fs.open() is called. Two
of these are pipe and 1 eventpoll.
These 3 are never cleaned up and stay around. I track this by running it
in the debug mode and put a break point and use
Lsof -p <pid> to see the fd's. I do a diff of the output before the open
and after the open.

What I don't understand is why this doesn't get cleaned up when done in
a separate thread but does when its done in a single thread.

This is a problem in the real system because I run out of fd's and am no
longer able to open any more files after a few weeks.
This forces me to do a system restart to flush things out.

Cheers Arv

-----Original Message-----
From: Raghu Angadi [mailto:rangadi@yahoo-inc.com] 
Sent: Thursday, July 31, 2008 2:33 PM
To: core-user@hadoop.apache.org
Subject: Re: File Descriptors not cleaned up


Also could you respond to the earlier questions regd your test program
(slightly corrected) :

"What do you see in your test program and how is it different from what
you expect? In addition, why is that a problem?"

Raghu.

Arv Mistry wrote:
>  
> Raghu,
> 
> This is a real scenario for our application which nightly does 
> basically what the test stub is doing and what happens there now is 
> after several weeks the system stops processing requests with an 
> exception "Too many files open".
> 
> The default ulimit is 1024 files (I think this is per process) so once

> my process exceeds this it will throw an exception and I am forced to 
> restart.
> 
> Why does this not happen when I'm in a single-threaded mode?
> 
> Will you be able to provide a fix?
> 
> Cheers Arv
> 
> -----Original Message-----
> From: Raghu Angadi [mailto:rangadi@yahoo-inc.com]
> Sent: Thursday, July 31, 2008 1:10 PM
> To: core-user@hadoop.apache.org
> Subject: Re: File Descriptors not cleaned up
> 
> 
> I might have missed something earlier and might be asking some thing 
> you already answered. Hope its ok :
> 
> What do expect to see in you test program and how is it different from

> what you expect? In addition, why is that a problem?
> 
> Hadoop implementation *does leave* the 3 fds you mentioned. We could 
> get rid of it.. but we don't (yet). We could have another clean up 
> thread or scheduled some thing with JVM timer.
> 
> I can write a test that leaves hundreds or thousands of fds open. We 
> just thought that is case is not practical.
> 
> It is still not clear to me if you are saying your code is leaving 
> lots and lots of fds open or if you are just wondering about the 3 you

> mentioned earlier.
> 
> Please check out a few comments starting at
> https://issues.apache.org/jira/browse/HADOOP-2346?focusedCommentId=125
> 66
> 250#action_12566250
> for more background on design choices made.
> 
> Raghu.
> 
> Arv Mistry wrote:
>> I guess the attachment got stripped, so here it is inline ...
>>
>>  public class TestFsHadoop {
>>
>> 	public static Configuration conf       = null;
>> 	public static FileSystem fs = null;
>>
>> 		
>> 	/**
>> 	 * @param args
>> 	 */
>> 	public static void main(String[] args) {
>> 		
>>     	try {
>>     		    		
>>     		initHdfsReader();
>>     		
>>     		// Just have one handle to the FS. This only closed when
> the
>> application is shutdown.
>>     		fs = getFileSystem();
>>     					
>>     		// Spawn a new thread to do the work
>>     		TestThread t = new TestFsHadoop().new TestThread();
>>     		t.start();			
>>     		
>>     	} catch (Exception e) {
>>     		e.printStackTrace();
>>     	}
>> 	}
>>
>> 	static public FileSystem getFileSystem () {
>> 		
>> 		FileSystem fs = null;
>> 		
>> 		try {
>> 			
>> 			fs = FileSystem.get(conf);
>> 			
>> 			fs.setWorkingDirectory(new
>> Path("/opt/hadoop/data"));
>> 			
>> 		} catch (Exception e) {
>> 			e.printStackTrace();
>> 		}
>> 		
>> 		System.out.println ("Returning FS " + fs);
>> 		
>> 		return (fs);
>> 	}	
>> 	
>> 	
>> 	/**
>> 	 * 
>> 	 */
>> 	static private void initHdfsReader () {
>> 	    
>> 	    try {	 
>> 	    	
>> 	    	conf = new Configuration();
>> 	    	
>> 	    	String url =
>> "File:///opt/profilecluster/hadoop-0.17.0/conf/hadoop-default.xml";
>>
>> 	    	Path path = new Path (url);
>> 	    	
>> 	    	conf.addResource(path);
>> 	    	
>> 	    	url =
>> "File:///opt/profilecluster/hadoop-0.17.0/conf/hadoop-site.xml";
>>
>> 	    	path = new Path (url);
>> 	    	
>> 	    	conf.addResource(path);
>> 				    
>> 		
>> 	    } catch (Exception e) {
>> 	    	e.printStackTrace();
>> 	    }
>> 	}
>> 	
>> 	public class TestThread extends Thread {
>> 		
>> 		/**
>> 		 *
>> 		 */
>> 		public void run() {
>> 					
>> 			try {  		
>>
>> 				// hard-coded to open a file for this
>> test harness				
>> 				 Path p = new
>> Path("/opt/hadoop/data/clickstream/cs_1_20080729_1_of_5.dat.gz");
>> 				 
>> 				FSDataInputStream fis = fs.open(p);
>>
>> 				
>> 				byte[] in = new byte[5];
>> 				
>> 				int bytesRead = 0;
>>
>> 				while (((bytesRead = fis.read(in)) !=
>> -1) && (bytesRead > 0)) {
>> 					// ... Do Stuff ... 
>> 				}
>> 				
>> 				fis.close();
>> 				
>> 			} catch (Exception e) {			
>> 				e.printStackTrace();
>> 			}
>> 		}
>> 	}
>> 	
>> }
>>
>> -----Original Message-----
>> From: Arv Mistry [mailto:arv@kindsight.net]
>> Sent: Thursday, July 31, 2008 9:30 AM
>> To: core-user@hadoop.apache.org
>> Subject: RE: File Descriptors not cleaned up
>>
>>  
>> I've simplified the code into a simple test harness with just hadoop 
>> (see attached file)
>>
>> I found that I can only reproduce this problem when I am doing the
>> fs.open() in a different thread. Even though in that same thread I am

>> doing a close().
>>
>> Cheers Arv
>>
>> -----Original Message-----
>> From: Raghu Angadi [mailto:rangadi@yahoo-inc.com]
>> Sent: Wednesday, July 30, 2008 7:36 PM
>> To: core-user@hadoop.apache.org
>> Subject: Re: File Descriptors not cleaned up
>>
>> Arv Mistry wrote:
>>>  
>>> Thanks for responding Raghu,
>>>
>>> This code is run every hour, where I open a file ( a different file 
>>> each
>>> time) and write it across the network to another location. So if 
>>> everytime it adds an additional 3 fd's then after some time I'm 
>>> going
> 
>>> to run out of fd's
>> It should not add 3 fds every time. If you do see a practical case 
>> where you trace running out of fds to these three fds, please let us
> know.
>> Raghu.
>>

Mime
View raw message