Flume error in FileChannel

2013-10-15 Thread Bhaskar V. Karambelkar
Recently we switched over from Memory Channel to File Channel, as Memory
Channel has some GC issues.
Occasionally in File Channel I see this exception

org.apache.flume.ChannelException: Put queue for FileBackedTransaction of
capacity 5000 full, consider committing more frequently, increasing
capacity or increasing thread count. [channel=fileChannelD1]

Client batchSize is 1,000, and HDFS Sink batch size is also 1,000.
The channel capacity is 1M (1,000,000), and Channel Tx Capacity is 5,000

The underlying directories are not full, so the channel should have enough
space, nor does the channel has any backlog.

What I'm confused by are the 3 options the Exception mentions.

How do I , commit more frequently ? or increase capacity ? (Capacity of
Channel is 1M, and that is not full), or increase thread count ?( I see no
option of thread count in file channel, or is this referring to threadcout
of the HDFS sink which reads from this sink ?)

Lastly, would GC in Hadoop (mostly Namenode) cause HDFS Timeout issues in
HDFS Sink, coz we see HDFS Timeout errors, more or less at the same time
across all our flume nodes, so I suspect it could be NameNode GC causing
timeout issues.


thanks
Bhaskar


Re: Flume error in FileChannel

2013-10-15 Thread Hari Shreedharan
What source are you using? Looks like the source is writing  5K events in one 
transaction 


Thanks,
Hari


On Tuesday, October 15, 2013 at 12:24 PM, Bhaskar V. Karambelkar wrote:

 Recently we switched over from Memory Channel to File Channel, as Memory 
 Channel has some GC issues.
 Occasionally in File Channel I see this exception
 
 org.apache.flume.ChannelException: Put queue for FileBackedTransaction of 
 capacity 5000 full, consider committing more frequently, increasing capacity 
 or increasing thread count. [channel=fileChannelD1]
 
 Client batchSize is 1,000, and HDFS Sink batch size is also 1,000.
 The channel capacity is 1M (1,000,000), and Channel Tx Capacity is 5,000
 
 The underlying directories are not full, so the channel should have enough 
 space, nor does the channel has any backlog. 
 
 What I'm confused by are the 3 options the Exception mentions.
 
 How do I , commit more frequently ? or increase capacity ? (Capacity of 
 Channel is 1M, and that is not full), or increase thread count ?( I see no 
 option of thread count in file channel, or is this referring to threadcout of 
 the HDFS sink which reads from this sink ?) 
 
 Lastly, would GC in Hadoop (mostly Namenode) cause HDFS Timeout issues in 
 HDFS Sink, coz we see HDFS Timeout errors, more or less at the same time 
 across all our flume nodes, so I suspect it could be NameNode GC causing 
 timeout issues. 
 
 
 thanks
 Bhaskar
 



Re: Flume error in FileChannel

2013-10-15 Thread Bhaskar V. Karambelkar
Source is Avro Source which gets evnets fed by a custom JVM application
using the flume client SDK.

So referring to the client SDK, if the batchSize property has be set to
1,000, but I pass say 10,000 events in the client.addBatch(ListEvent) call
what happens ?


On Tue, Oct 15, 2013 at 3:54 PM, Hari Shreedharan hshreedha...@cloudera.com
 wrote:

  What source are you using? Looks like the source is writing  5K events
 in one transaction


 Thanks,
 Hari

 On Tuesday, October 15, 2013 at 12:24 PM, Bhaskar V. Karambelkar wrote:

 Recently we switched over from Memory Channel to File Channel, as Memory
 Channel has some GC issues.
 Occasionally in File Channel I see this exception

 org.apache.flume.ChannelException: Put queue for FileBackedTransaction of
 capacity 5000 full, consider committing more frequently, increasing
 capacity or increasing thread count. [channel=fileChannelD1]

 Client batchSize is 1,000, and HDFS Sink batch size is also 1,000.
 The channel capacity is 1M (1,000,000), and Channel Tx Capacity is 5,000

 The underlying directories are not full, so the channel should have enough
 space, nor does the channel has any backlog.

 What I'm confused by are the 3 options the Exception mentions.

 How do I , commit more frequently ? or increase capacity ? (Capacity of
 Channel is 1M, and that is not full), or increase thread count ?( I see no
 option of thread count in file channel, or is this referring to threadcout
 of the HDFS sink which reads from this sink ?)

 Lastly, would GC in Hadoop (mostly Namenode) cause HDFS Timeout issues in
 HDFS Sink, coz we see HDFS Timeout errors, more or less at the same time
 across all our flume nodes, so I suspect it could be NameNode GC causing
 timeout issues.


 thanks
 Bhaskar





