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

[email protected] commented on FLUME-798:
-----------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/2869/#review3493
-----------------------------------------------------------

Ship it!


Hey Prasad,

I'm a little concerned about the possibility of having a deadlock when you 
disabling a slow/blocked append, but this an improvement because if gives you 
the option to chooses one behavior or the other.

Please fix the typos and commit.

Thanks!
Jon.


flume-core/src/main/java/com/cloudera/flume/handlers/rolling/RollSink.java
<https://reviews.apache.org/r/2869/#comment7774>

    spelling? rollCanceledAppends 
    
    hm.. I apparently dictionary says cancelled and canceled  are both ok..



flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestSlowSinkRoll.java
<https://reviews.apache.org/r/2869/#comment7773>

    typo? gots



flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestSlowSinkRoll.java
<https://reviews.apache.org/r/2869/#comment7772>

    typo? testSlowSinkdRoll (extra d?)



flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestSlowSinkRoll.java
<https://reviews.apache.org/r/2869/#comment7771>

    Typo? testWaitingSlowSinkdRoll (extra d?)


- jmhsieh


On 2011-11-23 00:43:30, Prasad Mujumdar wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/2869/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-11-23 00:43:30)
bq.  
bq.  
bq.  Review request for jmhsieh and Eric Sammer.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  If the append takes longer than a second, then the roll-trigger thread 
aborts the append. This results into an interrupted exception that is not 
handled gracefully and hence causes a nullpointer exception in the pumper 
thread.
bq.  
bq.  Change the interrupted exception to Runtime so that it can be handled by 
direct driver. There were a couple of other related issues that are also fixed 
in the roll sink.
bq.  Also it allows the wait time for trigger thread can be configured via 
flume.collector.roll.timeout property in flume-conf.xml. 
bq.  
bq.  
bq.  This addresses bug Flume-798.
bq.      https://issues.apache.org/jira/browse/Flume-798
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    flume-core/src/main/java/com/cloudera/flume/conf/FlumeConfiguration.java 
397dfef 
bq.    
flume-core/src/main/java/com/cloudera/flume/handlers/rolling/RollSink.java 
27302b1 
bq.    
flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestRollRollTags.java
 01d6574 
bq.    
flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestRollSink.java 
761643d 
bq.    
flume-core/src/test/java/com/cloudera/flume/handlers/rolling/TestSlowSinkRoll.java
 PRE-CREATION 
bq.  
bq.  Diff: https://reviews.apache.org/r/2869/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Added new test TestSlowSinkRoll. will run full regression test.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Prasad
bq.  
bq.


                
> Blocked append interrupted by rotation event 
> ---------------------------------------------
>
>                 Key: FLUME-798
>                 URL: https://issues.apache.org/jira/browse/FLUME-798
>             Project: Flume
>          Issue Type: Bug
>          Components: Node
>    Affects Versions: v0.9.5
>            Reporter: Cameron Gandevia
>            Assignee: Prasad Mujumdar
>         Attachments: 
> 0001-FLUME-798-Modified-RollSink-to-not-cancel-pending-si.patch, 
> 0001-FLUME-798-Modified-RollSink-to-not-cancel-pending-si.patch, 
> Flume-798.patch
>
>
> Our flume collector seem's to work for a short period of time and then fails 
> with the following exception. When this happens the collector does not 
> reconnect and the system becomes inactive with the processes still running.
> 2011-10-14 01:49:47,386 [logicalNode collector0_log_dir-115] ERROR 
> com.cloudera.flume.core.connector.DirectDriver - Closing down due to 
> exception during append calls
> 2011-10-14 01:49:47,387 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.core.connector.DirectDriver - Connector logicalNode 
> collector0_log_dir-115 exited with error: Blocked append interrupted by 
> rotation event
> java.lang.InterruptedException: Blocked append interrupted by rotation event
>         at 
> com.cloudera.flume.handlers.rolling.RollSink.append(RollSink.java:209)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at com.cloudera.flume.core.MaskDecorator.append(MaskDecorator.java:43)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.debug.InsistentOpenDecorator.append(InsistentOpenDecorator.java:169)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.debug.StubbornAppendSink.append(StubbornAppendSink.java:71)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.debug.InsistentAppendDecorator.append(InsistentAppendDecorator.java:110)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.endtoend.AckChecksumChecker.append(AckChecksumChecker.java:113)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.batch.UnbatchingDecorator.append(UnbatchingDecorator.java:62)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.handlers.batch.GunzipDecorator.append(GunzipDecorator.java:81)
>         at 
> com.cloudera.flume.collector.CollectorSink.append(CollectorSink.java:222)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.core.extractors.DateExtractor.append(DateExtractor.java:129)
>         at 
> com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
>         at 
> com.cloudera.flume.core.extractors.RegexExtractor.append(RegexExtractor.java:88)
>         at 
> com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:133)
> 2011-10-14 01:49:47,388 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.collector.CollectorSource - closed
> 2011-10-14 01:49:48,391 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.handlers.thrift.ThriftEventSource - Closed server on port 
> 36892...
> 2011-10-14 01:49:48,391 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.handlers.thrift.ThriftEventSource - Queue still has 1000 
> elements ...
> 2011-10-14 01:49:58,399 [logicalNode collector0_log_dir-115] WARN  
> com.cloudera.flume.handlers.thrift.ThriftEventSource - Close timed out due to 
> no progress.  Closing despite having 1000 values still enqueued
> 2011-10-14 01:49:58,399 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.handlers.rolling.RollSink - closing RollSink 
> 'escapedCustomDfs("hdfs://van-mang-perf-hadoop-namenode1.net:8020/rawLogs/%{dateyear}-%{datemonth}-%{dateday}/%{datehr}00","raw-%{rolltag}"
>  )'
> 2011-10-14 01:49:58,400 [logicalNode collector0_log_dir-115] INFO  
> com.cloudera.flume.handlers.rolling.RollSink - double close 
> 'escapedCustomDfs("hdfs://van-mang-perf-hadoop-namenode1.net:8020/rawLogs/%{dateyear}-%{datemonth}-%{dateday}/%{datehr}00","raw-%{rolltag}"
>  )'
> 2011-10-14 01:49:58,400 [logicalNode collector0_log_dir-115] ERROR 
> com.cloudera.flume.core.connector.DirectDriver - Exiting driver logicalNode 
> collector0_log_dir-115 in error state CollectorSource | RegexExtractor 
> because Blocked append interrupted by rotation event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to