flume-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Nishant Neeraj <nishant.has.a.quest...@gmail.com>
Subject Re: HDFS Sink keeps .tmp files and closes with exception
Date Sat, 20 Oct 2012 03:31:21 GMT
Thanks, Bhaskar, Hari. I will give it a try.

appreciate your help,
Nishant

On Sat, Oct 20, 2012 at 5:43 AM, Hari Shreedharan <hshreedharan@cloudera.com
> wrote:

>  Thanks Bhaskar for the details. Regarding this issue, agreed HADOOP-4829
> is needed to fix it. I was just remarking on this configuration which
> seemed to never roll the files. Either way, yes HADOOP-4829 is required,
> else ".tmp" files are left around. But these files should be closed by
> the name node automatically later, though the ".tmp" extension will not be
> removed - that will need to be removed manually.
>
> Nishant - In this case, I agree with Bhaskar's advise. Also please make
> sure you close your files using one of the criteria in the flume
> configuration - else your file will be closed and renamed only when Flume
> is shutdown.
>
>
> Thanks,
> Hari
>
> --
> Hari Shreedharan
>
> On Friday, October 19, 2012 at 4:42 PM, Bhaskar V. Karambelkar wrote:
>
> Hari,
> There are 2 sides to this problem,
> The flume side issue was fixed in 1.2.0, FLUME-1163.
> But the solution in FLUME-1163, requires that the Hadoop Libraries
> used with flume, support the 'fs.automatic.shutdown' flag.
> Unfortunately hadoop 1.x branch, does not have that feature pulled in
> https://issues.apache.org/jira/browse/HADOOP-4829
>
> So if you use flume with hadoop libs from 1.x series, you'll still get
> the .tmp files leftover, and the exception mentioned.
>
> CDH3u5, does indeed pull in HADOOP-4829, so flume + CDH3u5 works
> correctly and the files get properly closed at flume shutdown.
>
> Do note, that this is a Hadoop Client side problem, so on your hadoop
> cluster you can use 1.x version, and connect to it using flume +
> CDH3u5 combo, and that should work as long as CDH3u5 libs are able to
> talk with Hadoop 1.x cluster. But usually hadoop even on client side
> is very picky about versions.
>
> Here's a mail thread, where I discussed this before.
>
> http://mail-archives.apache.org/mod_mbox/flume-user/201208.mbox/%3CCAJLbxRad_DUuz3NrPBQo4gtqAdq%2BCLBUvD2_q7-0%2B-L8Afax_A%40mail.gmail.com%3E
>
> I can say with certainty, that if you use HDFS 1.x client side, the
> .tmp files will not be renamed at shutdown, and you will get
> exceptions as mentioned in the original mail.
>
> thanks
> Bhaskar
>
>
>
> On Fri, Oct 19, 2012 at 4:37 PM, Hari Shreedharan
> <hshreedharan@cloudera.com> wrote:
>
> Nishant,
>
> a: if CDH4 was working for you, you could use it with hadoop-2.x or CDH3u5
> with hadoop-1.x.
> b: Looks like your rollSize/rollCount/rollInterval are all 0. Can you
> increase rollCount to say 1000 or so? If you see here:
> http://flume.apache.org/FlumeUserGuide.html#hdfs-sink, if you set the
> roll*
> configuration params to 0, they would never roll the files. If files are
> not
> rolled, they are not closed, and HDFS will show those as 0-sized files.
> Once
> the roll happens, HDFS GUI will show you the real file size. You can use
> any
> one of the three roll* config parameters to roll the files.
>
>
>
> Thanks,
> Hari
>
>
> --
> Hari Shreedharan
>
> On Friday, October 19, 2012 at 1:29 PM, Nishant Neeraj wrote:
>
> Thanks for the responses.
>
> a: Got rid of all the CDH stuffs. (basically, started on a fresh AWS
> instance)
> b: Installed from binary files.
>
> It DID NOT work. Here is what I observed:
> flume-ng version: Flume 1.2.0
> Hadoop: 1.0.4
>
> This is what my configuration is:
>
> agent1.sinks.fileSink1.type = hdfs
> agent1.sinks.fileSink1.channel = memChannel1
> agent1.sinks.fileSink1.hdfs.path =
> hdfs://localhost:54310/flume/agg1/%y-%m-%d
> agent1.sinks.fileSink1.hdfs.filePrefix = agg2
> agent1.sinks.fileSink1.hdfs.rollInterval = 0
> agent1.sinks.fileSink1.hdfs.rollSize = 0
> agent1.sinks.fileSink1.hdfs.rollCount = 0
> agent1.sinks.fileSink1.hdfs.fileType = DataStream
> agent1.sinks.fileSink1.hdfs.writeFormat = Text
> #agent1.sinks.fileSink1.hdfs.batchSize = 10
>
> #1: startup error
> -----------------------------------
> With new intallation, I start to find this exception on start of Flume (it
> does not stop me from adding data to hdfs)
>
> 2012-10-19 19:48:32,191 (conf-file-poller-0) [INFO -
>
> org.apache.flume.sink.DefaultSinkFactory.create(DefaultSinkFactory.java:70)]
> Creating instance of sink: fileSink1, type: hdfs
> 2012-10-19 19:48:32,296 (conf-file-poller-0) [DEBUG -
> org.apache.hadoop.conf.Configuration.<init>(Configuration.java:227)]
> java.io.IOException: config()
> at org.apache.hadoop.conf.Configuration.<init>(Configuration.java:227)
> at org.apache.hadoop.conf.Configuration.<init>(Configuration.java:214)
> at
>
> org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:184)
> at
>
> org.apache.hadoop.security.UserGroupInformation.isSecurityEnabled(UserGroupInformation.java:236)
> at
>
> org.apache.flume.sink.hdfs.HDFSEventSink.authenticate(HDFSEventSink.java:516)
> at
> org.apache.flume.sink.hdfs.HDFSEventSink.configure(HDFSEventSink.java:238)
> at org.apache.flume.conf.Configurables.configure(Configurables.java:41)
> at
>
> org.apache.flume.conf.properties.PropertiesFileConfigurationProvider.loadSinks
> (PropertiesFileConfigurationProvider.java:373)
> at
> org.apache.flume.conf.properties.PropertiesFileConfigurationProvider.load
> (PropertiesFileConfigurationProvider.java:223)
> at
>
> org.apache.flume.conf.file.AbstractFileConfigurationProvider.doLoad(AbstractFileConfigurationProvider.java:123)
> at
>
> org.apache.flume.conf.file.AbstractFileConfigurationProvider.access$300(AbstractFileConfigurationProvider.java:38)
> at
>
> org.apache.flume.conf.file.AbstractFileConfigurationProvider$FileWatcherRunnable.run
> (AbstractFileConfigurationProvider.java:202)
> -- snip --
> #2: the old issue continues
> ------------------------------------
> When I start loading source, I see console shows that events gets
> generated.
> But HDFS GUI shows 0KB file with .tmp extention. Adding hdfs.batchSize has
> no effect, I would assume this should have flushed the content to the temp
> file. But no. I tried with smaller and bigger values of hdfs.batchSize, no
> effect.
>
> When I shutdown Flume, I see the data gets purged to the temp file. BUT the
> temp file is still holding the .tmp extention. So, basically NO WAY TO HAVE
> ONE SINGLE AGGRAGATED FILE of all the logs. If I make the rollSize setting
> to positive, things start to work, but forfeits the purpose.
>
> Evenwith roll non-zero value, the last file stays as .tmp when I close
> Flume
>
> #3: Shutdown throws exception
> ------------------------------------
> Closing flume ends with this excpetion, (the data in the file looks OK,
> though)
>
> 2012-10-19 20:07:55,543 (hdfs-fileSink1-call-runner-7) [DEBUG -
> org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:247)]
> Closing hdfs://localhost:54310/flume/agg1/12-10-19/agg2.1350676790623.tmp
> 2012-10-19 20:07:55,543 (hdfs-fileSink1-call-runner-7) [WARN -
> org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:253)]
> failed to close() HDFSWriter for file
> (hdfs://localhost:54310/flume/agg1/12-10-19/agg2.1350676790623.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)
> -- snip --
>
> Couple of side notes:
>
> #1: For weird reasons, I did not have to prefix hdfs://localhost:54310 in
> my
> previous config (the one using CDH4 version) and thing were as good as in
> this installation except there was not many exceptions.
>
> #2: I have
> java version "1.6.0_24"
> OpenJDK Runtime Environment (IcedTea6 1.11.4)
> (6b24-1.11.4-1ubuntu0.12.04.1)
> OpenJDK 64-Bit Server VM (build 20.0-b12, mixed mode)
> #3: I did not create a special hadoop:hduser this time. Just dumped the
> file
> in $HOME, changed config files: *-site.xml , -env.sh, flume.sh. And
> exported
> appropriate variables.
>
> #4. here is what my config files look like:
>
> <!-- core-site.xml -->
> <configuration>
> <property>
> <name>hadoop.tmp.dir</name>
> <value>/home/ubuntu/hadoop/tmp</value>
> <description>A base for other temporary directories.</description>
> </property>
>
> <property>
> <name>fs.default.name</name>
> <value>hdfs://localhost:54310</value>
> </property>
> </configuration>
>
> <!-- hdfs-site.xml -->
> <configuration>
> <property>
> <name>dfs.replication</name>
> <value>1</value>
> </property>
> </configuration>
>
> <!-- mapred-site.xml -->
>
> <configuration>
> <property>
> <name>mapred.job.tracker</name>
> <value>localhost:54311</value>
> </property>
> </configuration>
>
> #5: /home/ubuntu/hadoop/tmp has chmod 777 (tried 750 as well)
>
> thanks for your time
> - Nishant
>
> On Fri, Oct 19, 2012 at 4:30 AM, Hari Shreedharan
> <hshreedharan@cloudera.com> wrote:
>
> Nishant,
>
> CDH4+ Flume is built against Hadoop-2, and may not work correctly against
> Hadoop-1.x, since Hadoop's interfaces changed in the mean time. You could
> also use Apache Flume-1.2.0 or the upcoming Apache Flume-1.3.0 directly
> against Hadoop-1.x without issues, as they are built against Hadoop-1.x.
>
>
> Thanks,
> Hari
>
> --
> Hari Shreedharan
>
> On Thursday, October 18, 2012 at 1:18 PM, Nishant Neeraj wrote:
>
> I am working on a POC using
>
> flume-ng version Flume 1.2.0-cdh4.1.1
> Hadoop 1.0.4
>
> The config looks like this
>
> #Flume agent configuration
> agent1.sources = avroSource1
> agent1.sinks = fileSink1
> agent1.channels = memChannel1
>
> agent1.sources.avroSource1.type = avro
> agent1.sources.avroSource1.channels = memChannel1
> agent1.sources.avroSource1.bind = 0.0.0.0
> agent1.sources.avroSource1.port = 4545
>
> agent1.sources.avroSource1.interceptors = b
> agent1.sources.avroSource1.interceptors.b.type =
> org.apache.flume.interceptor.TimestampInterceptor$Builder
>
> agent1.sinks.fileSink1.type = hdfs
> agent1.sinks.fileSink1.channel = memChannel1
> agent1.sinks.fileSink1.hdfs.path = /flume/agg1/%y-%m-%d
> agent1.sinks.fileSink1.hdfs.filePrefix = agg
> agent1.sinks.fileSink1.hdfs.rollInterval = 0
> agent1.sinks.fileSink1.hdfs.rollSize = 0
> agent1.sinks.fileSink1.hdfs.rollCount = 0
> agent1.sinks.fileSink1.hdfs.fileType = DataStream
> agent1.sinks.fileSink1.hdfs.writeFormat = Text
>
>
> agent1.channels.memChannel1.type = memory
> agent1.channels.memChannel1.capacity = 1000
> agent1.channels.memChannel1.transactionCapacity = 1000
>
>
> Basically, I do not want to roll the file at all. I am just wanting to tail
> and watch the show from Hadoop UI. The problem is it does not work. The
> console keeps saying,
>
> agg.1350590350462.tmp 0 KB 2012-10-18 19:59
>
> Flume console shows events getting pushes. When I stop the flume, I see the
> file gets populated, but the '.tmp' is still in the file name. And I see
> this exception on close.
>
> 2012-10-18 20:06:49,315 (hdfs-fileSink1-call-runner-8) [DEBUG -
> org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:254)]
> Closing /flume/agg1/12-10-18/agg.1350590350462.tmp
> 2012-10-18 20:06:49,316 (hdfs-fileSink1-call-runner-8) [WARN -
> org.apache.flume.sink.hdfs.BucketWriter.doClose(BucketWriter.java:260)]
> failed to close() HDFSWriter for file
> (/flume/agg1/12-10-18/agg.1350590350462.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:257)
> at org.apache.flume.sink.hdfs.BucketWriter.access$400(BucketWriter.java:50)
> at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:243)
> at org.apache.flume.sink.hdfs.BucketWriter$3.run(BucketWriter.java:240)
> at
>
> org.apache.flume.sink.hdfs.BucketWriter.runPrivileged(BucketWriter.java:127)
> at org.apache.flume.sink.hdfs.BucketWriter.close(BucketWriter.java:240)
> at org.apache.flume.sink.hdfs.HDFSEventSink$3.call(HDFSEventSink.java:748)
> at org.apache.flume.sink.hdfs.HDFSEventSink$3.call(HDFSEventSink.java:745)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at
>
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:679)
>
>
> Thanks
> Nishant
>
>
>

Mime
View raw message