Flume error in FIleChannel

2013-06-27 Thread Rahul Ravindran
Hi,
  We are using CDH flume 1.3 (which ships with 4.2.1). We see this error in our 
flume logs in our production system and restarting flume did not help. Looking 
at the flume code, it appears to be expecting the byte to be an OPERATION, but 
is not. Any ideas on what happened?

Thanks,
~Rahul.


27 Jun 2013 05:58:12,246 INFO  [Log-BackgroundWorker-ch3] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-780.meta currentPosition = 68617970, logWriteOrderID = 
1400053754907
27 Jun 2013 05:58:12,248 INFO  [Log-BackgroundWorker-ch3] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume3/data/log-780 position: 68617970 logWriteOrderID: 1400053754907
27 Jun 2013 05:58:12,529 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108)  
- Start checkpoint for /flume2/checkpoint/checkpoint, elements to sync = 43941
27 Jun 2013 05:58:12,531 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)  - 
Updating checkpoint metadata: logWriteOrderID: 1400053760540, queueSize: 0, 
queueHead: 16264802
27 Jun 2013 05:58:12,583 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-786.meta currentPosition = 66046989, logWriteOrderID = 
1400053760540
27 Jun 2013 05:58:12,585 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume2/data/log-786 position: 66046989 logWriteOrderID: 1400053760540
27 Jun 2013 05:58:17,679 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108)  
- Start checkpoint for /flume1/checkpoint/checkpoint, elements to sync = 225955
27 Jun 2013 05:58:17,682 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)  - 
Updating checkpoint metadata: logWriteOrderID: 1400053832535, queueSize: 
7255426, queueHead: 1778328
27 Jun 2013 05:58:17,736 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-781.meta currentPosition = 652840345, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:17,738 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume1/data/log-781 position: 652840345 logWriteOrderID: 
1400053832535
27 Jun 2013 05:58:17,739 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing 
RandomReader /flume1/data/log-779
27 Jun 2013 05:58:17,745 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-779.meta currentPosition = 1599537606, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:17,746 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing 
RandomReader /flume1/data/log-780
27 Jun 2013 05:58:17,752 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-780.meta currentPosition = 1610002802, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:25,538 ERROR [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.HDFSEventSink.process:457)  - process failed
java.lang.IllegalStateException: 1
        at 
com.google.common.base.Preconditions.checkState(Preconditions.java:145)
        at 
org.apache.flume.channel.file.LogFile$RandomReader.get(LogFile.java:335)
        at org.apache.flume.channel.file.Log.get(Log.java:478)
        at 
org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doTake(FileChannel.java:500)
        at 
org.apache.flume.channel.BasicTransactionSemantics.take(BasicTransactionSemantics.java:113)
        at 
org.apache.flume.channel.BasicChannelSemantics.take(BasicChannelSemantics.java:95)
        at 
org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:386)
        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)
27 Jun 2013 05:58:25,558 ERROR [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.SinkRunner$PollingRunner.run:160)  - Unable to deliver event. 
Exception follows.
org.apache.flume.EventDeliveryException: java.lang.IllegalStateException: 1
        at 
org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:461)
        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)
Caused by: java.lang.IllegalStateException: 1
        at 

Re: Flume error in FIleChannel

2013-06-27 Thread Hari Shreedharan
Looks like the file may have been corrupted. Can you verify if you are out
of disk space or can see something that might have caused the data to be
corrupted?

Hari


