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

Jeff Parsons commented on FLUME-2375:
-------------------------------------

We've encountered a similar problem. Here's the 'full' stack trace - this just 
loops indefinitely until we restart the Flume agent. This Flume 1.4.0 
(cdh5.0.1+113-1.cdh5.0.1.p0.25.el6) on CentOS 6.5. We believe the issue is due 
to intermittent network saturation. We are working to migrate the datanodes to 
an isolated network in the near future, but it would be great if Flume could 
reliable recover. We are using a memory channel in this context.

{noformat}
07 Apr 2015 20:42:00,905 ERROR [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.AbstractHDFSWriter.isUnderReplicated:82)  - 
Unexpected error while checking replication factor
java.lang.reflect.InvocationTargetException
        at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.apache.flume.sink.hdfs.AbstractHDFSWriter.getNumCurrentReplicas(AbstractHDFSWriter.java:147)
        at 
org.apache.flume.sink.hdfs.AbstractHDFSWriter.isUnderReplicated(AbstractHDFSWriter.java:68)
        at 
org.apache.flume.sink.hdfs.BucketWriter.shouldRotate(BucketWriter.java:505)
        at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:440)
        at 
org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:401)
        at 
org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:68)
        at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
        at java.lang.Thread.run(Thread.java:744)
Caused by: java.net.SocketTimeoutException: 70000 millis timeout while waiting 
for channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/flumeHost:47580 
remote=/datanode-05:50010]
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118)
        at java.io.FilterInputStream.read(FilterInputStream.java:83)
        at java.io.FilterInputStream.read(FilterInputStream.java:83)
        at 
org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:1985)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.transfer(DFSOutputStream.java:1063)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:1031)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:1174)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.processDatanodeError(DFSOutputStream.java:924)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:486)
07 Apr 2015 20:42:00,907 WARN  [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.BucketWriter.append:477)  - Caught IOException 
writing to HDFSWriter (70000 millis timeout while waiting for channel to be 
ready for read. ch : java.nio.channels.SocketChannel[connected 
local=/flumeHost:47580 remote=/datanode-05:50010]). Closing file 
(hdfs://namenode:8020/events/categoryname/2015-04-07/partial_FlumeData_flumeHost.1428437342790.tmp)
 and rethrowing exception.
07 Apr 2015 20:42:00,907 WARN  [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.BucketWriter.append:483)  - Caught IOException 
while closing file 
(hdfs://namenode:8020/events/categoryname/2015-04-07/partial_FlumeData_flumeHost.1428437342790.tmp).
 Exception follows.
java.net.SocketTimeoutException: 70000 millis timeout while waiting for channel 
to be ready for read. ch : java.nio.channels.SocketChannel[connected 
local=/flumeHost:47580 remote=/datanode-05:50010]
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118)
        at java.io.FilterInputStream.read(FilterInputStream.java:83)
        at java.io.FilterInputStream.read(FilterInputStream.java:83)
        at 
org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:1985)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.transfer(DFSOutputStream.java:1063)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:1031)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:1174)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.processDatanodeError(DFSOutputStream.java:924)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:486)
{noformat}



> HDFS sink's fail to recover from datanode unavailability
> --------------------------------------------------------
>
>                 Key: FLUME-2375
>                 URL: https://issues.apache.org/jira/browse/FLUME-2375
>             Project: Flume
>          Issue Type: Bug
>    Affects Versions: v1.4.0
>            Reporter: David Stendardi
>              Labels: hdfs, hdfssink
>
> Hello !
> We are running flume-ng with version cdh-4.5-1.4. When a datanode used by 
> flume-ng goes done, we get the following exceptions :  
> {code}
> 30 Apr 2014 01:10:38,130 ERROR 
> [SinkRunner-PollingRunner-DefaultSinkProcessor] 
> (org.apache.flume.sink.hdfs.AbstractHDFSWriter.isUnderReplicated:96)  - 
> Unexpected error while checking replication factor
> java.lang.reflect.InvocationTargetException
>         at sun.reflect.GeneratedMethodAccessor40.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at 
> org.apache.flume.sink.hdfs.AbstractHDFSWriter.getNumCurrentReplicas(AbstractHDFSWriter.java:162)
>         at 
> org.apache.flume.sink.hdfs.AbstractHDFSWriter.isUnderReplicated(AbstractHDFSWriter.java:82)
>         at 
> org.apache.flume.sink.hdfs.BucketWriter.shouldRotate(BucketWriter.java:452)
>         at 
> org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:387)
>         at 
> org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:392)
>         at 
> org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:68)
>         at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>         at java.lang.Thread.run(Thread.java:662)
> {code}
> These exceptions are logged but not rethrown, and the 
> AbstractHdfsSink::isUnderReplicated still returns false so the writer 
> continue to try writing on the node.
> Here is how we configured our sink : 
> {code}
> collector.sinks.hdfs.channel = hdfs
> collector.sinks.hdfs.type = hdfs
> collector.sinks.hdfs.hdfs.path = 
> /flume-ng/%{env}/%{avro.fqn}/from_year=%Y/from_date=%Y-%m-%d
> collector.sinks.hdfs.hdfs.filePrefix = <%= @hostname %>-%H-%{avro.fp}
> collector.sinks.hdfs.hdfs.fileSuffix = .avro
> collector.sinks.hdfs.hdfs.rollInterval = 3605
> collector.sinks.hdfs.hdfs.rollSize = 0
> collector.sinks.hdfs.hdfs.rollCount = 0
> collector.sinks.hdfs.hdfs.batchSize = 1000
> collector.sinks.hdfs.hdfs.txnEventMax = 1000
> collector.sinks.hdfs.hdfs.callTimeout = 20000
> collector.sinks.hdfs.hdfs.fileType = DataStream
> collector.sinks.hdfs.serializer = 
> com.viadeo.event.flume.serializer.AvroEventSerializer$Builder
> {code}



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

Reply via email to