flume-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Suhas Satish <suhas.sat...@gmail.com>
Subject Re: flume agent with HDFS sink, syslog source and memory channel - stuck on hdfs IOException
Date Thu, 17 Oct 2013 00:15:35 GMT
There already exists a JIRA. I have come up with a local fix which works.
https://issues.apache.org/jira/browse/FLUME-1654

Will be uploading a patch soon.

Cheers,
Suhas.


On Tue, Oct 15, 2013 at 1:15 PM, Roshan Naik <roshan@hortonworks.com> wrote:

> Paul,
>    HDFS sink issue apart... it sounds like this is a setup where  Hive  s
> being allowed to read through new files/directories flowing into the
> partition while HDFS sink is still writing to it. To my knowledge, in Hive,
> a partition is considered immutable and it should not be updated once the
> partition is created. So only once the HDFS sink has rolled over to the
> next directory, the previous directory should be exposed to Hive.
> -roshan
>
>
> On Tue, Oct 15, 2013 at 11:23 AM, Paul Chavez <
> pchavez@verticalsearchworks.com> wrote:
>
>> I can’t speak for Suhas, but I face a similar issue in production. For me
>> it occurs when someone queries a .tmp file from Hive or Pig. This causes
>> the HDFS sink to lose the ability to close and rename the file and then the
>> HDFS sink is completely out of commission until the agent is restarted.
>> We’ve mitigated this in our environment by careful Hive partition
>> coordination but it still crops up in cases where people are running ad-hoc
>> queries they probably shouldn’t be. We are waiting to get the latest CDH in
>> production which eliminates the .tmp file issue but I would still like to
>> have a more resilient HDFS sink and so I support development effort in this
>> area.****
>>
>> ** **
>>
>> Thanks,****
>>
>> Paul Chavez****
>>
>> ** **
>>
>> ** **
>>
>> *From:* Roshan Naik [mailto:roshan@hortonworks.com]
>> *Sent:* Tuesday, October 15, 2013 11:14 AM
>> *To:* dev@flume.apache.org
>> *Cc:* user@flume.apache.org; commits@flume.apache.org
>> *Subject:* Re: flume agent with HDFS sink, syslog source and memory
>> channel - stuck on hdfs IOException****
>>
>> ** **
>>
>> sounds like a valid bug. i am curious though... is there a use real use
>> scenario you are facing in production ?****
>>
>> ** **
>>
>> On Mon, Oct 14, 2013 at 7:39 PM, Suhas Satish <suhas.satish@gmail.com>
>> wrote:****
>>
>> In summary, although the flume-agent JVM doesnt exit, once a HDFS IO
>> exception
>> occurs due to deleting a .tmp file, the agent doesn't recover from this
>> to log
>> other hdfs sink outputs generated by syslog source.
>>
>> There was only 1 JIRA remotely related to this HDFS sink issue I found in
>> Apache which we didn't have. I tested by pulling-in jira patch
>> FLUME-2007 into flume-1.4.0.
>>
>> https://github.com/apache/flume/commit/5b5470bd5d3e94842032009c36788d4ae346674bhttps://issues.apache.org/jira/browse/FLUME-2007<https://github.com/apache/flume/commit/5b5470bd5d3e94842032009c36788d4ae346674bhttps:/issues.apache.org/jira/browse/FLUME-2007>
>>
>> But it doesn't solve this issue.
>>
>> Should I open a new jira ticket?
>>
>>
>>
>> Thanks,
>> Suhas.
>>
>>
>> On Fri, Oct 11, 2013 at 4:13 PM, Suhas Satish <suhas.satish@gmail.com
>> >wrote:****
>>
>>
>> > Hi I have the  following flume configuration file   flume-syslog.conf
>> > (attached) -
>> >
>> > 1.) I laucnh it with -
>> >
>> > bin/flume-ng agent -n agent -c conf -f conf/flume-syslog.conf
>> >
>> > 2.) Generate log output using loggen (provided by syslog-ng):
>> > loggen -I 30 -s 300 -r 900 localhost 13073
>> >
>> > 3.) I verify flume output is generated under /flume_import/ on hadoop
>> cluster.
>> >
>> > It generates output of the form -
>> >
>> > -rwxr-xr-x   3 root root     139235 2013-10-11 14:35
>> > /flume_import/2013/10/14/logdata-2013-10-14-35-45.1381527345384.tmp
>> > -rwxr-xr-x   3 root root     138095 2013-10-11 14:35
>> > /flume_import/2013/10/14/logdata-2013-10-14-35-46.1381527346543.tmp
>> > -rwxr-xr-x   3 root root     135795 2013-10-11 14:35
>> > /flume_import/2013/10/14/logdata-2013-10-14-35-47.1381527347670.tmp
>> >
>> >
>> > 4.)  Delete the flume output files while loggen is still running and
>> Flume is
>> > generating the sink output.
>> >
>> > hadoop fs -rmr
>> /flume_import/2013/10/14/logdata-2013-10-14-35-47.1381527347670.tmp
>> >
>> > 5. )This  gives me the following exception in the flume log. Although
>> the flume agent JVM continues to run, it does not generate any more output
>> files from syslog-ng  until the flume agent JVM is restarted. Is flume
>>  expected to behave like this or  should it handle IOException gracefully
>> and continue to log output of syslog to other output directories?
>> >
>> > 10 Oct 2013 16:55:42,092 WARN
>>  [SinkRunner-PollingRunner-DefaultSinkProcessor]
>> > (org.apache.flume.sink.hdfs.BucketWriter.append:430)  - Caught
>> IOException
>> > while closing file
>> >
>> (maprfs:///flume_import/2013/10/16//logdata-2013-10-16-50-03.1381449008596.tmp).
>> > Exception follows.
>> > java.io.IOException: 2049.112.5249612
>> > /flume_import/2013/10/16/logdata-2013-10-16-50-03.1381449008596.tmp
>> (Stale file
>> > handle)
>> >     at com.mapr.fs.Inode.throwIfFailed(Inode.java:269)
>> >     at com.mapr.fs.Inode.flushJniBuffers(Inode.java:402)
>> >     at com.mapr.fs.Inode.syncInternal(Inode.java:478)
>> >     at com.mapr.fs.Inode.syncUpto(Inode.java:484)
>> >     at com.mapr.fs.MapRFsOutStream.sync(MapRFsOutStream.java:244)
>> >     at
>> com.mapr.fs.MapRFsDataOutputStream.sync(MapRFsDataOutputStream.java:68)
>> >     at
>> org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:946)
>> >     at
>> >
>> org.apache.flume.sink.hdfs.HDFSSequenceFile.sync(HDFSSequenceFile.java:107)
>> >     at
>> org.apache.flume.sink.hdfs.BucketWriter$5.call(BucketWriter.java:356)
>> >     at
>> org.apache.flume.sink.hdfs.BucketWriter$5.call(BucketWriter.java:353)
>> >     at
>> org.apache.flume.sink.hdfs.BucketWriter$8$1.run(BucketWriter.java:536)
>> >     at
>> >
>> org.apache.flume.sink.hdfs.BucketWriter.runPrivileged(BucketWriter.java:160)
>> >     at
>> >
>> org.apache.flume.sink.hdfs.BucketWriter.access$1000(BucketWriter.java:56)
>> >     at
>> org.apache.flume.sink.hdfs.BucketWriter$8.call(BucketWriter.java:533)
>> >     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(T
>> >
>> >
>> > 6.) I found the following related post
>> >
>> >
>> http://mail-archives.apache.org/mod_mbox/flume-user/201305.mbox/%3C24FEFF6B2CA048F7A4A7D9D6E30084FB@cloudera.com%3E
>> >
>> > Not sure if its related to this issue. Can anyone comment?
>> >
>> > Thanks,
>> > Suhas.
>> >****
>>
>> ** **
>>
>>
>> CONFIDENTIALITY NOTICE
>> NOTICE: This message is intended for the use of the individual or entity
>> to which it is addressed and may contain information that is confidential,
>> privileged and exempt from disclosure under applicable law. If the reader
>> of this message is not the intended recipient, you are hereby notified that
>> any printing, copying, dissemination, distribution, disclosure or
>> forwarding of this communication is strictly prohibited. If you have
>> received this communication in error, please contact the sender immediately
>> and delete it from your system. Thank You.****
>>
>
>
> CONFIDENTIALITY NOTICE
> NOTICE: This message is intended for the use of the individual or entity
> to which it is addressed and may contain information that is confidential,
> privileged and exempt from disclosure under applicable law. If the reader
> of this message is not the intended recipient, you are hereby notified that
> any printing, copying, dissemination, distribution, disclosure or
> forwarding of this communication is strictly prohibited. If you have
> received this communication in error, please contact the sender immediately
> and delete it from your system. Thank You.
>

Mime
View raw message