On Thu, Jun 27, 2013 at 6:41 AM, Rahul Ravindran rahu...@yahoo.com wrote:

 Hi,
   We are using CDH flume 1.3 (which ships with 4.2.1). We see this error
 in our flume logs in our production system and restarting flume did not
 help. Looking at the flume code, it appears to be expecting the byte to be
 an OPERATION, but is not. Any ideas on what happened?

 Thanks,
 ~Rahul.


 27 Jun 2013 05:58:12,246 INFO  [Log-BackgroundWorker-ch3]
 (org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)
  - Updating log-780.meta currentPosition = 68617970, logWriteOrderID =
 1400053754907
 27 Jun 2013 05:58:12,248 INFO  [Log-BackgroundWorker-ch3]
 (org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated
 checkpoint for file: /flume3/data/log-780 position: 68617970
 logWriteOrderID: 1400053754907
 27 Jun 2013 05:58:12,529 INFO  [Log-BackgroundWorker-ch2]
 (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108)
  - Start checkpoint for /flume2/checkpoint/checkpoint, elements to sync =
 43941
 27 Jun 2013 05:58:12,531 INFO  [Log-BackgroundWorker-ch2]
 (org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)
  - Updating checkpoint metadata: logWriteOrderID: 1400053760540, queueSize:
 0, queueHead: 16264802
 27 Jun 2013 05:58:12,583 INFO  [Log-BackgroundWorker-ch2]
 (org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)
  - Updating log-786.meta currentPosition = 66046989, logWriteOrderID =
 1400053760540
 27 Jun 2013 05:58:12,585 INFO  [Log-BackgroundWorker-ch2]
 (org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated
 checkpoint for file: /flume2/data/log-786 position: 66046989
 logWriteOrderID: 1400053760540
 27 Jun 2013 05:58:17,679 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108)
  - Start checkpoint for /flume1/checkpoint/checkpoint, elements to sync =
 225955
 27 Jun 2013 05:58:17,682 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)
  - Updating checkpoint metadata: logWriteOrderID: 1400053832535, queueSize:
 7255426, queueHead: 1778328
 27 Jun 2013 05:58:17,736 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)
  - Updating log-781.meta currentPosition = 652840345, logWriteOrderID =
 1400053832535
 27 Jun 2013 05:58:17,738 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated
 checkpoint for file: /flume1/data/log-781 position: 652840345
 logWriteOrderID: 1400053832535
 27 Jun 2013 05:58:17,739 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing
 RandomReader /flume1/data/log-779
 27 Jun 2013 05:58:17,745 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)
  - Updating log-779.meta currentPosition = 1599537606, logWriteOrderID =
 1400053832535
 27 Jun 2013 05:58:17,746 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing
 RandomReader /flume1/data/log-780
 27 Jun 2013 05:58:17,752 INFO  [Log-BackgroundWorker-ch1]
 (org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)
  - Updating log-780.meta currentPosition = 1610002802, logWriteOrderID =
 1400053832535
 27 Jun 2013 05:58:25,538 ERROR
 [SinkRunner-PollingRunner-DefaultSinkProcessor]
 (org.apache.flume.sink.hdfs.HDFSEventSink.process:457)  - process failed
 java.lang.IllegalStateException: 1
 at
 com.google.common.base.Preconditions.checkState(Preconditions.java:145)
 at
 org.apache.flume.channel.file.LogFile$RandomReader.get(LogFile.java:335)
 at org.apache.flume.channel.file.Log.get(Log.java:478)
 at
 org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doTake(FileChannel.java:500)
 at
 org.apache.flume.channel.BasicTransactionSemantics.take(BasicTransactionSemantics.java:113)
 at
 org.apache.flume.channel.BasicChannelSemantics.take(BasicChannelSemantics.java:95)
 at
 org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:386)
 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)
 27 Jun 2013 05:58:25,558 ERROR
 [SinkRunner-PollingRunner-DefaultSinkProcessor]
 (org.apache.flume.SinkRunner$PollingRunner.run:160)  - Unable to deliver
 event. Exception follows.
 org.apache.flume.EventDeliveryException: java.lang.IllegalStateException: 1
 at
 org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:461)
 at
 

