|
|
-
Re: At flume shutdown only one HDFSSink is closed properlyMike Percy 2012-08-24, 01:35
Hmm... this likely happens because Hadoop statically caches the FileSystem
object, so as it turns out, the multiple Sinks are sharing the same FileSystem objects. I think the only reason we need to explicitly close the FileSystem objects is to support the deleteOnExit feature. We are explicitly closing them because we removed the automatic shutdown hook typically installed by Hadoop to invoke FileSystem.close(), since it was interfering with the .tmp rolling. I wonder if we can get away with never closing them in our case... I'm not sure if we need the deleteOnExit() functionality implicitly for any reason, or if there are other more important reasons behind why the FileSystem objects should be closed. Regards, Mike On Thu, Aug 23, 2012 at 3:24 PM, Bhaskar V. Karambelkar <[EMAIL PROTECTED] > wrote: > I have 3 HDFS sinks all writing to the same namenode, but different paths. > > e.g. sink1 = hdfs://namenode/path1 > sink2 = hdfs://namenode/path2 > etc. > > When flume is shutdown (kill <flume-pid>), the file for the first sink is > closed correctly and renamed to remove the .tmp extension > but the second file's closing throws the following exception and the > file's .tmp extension is also not removed. > I see this happening very consistently, for 1+ HDFS sinks, only the first > one is closed properly and renamed, the rest all throw exception > when being closed, and are not renamed to remove the .tmp extension. > > 2012-08-23 19:51:39,837 WARN hdfs.BucketWriter: failed to close() > HDFSWriter for file > (hdfs://hadoop-namnode:9000/flume/avro/2012/08/23/l/avro_.1345751470750.tmp). > Exception follows. > java.io.IOException: Filesystem closed > at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:264) > at org.apache.hadoop.hdfs.DFSClient.access$1100(DFSClient.java:74) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3667) > at > org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) > at > org.apache.flume.sink.hdfs.HDFSDataStream.close(HDFSDataStream.java:103) > at > org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:250) > at > org.apache.flume.sink.hdfs.BucketWriter.access$400(BucketWriter.java:48) > at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:236) > at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:233) > at > org.apache.flume.sink.hdfs.BucketWriter.runPrivileged(BucketWriter.java:125) > at org.apache.flume.sink.hdfs.BucketWriter.close(BucketWriter.java:233) > at > org.apache.flume.sink.hdfs.HDFSEventSink$3.call(HDFSEventSink.java:747) > at > org.apache.flume.sink.hdfs.HDFSEventSink$3.call(HDFSEventSink.java:744) > at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) > at java.util.concurrent.FutureTask.run(FutureTask.java:138) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:662) > 2012-08-23 19:51:39,840 WARN hdfs.HDFSEventSink: Exception while closing > hdfs://hadoop-namenode:9000/flume/avro/2012/08/23/avro_. Exception follows. > java.io.IOException: Filesystem closed > at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:264) > at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:873) > at > org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:513) > at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:768) > at > org.apache.flume.sink.hdfs.BucketWriter.renameBucket(BucketWriter.java:369) > at > org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:269) > at > org.apache.flume.sink.hdfs.BucketWriter.access$400(BucketWriter.java:48) > at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:236) > at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:233) |