flume-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Nazar Volynets (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (FLUME-2584) Need graceful shutdown of HDFSEventSink
Date Tue, 28 Aug 2018 14:50:00 GMT

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

Nazar Volynets edited comment on FLUME-2584 at 8/28/18 2:49 PM:
----------------------------------------------------------------

Hi All,

Bumped with the same issue. And can confirm that there is no _data loss_.
But this issue is a root cause of *{color:red}data duplication{color}*. At least can confirm
(verified) that for the following scenarios:
{code}
#1
Kafka source --> File chanell --> HDFS sink
#2
Kafka chanell --> HDFS sink 
{code}
Based on scenario #1 it means that there is data _duplication_ for all possible use cases
where HDFS sink is used in a Flume's flow.

Moreover this _data duplication_ is *significant* in the following use cases:
* Flume agent is under high load
* Multiple Flume agents are shipping data into HDFS

Basically issue is here:
https://github.com/apache/flume/blob/release-1.8.0/flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java#L100

Please increase priority of this issue.

Thanks.


was (Author: nvolynets):
Hi All,

Bumped with the same issue. And can confirm that there is no _data loss_.
But this issue is a root cause of *{color:red}data duplication{color}*. At least can confirm
(verified) that for the following scenarios:
{code}
#1
Kafka source --> File chanell --> HDFS sink
#2
Kafka chanell --> HDFS sink 
{code}
Based on scenario #1 it means that there is data _duplication_ for all possible use cases
where HDFS sink is used in a Flume's flow.

Moreover this _data duplication_ is *significant* in the following use cases:
* Flume agent is under high load
* Multiple Flume agents are shipping data into HDFS

Please increase priority of this issue.

Thanks.

> Need graceful shutdown of HDFSEventSink
> ---------------------------------------
>
>                 Key: FLUME-2584
>                 URL: https://issues.apache.org/jira/browse/FLUME-2584
>             Project: Flume
>          Issue Type: Bug
>          Components: Sinks+Sources
>    Affects Versions: 1.5.1
>         Environment: Tested with Flume 1.5.0 and 1.5.2 on Redhat 6.
>            Reporter: Sverre Bakke
>            Priority: Minor
>
> When pressing ctrl+c (or otherwise signal to kill the flume process), the HDFS sink seems
to have issues dealing with interruption. 100% of the times I do this I get the following
errors back:
> 2015-01-05 11:16:52,448 (SinkRunner-PollingRunner-DefaultSinkProcessor) [WARN - org.apache.flume.sink.hdfs.BucketWriter.callWithTimeout(BucketWriter.java:752)]
      Unexpected Exception null
> java.lang.InterruptedException
>         at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:400)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:199)
>         at org.apache.flume.sink.hdfs.BucketWriter.callWithTimeout(BucketWriter.java:725)
>         at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:262)
>         at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:554)
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:426)
>         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:745)
> 2015-01-05 11:16:52,451 (SinkRunner-PollingRunner-DefaultSinkProcessor) [ERROR - org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:467)]
proc      ess failed
> java.lang.RuntimeException: java.lang.InterruptedException
>         at com.google.common.base.Throwables.propagate(Throwables.java:156)
>         at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:291)
>         at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:554)
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:426)
>         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:745)
> Caused by: java.lang.InterruptedException
>         at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:400)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:199)
>         at org.apache.flume.sink.hdfs.BucketWriter.callWithTimeout(BucketWriter.java:725)
>         at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:262)
>         ... 5 more
> 2015-01-05 11:16:52,452 (SinkRunner-PollingRunner-DefaultSinkProcessor) [ERROR - org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:160)]
Unable to       deliver event. Exception follows.
> org.apache.flume.EventDeliveryException: java.lang.RuntimeException: java.lang.InterruptedException
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:471)
>         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:745)
> Caused by: java.lang.RuntimeException: java.lang.InterruptedException
>         at com.google.common.base.Throwables.propagate(Throwables.java:156)
>         at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:291)
>         at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:554)
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:426)
>         ... 3 more
> Caused by: java.lang.InterruptedException
>         at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:400)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:199)
>         at org.apache.flume.sink.hdfs.BucketWriter.callWithTimeout(BucketWriter.java:725)
>         at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:262)
>         ... 5 more
> or
> 2015-01-05 11:20:34,706 (agent-shutdown-hook) [DEBUG - org.apache.flume.SinkRunner.stop(SinkRunner.java:104)]
Waiting for runner thread to exit
> 2015-01-05 11:20:34,709 (SinkRunner-PollingRunner-DefaultSinkProcessor) [ERROR - org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:467)]
process failed
> java.lang.InterruptedException: Timed out before HDFS call was made. Your hdfs.callTimeout
might be set too low or HDFS calls are taking too long.
>         at org.apache.flume.sink.hdfs.BucketWriter.checkAndThrowInterruptedException(BucketWriter.java:699)
>         at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:523)
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:426)
>         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:745)
> 2015-01-05 11:20:34,711 (SinkRunner-PollingRunner-DefaultSinkProcessor) [ERROR - org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:160)]
Unable to deliver event. Exception follows.
> org.apache.flume.EventDeliveryException: java.lang.InterruptedException: Timed out before
HDFS call was made. Your hdfs.callTimeout might be set too low or HDFS calls are taking too
long.
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:471)
>         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:745)
> Caused by: java.lang.InterruptedException: Timed out before HDFS call was made. Your
hdfs.callTimeout might be set too low or HDFS calls are taking too long.
>         at org.apache.flume.sink.hdfs.BucketWriter.checkAndThrowInterruptedException(BucketWriter.java:699)
>         at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:523)
>         at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:426)
>         ... 3 more



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@flume.apache.org
For additional commands, e-mail: issues-help@flume.apache.org


Mime
View raw message