Re: Flume error in FIleChannel

2013-06-27 Thread Rahul Ravindran
There was no disk space issue and we ran fsck on the disks and did not find any 
file corruption issue. Did not see anything weird at that point in time either



 From: Hari Shreedharan hshreedha...@cloudera.com
To: user@flume.apache.org user@flume.apache.org; Rahul Ravindran 
rahu...@yahoo.com 
Sent: Thursday, June 27, 2013 11:24 AM
Subject: Re: Flume error in FIleChannel
 


Looks like the file may have been corrupted. Can you verify if you are out of 
disk space or can see something that might have caused the data to be corrupted?

Hari



On Thu, Jun 27, 2013 at 6:41 AM, Rahul Ravindran rahu...@yahoo.com wrote:

Hi,
  We are using CDH flume 1.3 (which ships with 4.2.1). We see this error in 
our flume logs in our production system and restarting flume did not help. 
Looking at the flume code, it appears to be expecting the byte to be an 
OPERATION, but is not. Any ideas on what happened?


Thanks,
~Rahul.




27 Jun 2013 05:58:12,246 INFO  [Log-BackgroundWorker-ch3] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-780.meta currentPosition = 68617970, logWriteOrderID = 
1400053754907
27 Jun 2013 05:58:12,248 INFO  [Log-BackgroundWorker-ch3] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume3/data/log-780 position: 68617970 logWriteOrderID: 
1400053754907
27 Jun 2013 05:58:12,529 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108) 
 - Start checkpoint for /flume2/checkpoint/checkpoint, elements to sync = 43941
27 Jun 2013 05:58:12,531 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)  - 
Updating checkpoint metadata: logWriteOrderID: 1400053760540, queueSize: 0, 
queueHead: 16264802
27 Jun 2013 05:58:12,583 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-786.meta currentPosition = 66046989, logWriteOrderID = 
1400053760540
27 Jun 2013 05:58:12,585 INFO  [Log-BackgroundWorker-ch2] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume2/data/log-786 position: 66046989 logWriteOrderID: 
1400053760540
27 Jun 2013 05:58:17,679 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:108) 
 - Start checkpoint for /flume1/checkpoint/checkpoint, elements to sync = 
225955
27 Jun 2013 05:58:17,682 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:120)  - 
Updating checkpoint metadata: logWriteOrderID: 1400053832535, queueSize: 
7255426, queueHead: 1778328
27 Jun 2013 05:58:17,736 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-781.meta currentPosition = 652840345, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:17,738 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.Log.writeCheckpoint:898)  - Updated checkpoint 
for file: /flume1/data/log-781 position: 652840345 logWriteOrderID: 
1400053832535
27 Jun 2013 05:58:17,739 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing 
RandomReader /flume1/data/log-779
27 Jun 2013 05:58:17,745 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-779.meta currentPosition = 1599537606, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:17,746 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFile$RandomReader.close:356)  - Closing 
RandomReader /flume1/data/log-780
27 Jun 2013 05:58:17,752 INFO  [Log-BackgroundWorker-ch1] 
(org.apache.flume.channel.file.LogFileV3$MetaDataWriter.markCheckpoint:85)  - 
Updating log-780.meta currentPosition = 1610002802, logWriteOrderID = 
1400053832535
27 Jun 2013 05:58:25,538 ERROR [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.HDFSEventSink.process:457)  - process failed
java.lang.IllegalStateException: 1
        at 
com.google.common.base.Preconditions.checkState(Preconditions.java:145)
        at 
org.apache.flume.channel.file.LogFile$RandomReader.get(LogFile.java:335)
        at org.apache.flume.channel.file.Log.get(Log.java:478)
        at 
org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doTake(FileChannel.java:500)
        at 
org.apache.flume.channel.BasicTransactionSemantics.take(BasicTransactionSemantics.java:113)
        at 
org.apache.flume.channel.BasicChannelSemantics.take(BasicChannelSemantics.java:95)
        at 
org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:386)
        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