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

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



bq.  On 2012-04-09 11:10:19, Brock Noland wrote:
bq.  > I think the change makes sense, but I am not sure if it solves the 
problem from the JIRA? From what I can tell about the error, it looks like HDFS 
is trying to add a shutdown hook after the shutdown has started.

The roller can cause the file to be closed during the process() and the stop() 
also closes the file. It looked like two threads are trying to close the same 
file simultaneously.  


- Prasad


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


On 2012-04-09 06:54:55, Prasad Mujumdar wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/4681/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2012-04-09 06:54:55)
bq.  
bq.  
bq.  Review request for Flume and Arvind Prabhakar.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  The sink runner's stop method first calls stop() to underlying sink and 
then shuts down the PollingRunner thread. If that thread is in middle of 
process, it leads to race conditions in the sink's process() and stop().
bq.  Rather than making all sinks to handle concurrently process() and stop(), 
its safer to shutdown the runner thread first and then stop the sink.
bq.  
bq.  
bq.  This addresses bug FLUME-1110.
bq.      https://issues.apache.org/jira/browse/FLUME-1110
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    flume-ng-core/src/main/java/org/apache/flume/SinkRunner.java e73c09b 
bq.  
bq.  Diff: https://reviews.apache.org/r/4681/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  full regression test run
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Prasad
bq.  
bq.


                
> HDFS Sink throws IllegalStateException when flume-daemon shuts down
> -------------------------------------------------------------------
>
>                 Key: FLUME-1110
>                 URL: https://issues.apache.org/jira/browse/FLUME-1110
>             Project: Flume
>          Issue Type: Bug
>          Components: Sinks+Sources
>    Affects Versions: v1.1.0
>            Reporter: Prasad Mujumdar
>            Assignee: Prasad Mujumdar
>             Fix For: v1.2.0
>
>
> When using HDFS sink, if you shutdown the daemon (sudo 
> /etc/init.d/flume-ng-node stop), then an IllegalStateException is shown in 
> the logs (/var/log/flume-ng/flume.log).
> 2012-04-06 10:44:19,912 ERROR hdfs.HDFSEventSink: Error calling 
> org.apache.flume.sink.hdfs.HDFSEventSink$4@32091738
> java.lang.IllegalStateException: Shutdown in progress
> at java.lang.ApplicationShutdownHooks.add(ApplicationShutdownHooks.java:39)
> at java.lang.Runtime.addShutdownHook(Runtime.java:192)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:1607)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1579)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:228)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:183)
> at org.apache.flume.sink.hdfs.BucketWriter.renameBucket(BucketWriter.java:196)
> at org.apache.flume.sink.hdfs.BucketWriter.close(BucketWriter.java:122)
> at org.apache.flume.sink.hdfs.HDFSEventSink$4.call(HDFSEventSink.java:440)
> at org.apache.flume.sink.hdfs.HDFSEventSink$4.call(HDFSEventSink.java:436)
> 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(ThreadPoolExecutor.java:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> 2012-04-06 10:44:19,927 INFO source.SyslogTcpSource: Syslog TCP Source 
> stopping...
> 2012-04-06 10:44:19,927 INFO source.SyslogTcpSource: Metrics:{ name:null 
> counters:{events.success=11002} }

--
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