[jira] [Updated] (FLUME-1423) Low throughput of FileChannel

2012-08-02 Thread Denny Ye (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denny Ye updated FLUME-1423:


Attachment: FLUME-1423.patch

> Low throughput of FileChannel
> -
>
> Key: FLUME-1423
> URL: https://issues.apache.org/jira/browse/FLUME-1423
> Project: Flume
>  Issue Type: Improvement
>  Components: Channel
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
>  Labels: file, filechannel
> Fix For: v1.3.0
>
> Attachments: FLUME-1423.patch
>
>
> The performance result of FileChannel is below expectation, almost 5MB/s. May 
> be it should be tune to reduce effect of ingest system.

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




Review Request: Low throughput of FileChannel

2012-08-02 Thread Denny Ye

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

Review request for Flume, Hari Shreedharan and Patrick Wendell.


Description
---

Here is the description in code changes
1. Remove the 'FileChannel.force(false)'. Each commit from Source will invoke 
this 'force' method. This method is too heavy for amounts of data comes. Each 
'force' action will be consume 50-500ms that it confirms data stored into disk. 
Normally, OS will flush data from kernal buffer to disk asynchronously with ms 
level latency. It may useless in each commit operation. Certainly, data loss 
may occurs in server crash not process crash. Server crash is infrequent.
2. Do not pre-allocate disk space. Disk doesn't need the pre-allocation.
3. Use 'RandomAccessFile.write()' to replace 'FileChannel.write()'. Both in my 
test result and low-level instruction, the former is better than the latter

Here I posted three changes, and I would like to use thread-level cached 
DirectByteBuffer to replace inner-heap ByteBuffer.allocate() (reuse outer-heap 
memory to reduce time that copying from heap to kernal). I will test this 
changes in next phase.


This addresses bug https://issues.apache.org/jira/browse/FLUME-1423.

https://issues.apache.org/jira/browse/https://issues.apache.org/jira/browse/FLUME-1423


Diffs
-

  
trunk/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java
 1363210 

Diff: https://reviews.apache.org/r/6329/diff/


Testing
---


Thanks,

Denny Ye



Re: Sending avro data from other languages

2012-08-02 Thread Juhani Connolly
On paper it certainly seems like a good solution, it's just unfortunate 
that some "supported" languages can't actually interface to it. I 
understand that thrift can be quite a nuisance to deal with at times.


On 08/02/2012 11:01 PM, Brock Noland wrote:

I cannot answer what made us move to Avro. However, I prefer Avro because
you don't have to build the thrift compiler and you aren't required to do
code generation.

On Wed, Aug 1, 2012 at 11:06 PM, Juhani Connolly <
juhani_conno...@cyberagent.co.jp> wrote:


It looks to me like this was because of the transceiver I was using.

Unfortunately it seems like avro doesn't have a python implementation of a
transceiver that fits the format expected by netty/avro(in fact it only has
one transceiver... HTTPTransceiver).

To address this, I'm thinking of putting together a thrift source(the
legacy source doesn't seem to be usable as it returns nothing, and lacks
batching). Does this seem like a reasonable solution to making it possible
to send data to flume from other languages(and allowing backoff on
failure?). Historically, what made us move away from thrift to avro?


On 07/30/2012 05:34 PM, Juhani Connolly wrote:


I'm playing around with making a standalone tail client in python(so that
I can access inode data) that tracks position in a file and then sends it
across avro to an avro sink.

However I'm having issues with the avro part of this and wondering if
anyone more familiar with it could help.

I took the flume.avdl file and converted it using "java -jar
~/Downloads/avro-tools-1.6.3.**jar idl flume.avdl flume.avpr"

I then run it through a simple test program to see if its sending the
data correctly and it sends from the python client fine, but the sink end
OOM's because presumably the wire format is wrong:

2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 => /172.28.19.112:41414] OPEN
2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 => /172.28.19.112:41414] BOUND: /172.28.19.112:41414
2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 => /172.28.19.112:41414] CONNECTED: /
172.22.114.32:55671
2012-07-30 17:22:57,646 WARN ipc.NettyServer: Unexpected exception from
downstream.
java.lang.OutOfMemoryError: Java heap space
 at java.util.ArrayList.(**ArrayList.java:112)
 at org.apache.avro.ipc.**NettyTransportCodec$**NettyFrameDecoder.
**decodePackHeader(**NettyTransportCodec.java:154)
 at org.apache.avro.ipc.**NettyTransportCodec$**
NettyFrameDecoder.decode(**NettyTransportCodec.java:131)
 at org.jboss.netty.handler.codec.**frame.FrameDecoder.callDecode(
**FrameDecoder.java:282)
 at org.jboss.netty.handler.codec.**frame.FrameDecoder.**
messageReceived(FrameDecoder.**java:216)
 at org.jboss.netty.channel.**Channels.fireMessageReceived(**
Channels.java:274)
 at org.jboss.netty.channel.**Channels.fireMessageReceived(**
Channels.java:261)
 at org.jboss.netty.channel.**socket.nio.NioWorker.read(**
NioWorker.java:351)
 at org.jboss.netty.channel.**socket.nio.NioWorker.**
processSelectedKeys(NioWorker.**java:282)
 at org.jboss.netty.channel.**socket.nio.NioWorker.run(**
NioWorker.java:202)
 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:619)
2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 :> /172.28.19.112:41414] DISCONNECTED
2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 :> /172.28.19.112:41414] UNBOUND
2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
172.22.114.32:55671 :> /172.28.19.112:41414] CLOSED

I've dumped the test program and its output

http://pastebin.com/1DtXZyTu
http://pastebin.com/T9kaqKHY








Re: Low throughput of FileChannel

2012-08-02 Thread Hari Shreedharan
Thanks for filing the jira. It would be nice if you could update the jira with 
what you did/plan to do differently to improve performance. If you already have 
the patch, please consider submitting it for review. We will try and expedite 
its commit.  

Thanks
Hari

--  
Hari Shreedharan


On Thursday, August 2, 2012 at 8:27 PM, Hari Shreedharan wrote:

> Denny,  
>  
> Please file a jira and post your code changes if you would like to contribute 
> it to Apache Flume. One of us will be glad to review and commit it. This way, 
> it will benefit the community in general. This will also allow us to discuss 
> the performance benefits of your code changes.  
>  
> Thanks,
> Hari
>  
> --  
> Hari Shreedharan
>  
>  
> On Thursday, August 2, 2012 at 8:22 PM, Denny Ye wrote:
>  
> > hi Hari,  
> > Mostly channels in my production environment will be configured with 
> > FileChannel. It may impact our platform performance. Also I'm not sure if 
> > anyone already have got better throughput. If anyone have similar result 
> > with me, I'd like to post my code changes to discuss.
> >  
> > -Regards
> > Denny Ye
> >  
> > 2012/8/3 Hari Shreedharan  > (mailto:hshreedha...@cloudera.com)>
> > > Denny,
> > >  
> > > I am not sure if anyone has actually benchmarked the FileChannel. What 
> > > kind of performance are you getting as of now? If you have a patch that 
> > > can improve the performance a lot, please feel free to submit it. We'd 
> > > definitely like to get such a patch committed.
> > >  
> > > Thanks
> > > Hari
> > >  
> > > --
> > > Hari Shreedharan
> > >  
> > >  
> > > On Thursday, August 2, 2012 at 8:02 PM, Denny Ye wrote:
> > >  
> > > > hi all,
> > > > I posted performance of MemoryChannel last week. That's normal 
> > > > throughput in most environment. Therefore, the performance result of 
> > > > FileChannel is below expectation with same environments and parameters, 
> > > > almost 5MB/s.
> > > >
> > > > I want to know your throughput result of FileChannel specially. Am 
> > > > I walking with wrong way? It's hard to believe the result.
> > > >
> > > >Also I have tuning with several code changes, the throughput 
> > > > increasing to 30MB/s. I think there also have lots of points to impact 
> > > > the performance.
> > > >
> > > > Any guys, would you give me your throughput result or feedback for 
> > > > tuning?
> > > >
> > > > -Regards
> > > > Denny Ye
> > > >
> > > >
> > > > -- Forwarded message --
> > > > From: Denny Ye mailto:denny...@gmail.com) 
> > > > (mailto:denny...@gmail.com)>
> > > > Date: 2012/7/25
> > > > Subject: Latest Flume test report and problem
> > > > To: dev@flume.apache.org (mailto:dev@flume.apache.org) 
> > > > (mailto:dev@flume.apache.org)
> > > >
> > > >
> > > > hi all,
> > > >I tested Flume in last week with 
> > > > ScribeSource(https://issues.apache.org/jira/browse/FLUME-1382) and HDFS 
> > > > Sink. More detailed conditions and deployment cases listed below. Too 
> > > > many 'Full GC' impact the throughput and amount of events promoted into 
> > > > old generation. I have applied some tuning methods, no much effect.
> > > >Could someone give me your feedback or tip to reduce the GC problem? 
> > > > Wish your attention.
> > > >
> > > > PS: Using Mike's report template at 
> > > > https://cwiki.apache.org/FLUME/flume-ng-performance-measurements.html
> > > >
> > > > Flume Performance Test 2012-07-25
> > > > Overview
> > > > The Flume agent was run on its own physical machine in a single JVM. A 
> > > > separate client machine generated load against the Flume box in 
> > > > List format. Flume stored data onto a 4-node HDFS cluster 
> > > > configured on its own separate hardware. No virtual machines were used 
> > > > in this test.
> > > > Hardware specs
> > > > CPU: Inter Xeon L5640 2 x quad-core @ 2.27 GHz (12 physical cores)
> > > > Memory: 16 GB
> > > > OS: CentOS release 5.3 (Final)
> > > > Flume configuration
> > > > JAVA Version: 1.6.0_20 (Java HotSpot 64-Bit Server VM)
> > > > JAVA OPTS: -Xms1024m -Xmx4096m -XX:PermSize=256m -XX:NewRatio=1 
> > > > -XX:SurvivorRatio=5 -XX:InitialTenuringThreshold=15 
> > > > -XX:MaxTenuringThreshold=31 -XX:PretenureSizeThreshold=4096
> > > > Num. agents: 1
> > > > Num. parallel flows: 5
> > > > Source: ScribeSource
> > > > Channel: MemoryChannel
> > > > Sink: HDFSEventSink
> > > > Selector: RandomSelector
> > > > Config-file
> > > > # list sources, channels, sinks for the agent
> > > > agent.sources = seqGenSrc
> > > > agent.channels = mc1 mc2 mc3 mc4 mc5
> > > > agent.sinks = hdfsSin1 hdfsSin2 hdfsSin3 hdfsSin4 hdfsSin5
> > > >
> > > > # define sources
> > > > agent.sources.seqGenSrc.type = 
> > > > org.apache.flume.source.scribe.ScribeSource
> > > > agent.sources.seqGenSrc.selector.type = io.flume.RandomSelector
> > > >
> > > > # define sinks
> > > > agent.sinks.hdfsSin1.type = hdfs
> > > > agent.sinks.hdfsSin1.hdfs.path = /flume_test/data1/
> > > > agent.sinks.hdfsSin1.hdfs.rollI

[jira] [Commented] (FLUME-1423) Low throughput of FileChannel

2012-08-02 Thread Denny Ye (JIRA)

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

Denny Ye commented on FLUME-1423:
-

Patrick, I forwarded the mail-list to you. It contains testing environment, 
configuration. Only difference between file and memory channel benchmark is 
channel type. There is single Flume process hold local disk.

> Low throughput of FileChannel
> -
>
> Key: FLUME-1423
> URL: https://issues.apache.org/jira/browse/FLUME-1423
> Project: Flume
>  Issue Type: Improvement
>  Components: Channel
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
>  Labels: file, filechannel
> Fix For: v1.3.0
>
>
> The performance result of FileChannel is below expectation, almost 5MB/s. May 
> be it should be tune to reduce effect of ingest system.

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




[jira] [Commented] (FLUME-1423) Low throughput of FileChannel

2012-08-02 Thread Patrick Wendell (JIRA)

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

Patrick Wendell commented on FLUME-1423:


Danny - can you give more information about the environment you are running on? 

In particular: 

Can you test the throughout of the exact same setup with an in-memory channel, 
and give the resulting throughput, to ensure that this is indeed the bottleneck?

What are the access patterns of the disk that the FileChannel is using? Is it a 
dedicated disk? Or are there other processes reading/writing against the disk?

Can you give the configuration of the agent, including the sources and 
information about any batching taking place within those sources?


Thanks,
Patrick

> Low throughput of FileChannel
> -
>
> Key: FLUME-1423
> URL: https://issues.apache.org/jira/browse/FLUME-1423
> Project: Flume
>  Issue Type: Improvement
>  Components: Channel
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
>  Labels: file, filechannel
> Fix For: v1.3.0
>
>
> The performance result of FileChannel is below expectation, almost 5MB/s. May 
> be it should be tune to reduce effect of ingest system.

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




[jira] [Assigned] (FLUME-1424) File Channel should support encryption

2012-08-02 Thread Arvind Prabhakar (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1424?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arvind Prabhakar reassigned FLUME-1424:
---

Assignee: Arvind Prabhakar

> File Channel should support encryption
> --
>
> Key: FLUME-1424
> URL: https://issues.apache.org/jira/browse/FLUME-1424
> Project: Flume
>  Issue Type: Bug
>Reporter: Arvind Prabhakar
>Assignee: Arvind Prabhakar
>
> When persisting the data to disk, the File Channel should allow some form of 
> encryption to ensure safety of data.

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




[jira] [Created] (FLUME-1424) File Channel should support encryption

2012-08-02 Thread Arvind Prabhakar (JIRA)
Arvind Prabhakar created FLUME-1424:
---

 Summary: File Channel should support encryption
 Key: FLUME-1424
 URL: https://issues.apache.org/jira/browse/FLUME-1424
 Project: Flume
  Issue Type: Bug
Reporter: Arvind Prabhakar


When persisting the data to disk, the File Channel should allow some form of 
encryption to ensure safety of data.

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




Re: Review Request: Flume adopt message from existing local Scribe

2012-08-02 Thread Hari Shreedharan

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


I am not sure exactly how scribe works. From what you explained, it seems like 
you need to write a client of some sort from scribe side or scribe can somehow 
be configured to write to send the messages to a specified host/port? 

Anyway, what I was asking was if you could add a section to the Flume User 
Guide on how to set up the ScribeSource on Flume side, and also how to set up 
scribe to write to this source. You can find the Flume User Guide here: 
flume-ng-doc/sphinx/FlumeUserGuide.rst. If you could add an example of how to 
configure both scribe and flume so that you can dump events to flume, that 
would be great.

I'd like to commit this if I am able to test basic functionality, and improve 
and fix issues as they are noticed, since committing this will not affect the 
working of other components the remaining system.

So if you can help me set it up - configuring this and configuring scribe to 
write to this, it would be great.

- Hari Shreedharan


On July 30, 2012, 2:49 a.m., Denny Ye wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/6089/
> ---
> 
> (Updated July 30, 2012, 2:49 a.m.)
> 
> 
> Review request for Flume and Hari Shreedharan.
> 
> 
> Description
> ---
> 
> There may someone like me that want to replace central Scribe with Flume to 
> adopt existing ingest system, using smooth changes for application user.
> Here is the ScribeSource put into legacy folder without deserializing. 
> 
> 
> This addresses bug https://issues.apache.org/jira/browse/FLUME-1382.
> 
> https://issues.apache.org/jira/browse/https://issues.apache.org/jira/browse/FLUME-1382
> 
> 
> Diffs
> -
> 
>   trunk/flume-ng-dist/pom.xml 1363210 
>   trunk/flume-ng-sources/flume-scribe-source/pom.xml PRE-CREATION 
>   
> trunk/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/LogEntry.java
>  PRE-CREATION 
>   
> trunk/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ResultCode.java
>  PRE-CREATION 
>   
> trunk/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/Scribe.java
>  PRE-CREATION 
>   
> trunk/flume-ng-sources/flume-scribe-source/src/main/java/org/apache/flume/source/scribe/ScribeSource.java
>  PRE-CREATION 
>   trunk/flume-ng-sources/pom.xml PRE-CREATION 
>   trunk/pom.xml 1363210 
> 
> Diff: https://reviews.apache.org/r/6089/diff/
> 
> 
> Testing
> ---
> 
> I already used ScribeSource into local environment and tested in past week. 
> It can use the existing local Scribe interface
> 
> 
> Thanks,
> 
> Denny Ye
> 
>



[jira] [Created] (FLUME-1423) Low throughput of FileChannel

2012-08-02 Thread Denny Ye (JIRA)
Denny Ye created FLUME-1423:
---

 Summary: Low throughput of FileChannel
 Key: FLUME-1423
 URL: https://issues.apache.org/jira/browse/FLUME-1423
 Project: Flume
  Issue Type: Improvement
  Components: Channel
Affects Versions: v1.2.0
Reporter: Denny Ye
Assignee: Denny Ye
 Fix For: v1.3.0


The performance result of FileChannel is below expectation, almost 5MB/s. May 
be it should be tune to reduce effect of ingest system.

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




Re: Low throughput of FileChannel

2012-08-02 Thread Hari Shreedharan
Denny,  

Please file a jira and post your code changes if you would like to contribute 
it to Apache Flume. One of us will be glad to review and commit it. This way, 
it will benefit the community in general. This will also allow us to discuss 
the performance benefits of your code changes.  

Thanks,
Hari

--  
Hari Shreedharan


On Thursday, August 2, 2012 at 8:22 PM, Denny Ye wrote:

> hi Hari,  
> Mostly channels in my production environment will be configured with 
> FileChannel. It may impact our platform performance. Also I'm not sure if 
> anyone already have got better throughput. If anyone have similar result with 
> me, I'd like to post my code changes to discuss.
>  
> -Regards
> Denny Ye
>  
> 2012/8/3 Hari Shreedharan  (mailto:hshreedha...@cloudera.com)>
> > Denny,
> >  
> > I am not sure if anyone has actually benchmarked the FileChannel. What kind 
> > of performance are you getting as of now? If you have a patch that can 
> > improve the performance a lot, please feel free to submit it. We'd 
> > definitely like to get such a patch committed.
> >  
> > Thanks
> > Hari
> >  
> > --
> > Hari Shreedharan
> >  
> >  
> > On Thursday, August 2, 2012 at 8:02 PM, Denny Ye wrote:
> >  
> > > hi all,
> > > I posted performance of MemoryChannel last week. That's normal 
> > > throughput in most environment. Therefore, the performance result of 
> > > FileChannel is below expectation with same environments and parameters, 
> > > almost 5MB/s.
> > >
> > > I want to know your throughput result of FileChannel specially. Am I 
> > > walking with wrong way? It's hard to believe the result.
> > >
> > >Also I have tuning with several code changes, the throughput 
> > > increasing to 30MB/s. I think there also have lots of points to impact 
> > > the performance.
> > >
> > > Any guys, would you give me your throughput result or feedback for 
> > > tuning?
> > >
> > > -Regards
> > > Denny Ye
> > >
> > >
> > > -- Forwarded message --
> > > From: Denny Ye mailto:denny...@gmail.com) 
> > > (mailto:denny...@gmail.com)>
> > > Date: 2012/7/25
> > > Subject: Latest Flume test report and problem
> > > To: dev@flume.apache.org (mailto:dev@flume.apache.org) 
> > > (mailto:dev@flume.apache.org)
> > >
> > >
> > > hi all,
> > >I tested Flume in last week with 
> > > ScribeSource(https://issues.apache.org/jira/browse/FLUME-1382) and HDFS 
> > > Sink. More detailed conditions and deployment cases listed below. Too 
> > > many 'Full GC' impact the throughput and amount of events promoted into 
> > > old generation. I have applied some tuning methods, no much effect.
> > >Could someone give me your feedback or tip to reduce the GC problem? 
> > > Wish your attention.
> > >
> > > PS: Using Mike's report template at 
> > > https://cwiki.apache.org/FLUME/flume-ng-performance-measurements.html
> > >
> > > Flume Performance Test 2012-07-25
> > > Overview
> > > The Flume agent was run on its own physical machine in a single JVM. A 
> > > separate client machine generated load against the Flume box in 
> > > List format. Flume stored data onto a 4-node HDFS cluster 
> > > configured on its own separate hardware. No virtual machines were used in 
> > > this test.
> > > Hardware specs
> > > CPU: Inter Xeon L5640 2 x quad-core @ 2.27 GHz (12 physical cores)
> > > Memory: 16 GB
> > > OS: CentOS release 5.3 (Final)
> > > Flume configuration
> > > JAVA Version: 1.6.0_20 (Java HotSpot 64-Bit Server VM)
> > > JAVA OPTS: -Xms1024m -Xmx4096m -XX:PermSize=256m -XX:NewRatio=1 
> > > -XX:SurvivorRatio=5 -XX:InitialTenuringThreshold=15 
> > > -XX:MaxTenuringThreshold=31 -XX:PretenureSizeThreshold=4096
> > > Num. agents: 1
> > > Num. parallel flows: 5
> > > Source: ScribeSource
> > > Channel: MemoryChannel
> > > Sink: HDFSEventSink
> > > Selector: RandomSelector
> > > Config-file
> > > # list sources, channels, sinks for the agent
> > > agent.sources = seqGenSrc
> > > agent.channels = mc1 mc2 mc3 mc4 mc5
> > > agent.sinks = hdfsSin1 hdfsSin2 hdfsSin3 hdfsSin4 hdfsSin5
> > >
> > > # define sources
> > > agent.sources.seqGenSrc.type = org.apache.flume.source.scribe.ScribeSource
> > > agent.sources.seqGenSrc.selector.type = io.flume.RandomSelector
> > >
> > > # define sinks
> > > agent.sinks.hdfsSin1.type = hdfs
> > > agent.sinks.hdfsSin1.hdfs.path = /flume_test/data1/
> > > agent.sinks.hdfsSin1.hdfs.rollInterval = 300
> > > agent.sinks.hdfsSin1.hdfs.rollSize = 0
> > > agent.sinks.hdfsSin1.hdfs.rollCount = 100
> > > agent.sinks.hdfsSin1.hdfs.batchSize = 1
> > > agent.sinks.hdfsSin1.hdfs.fileType = DataStream
> > > agent.sinks.hdfsSin1.hdfs.txnEventMax = 1000
> > > # ... define sink #2 #3 #4 #5 ...
> > >
> > > # define channels
> > > agent.channels.mc1.type = memory
> > > agent.channels.mc1.capacity = 100
> > > agent.channels.mc1.transactionCapacity = 1000
> > > # ... define channel #2 #3 #4 #5 ...
> > >
> > > # specify the channel each sink and source should use
> > > agent.sources.seqGenSrc.ch

Re: Low throughput of FileChannel

2012-08-02 Thread Denny Ye
hi Hari,
Mostly channels in my production environment will be configured with
FileChannel. It may impact our platform performance. Also I'm not sure if
anyone already have got better throughput. If anyone have similar result
with me, I'd like to post my code changes to discuss.

-Regards
Denny Ye

2012/8/3 Hari Shreedharan 

> Denny,
>
> I am not sure if anyone has actually benchmarked the FileChannel. What
> kind of performance are you getting as of now? If you have a patch that can
> improve the performance a lot, please feel free to submit it. We'd
> definitely like to get such a patch committed.
>
> Thanks
> Hari
>
> --
> Hari Shreedharan
>
>
> On Thursday, August 2, 2012 at 8:02 PM, Denny Ye wrote:
>
> > hi all,
> > I posted performance of MemoryChannel last week. That's normal
> throughput in most environment. Therefore, the performance result of
> FileChannel is below expectation with same environments and parameters,
> almost 5MB/s.
> >
> > I want to know your throughput result of FileChannel specially. Am I
> walking with wrong way? It's hard to believe the result.
> >
> >Also I have tuning with several code changes, the throughput
> increasing to 30MB/s. I think there also have lots of points to impact the
> performance.
> >
> > Any guys, would you give me your throughput result or feedback for
> tuning?
> >
> > -Regards
> > Denny Ye
> >
> >
> > -- Forwarded message --
> > From: Denny Ye mailto:denny...@gmail.com)>
> > Date: 2012/7/25
> > Subject: Latest Flume test report and problem
> > To: dev@flume.apache.org (mailto:dev@flume.apache.org)
> >
> >
> > hi all,
> >I tested Flume in last week with ScribeSource(
> https://issues.apache.org/jira/browse/FLUME-1382) and HDFS Sink. More
> detailed conditions and deployment cases listed below. Too many 'Full GC'
> impact the throughput and amount of events promoted into old generation. I
> have applied some tuning methods, no much effect.
> >Could someone give me your feedback or tip to reduce the GC problem?
> Wish your attention.
> >
> > PS: Using Mike's report template at
> https://cwiki.apache.org/FLUME/flume-ng-performance-measurements.html
> >
> > Flume Performance Test 2012-07-25
> > Overview
> > The Flume agent was run on its own physical machine in a single JVM. A
> separate client machine generated load against the Flume box in
> List format. Flume stored data onto a 4-node HDFS cluster
> configured on its own separate hardware. No virtual machines were used in
> this test.
> > Hardware specs
> > CPU: Inter Xeon L5640 2 x quad-core @ 2.27 GHz (12 physical cores)
> > Memory: 16 GB
> > OS: CentOS release 5.3 (Final)
> > Flume configuration
> > JAVA Version: 1.6.0_20 (Java HotSpot 64-Bit Server VM)
> > JAVA OPTS: -Xms1024m -Xmx4096m -XX:PermSize=256m -XX:NewRatio=1
> -XX:SurvivorRatio=5 -XX:InitialTenuringThreshold=15
> -XX:MaxTenuringThreshold=31 -XX:PretenureSizeThreshold=4096
> > Num. agents: 1
> > Num. parallel flows: 5
> > Source: ScribeSource
> > Channel: MemoryChannel
> > Sink: HDFSEventSink
> > Selector: RandomSelector
> > Config-file
> > # list sources, channels, sinks for the agent
> > agent.sources = seqGenSrc
> > agent.channels = mc1 mc2 mc3 mc4 mc5
> > agent.sinks = hdfsSin1 hdfsSin2 hdfsSin3 hdfsSin4 hdfsSin5
> >
> > # define sources
> > agent.sources.seqGenSrc.type =
> org.apache.flume.source.scribe.ScribeSource
> > agent.sources.seqGenSrc.selector.type = io.flume.RandomSelector
> >
> > # define sinks
> > agent.sinks.hdfsSin1.type = hdfs
> > agent.sinks.hdfsSin1.hdfs.path = /flume_test/data1/
> > agent.sinks.hdfsSin1.hdfs.rollInterval = 300
> > agent.sinks.hdfsSin1.hdfs.rollSize = 0
> > agent.sinks.hdfsSin1.hdfs.rollCount = 100
> > agent.sinks.hdfsSin1.hdfs.batchSize = 1
> > agent.sinks.hdfsSin1.hdfs.fileType = DataStream
> > agent.sinks.hdfsSin1.hdfs.txnEventMax = 1000
> > # ... define sink #2 #3 #4 #5 ...
> >
> > # define channels
> > agent.channels.mc1.type = memory
> > agent.channels.mc1.capacity = 100
> > agent.channels.mc1.transactionCapacity = 1000
> > # ... define channel #2 #3 #4 #5 ...
> >
> > # specify the channel each sink and source should use
> > agent.sources.seqGenSrc.channels = mc1 mc2 mc3 mc4 mc5
> > agent.sinks.hdfsSin1.channel = mc1
> > # ... specify sink #2 #3 #4 #5 ...
> > Hadoop configuration
> > The HDFS sink was connected to a 4-node Hadoop cluster running CDH3u1.
> For different HDFS sink, HDFS wrote data into different path.
> > Visualization of test setup
> >
> https://lh3.googleusercontent.com/dGumq1pu1Wr3Bj8WJmRHOoLWmUlGqxC4wW7_XCNO9R1wuh15LRXaKKxGoccpjBXtgqcdSVW-vtg
> > There are 10 Scribe Clients and each client send 20 million LogEntry
> objects to ScribleSource.
> > Data description
> > List entries containing a string category and a ByteArray
> body. The ByteArray body size is 500 bytes.
> > Results
> > Throughput:
> > Average:   Source: 46.4 MB/s, Sink: 45.2 MB/s
> > Maximum:Source: 67.1 MB/s, Sink: 88.3 MB/s
> >
> > CPU:   Avera

Re: Low throughput of FileChannel

2012-08-02 Thread Hari Shreedharan
Denny,  

I am not sure if anyone has actually benchmarked the FileChannel. What kind of 
performance are you getting as of now? If you have a patch that can improve the 
performance a lot, please feel free to submit it. We'd definitely like to get 
such a patch committed.

Thanks
Hari

--  
Hari Shreedharan


On Thursday, August 2, 2012 at 8:02 PM, Denny Ye wrote:

> hi all,   
> I posted performance of MemoryChannel last week. That's normal throughput 
> in most environment. Therefore, the performance result of FileChannel is 
> below expectation with same environments and parameters, almost 5MB/s.  
> 
> I want to know your throughput result of FileChannel specially. Am I 
> walking with wrong way? It's hard to believe the result.
>  
>Also I have tuning with several code changes, the throughput increasing to 
> 30MB/s. I think there also have lots of points to impact the performance.  
>   
> Any guys, would you give me your throughput result or feedback for tuning?
>  
> -Regards
> Denny Ye
>  
>  
> -- Forwarded message --
> From: Denny Ye mailto:denny...@gmail.com)>
> Date: 2012/7/25
> Subject: Latest Flume test report and problem
> To: dev@flume.apache.org (mailto:dev@flume.apache.org)
>  
>  
> hi all,  
>I tested Flume in last week with 
> ScribeSource(https://issues.apache.org/jira/browse/FLUME-1382) and HDFS Sink. 
> More detailed conditions and deployment cases listed below. Too many 'Full 
> GC' impact the throughput and amount of events promoted into old generation. 
> I have applied some tuning methods, no much effect.  
>Could someone give me your feedback or tip to reduce the GC problem? Wish 
> your attention.  
>  
> PS: Using Mike's report template at 
> https://cwiki.apache.org/FLUME/flume-ng-performance-measurements.html  
>  
> Flume Performance Test 2012-07-25  
> Overview
> The Flume agent was run on its own physical machine in a single JVM. A 
> separate client machine generated load against the Flume box in 
> List format. Flume stored data onto a 4-node HDFS cluster 
> configured on its own separate hardware. No virtual machines were used in 
> this test.
> Hardware specs
> CPU: Inter Xeon L5640 2 x quad-core @ 2.27 GHz (12 physical cores)
> Memory: 16 GB
> OS: CentOS release 5.3 (Final)
> Flume configuration
> JAVA Version: 1.6.0_20 (Java HotSpot 64-Bit Server VM)
> JAVA OPTS: -Xms1024m -Xmx4096m -XX:PermSize=256m -XX:NewRatio=1 
> -XX:SurvivorRatio=5 -XX:InitialTenuringThreshold=15 
> -XX:MaxTenuringThreshold=31 -XX:PretenureSizeThreshold=4096
> Num. agents: 1
> Num. parallel flows: 5
> Source: ScribeSource
> Channel: MemoryChannel
> Sink: HDFSEventSink
> Selector: RandomSelector
> Config-file
> # list sources, channels, sinks for the agent
> agent.sources = seqGenSrc
> agent.channels = mc1 mc2 mc3 mc4 mc5
> agent.sinks = hdfsSin1 hdfsSin2 hdfsSin3 hdfsSin4 hdfsSin5
>   
> # define sources
> agent.sources.seqGenSrc.type = org.apache.flume.source.scribe.ScribeSource
> agent.sources.seqGenSrc.selector.type = io.flume.RandomSelector
>   
> # define sinks
> agent.sinks.hdfsSin1.type = hdfs
> agent.sinks.hdfsSin1.hdfs.path = /flume_test/data1/
> agent.sinks.hdfsSin1.hdfs.rollInterval = 300
> agent.sinks.hdfsSin1.hdfs.rollSize = 0
> agent.sinks.hdfsSin1.hdfs.rollCount = 100
> agent.sinks.hdfsSin1.hdfs.batchSize = 1
> agent.sinks.hdfsSin1.hdfs.fileType = DataStream
> agent.sinks.hdfsSin1.hdfs.txnEventMax = 1000
> # ... define sink #2 #3 #4 #5 ...
>   
> # define channels
> agent.channels.mc1.type = memory
> agent.channels.mc1.capacity = 100
> agent.channels.mc1.transactionCapacity = 1000
> # ... define channel #2 #3 #4 #5 ...
>   
> # specify the channel each sink and source should use
> agent.sources.seqGenSrc.channels = mc1 mc2 mc3 mc4 mc5
> agent.sinks.hdfsSin1.channel = mc1
> # ... specify sink #2 #3 #4 #5 ...
> Hadoop configuration
> The HDFS sink was connected to a 4-node Hadoop cluster running CDH3u1. For 
> different HDFS sink, HDFS wrote data into different path.
> Visualization of test setup
> https://lh3.googleusercontent.com/dGumq1pu1Wr3Bj8WJmRHOoLWmUlGqxC4wW7_XCNO9R1wuh15LRXaKKxGoccpjBXtgqcdSVW-vtg
> There are 10 Scribe Clients and each client send 20 million LogEntry objects 
> to ScribleSource.  
> Data description
> List entries containing a string category and a ByteArray body. The 
> ByteArray body size is 500 bytes.
> Results
> Throughput:
> Average:   Source: 46.4 MB/s, Sink: 45.2 MB/s
> Maximum:Source: 67.1 MB/s, Sink: 88.3 MB/s
>   
> CPU:   Average: 196%, Maximum: 440%
>   
> GC: Young GC: 1636 times,  Full GC: 384 times
>  
> No data loss.  
> Heap and GC
> By analyzing JVM Heap, we found that there are many LogEntry objects in 
> OldGen. We have tried to carry out some optimizations, but the results are 
> not satisfactory. We will continue to track this limitation.
>   
> FullGC Log examples:   
> [Full GC [PSYoungGen: 1497984K->0K(1797568K)] [PSOldGen: 
> 1720643K->1693741K

Low throughput of FileChannel

2012-08-02 Thread Denny Ye
hi all,
I posted performance of MemoryChannel last week. That's normal
throughput in most environment. Therefore, the performance result of
FileChannel is below expectation with same environments and parameters,
almost 5MB/s.

I want to know your throughput result of FileChannel specially. Am I
walking with wrong way? It's hard to believe the result.

   Also I have tuning with several code changes, the throughput increasing
to 30MB/s. I think there also have lots of points to impact the performance.

Any guys, would you give me your throughput result or feedback for
tuning?

-Regards
Denny Ye


-- Forwarded message --
From: Denny Ye 
Date: 2012/7/25
Subject: Latest Flume test report and problem
To: dev@flume.apache.org


hi all,

   I tested Flume in last week with ScribeSource(
https://issues.apache.org/jira/browse/FLUME-1382) and HDFS Sink. More
detailed conditions and deployment cases listed below. Too many 'Full GC'
impact the throughput and amount of events promoted into old generation. I
have applied some tuning methods, no much effect.

   Could someone give me your feedback or tip to reduce the GC problem?
Wish your attention.


PS: Using Mike's report template at
https://cwiki.apache.org/FLUME/flume-ng-performance-measurements.html

*
*

*Flume Performance Test 2012-07-25*

*Overview*

The Flume agent was run on its own physical machine in a single JVM. A
separate client machine generated load against the Flume box in
List format. Flume stored data onto a 4-node HDFS cluster
configured on its own separate hardware. No virtual machines were used in
this test.

*Hardware specs*

CPU: Inter Xeon L5640 2 x quad-core @ 2.27 GHz (12 physical cores)

Memory: 16 GB

OS: CentOS release 5.3 (Final)

*Flume configuration*

JAVA Version: 1.6.0_20 (Java HotSpot 64-Bit Server VM)

JAVA OPTS: -Xms1024m -Xmx4096m -XX:PermSize=256m -XX:NewRatio=1
-XX:SurvivorRatio=5 -XX:InitialTenuringThreshold=15
-XX:MaxTenuringThreshold=31 -XX:PretenureSizeThreshold=4096

Num. agents: 1

Num. parallel flows: 5

Source: ScribeSource

Channel: MemoryChannel

Sink: HDFSEventSink

Selector: RandomSelector

*Config-file*

# list sources, channels, sinks for the agent

agent.sources = seqGenSrc

agent.channels = mc1 mc2 mc3 mc4 mc5

agent.sinks = hdfsSin1 hdfsSin2 hdfsSin3 hdfsSin4 hdfsSin5



# define sources

agent.sources.seqGenSrc.type = org.apache.flume.source.scribe.ScribeSource

agent.sources.seqGenSrc.selector.type = io.flume.RandomSelector



# define sinks

agent.sinks.hdfsSin1.type = hdfs

agent.sinks.hdfsSin1.hdfs.path = /flume_test/data1/

agent.sinks.hdfsSin1.hdfs.rollInterval = 300

agent.sinks.hdfsSin1.hdfs.rollSize = 0

agent.sinks.hdfsSin1.hdfs.rollCount = 100

agent.sinks.hdfsSin1.hdfs.batchSize = 1

agent.sinks.hdfsSin1.hdfs.fileType = DataStream

agent.sinks.hdfsSin1.hdfs.txnEventMax = 1000

# ... define sink #2 #3 #4 #5 ...



# define channels

agent.channels.mc1.type = memory

agent.channels.mc1.capacity = 100

agent.channels.mc1.transactionCapacity = 1000

# ... define channel #2 #3 #4 #5 ...



# specify the channel each sink and source should use

agent.sources.seqGenSrc.channels = mc1 mc2 mc3 mc4 mc5

agent.sinks.hdfsSin1.channel = mc1

# ... specify sink #2 #3 #4 #5 ...

*Hadoop configuration*

The HDFS sink was connected to a 4-node Hadoop cluster running CDH3u1. For
different HDFS sink, HDFS wrote data into different path.

*Visualization of test setup*

https://lh3.googleusercontent.com/dGumq1pu1Wr3Bj8WJmRHOoLWmUlGqxC4wW7_XCNO9R1wuh15LRXaKKxGoccpjBXtgqcdSVW-vtg

There are 10 Scribe Clients and each client send 20 million LogEntry
objects to ScribleSource.

*Data description*

List entries containing a string category and a ByteArray body.
The ByteArray body size is 500 bytes.

*Results*

Throughput:

Average:   Source: 46.4 MB/s, Sink: 45.2 MB/s

Maximum:Source: 67.1 MB/s, Sink: 88.3 MB/s



CPU:   Average: 196%, Maximum: 440%



GC: Young GC: 1636 times,  Full GC: 384 times


No data loss.

*Heap and GC*

By analyzing JVM Heap, we found that there are many LogEntry objects in
OldGen. We have tried to carry out some optimizations, but the results are
not satisfactory. We will continue to track this limitation.



FullGC Log examples:

[Full GC [PSYoungGen: 1497984K->0K(1797568K)] [PSOldGen:
1720643K->1693741K(2097152K)] 3218627K->1693741K(3894720K) [PSPermGen:
14566K->14566K(262144K)], 5.0027700 secs] [Times: user=5.01 sys=0.00,
real=5.00 secs]

[Full GC [PSYoungGen: 1497960K->0K(1797568K)] [PSOldGen:
1693805K->1752540K(2097152K)] 3191765K->1752540K(3894720K) [PSPermGen:
14571K->14571K(262144K)], 5.0732570 secs] [Times: user=5.07 sys=0.00,
real=5.07 secs]

[Full GC [PSYoungGen: 1497984K->0K(1797568K)] [PSOldGen:
1752540K->1642553K(2097152K)] 3250524K->1642553K(3894720K) [PSPermGen:
14572K->14568K(262144K)], 5.0710730 secs] [Times: user=5.07 sys=0.01,
real=5.08 secs]



-Regards

Denny Ye


Re: Review Request: Improvement for Log4j configuration

2012-08-02 Thread Denny Ye


> On Aug. 2, 2012, 10:03 a.m., Mike Percy wrote:
> > The INFO Hadoop setting is certainly needed. Also, including the thread 
> > name is great. However I have heard that DailyRollingFileAppender is 
> > unreliable and can cause data loss at roll time. What's the reason behind 
> > that change?

Thanks Mike, I'm familiar with DatedFileAppender in my project and 
DailyRollingFileAppender in Hadoop. I think that's good in troubleshooting for 
me and QA to track issue at specified date. Log name suffixed with '-MM-dd' 
is better than 'flume-{number}.log'. Another tip in my experience is we have 
tools to clean up log file automatically with expire, it verify file suffix.

Also, following your concern about data loss in DailyRollingFileAppender. I 
found a Hadoop bug https://issues.apache.org/jira/browse/HADOOP-8149, the same 
problem and they changed DailyRollingFileAppender back to RollingFileAppender. 
I don't agree with them completely. It's hard to distinguish log at specified 
date.

May be the DatedFileAppender is another choose. Do you think so?  


- Denny


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


On Aug. 2, 2012, 7 a.m., Denny Ye wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/6316/
> ---
> 
> (Updated Aug. 2, 2012, 7 a.m.)
> 
> 
> Review request for Flume and Hari Shreedharan.
> 
> 
> Description
> ---
> 
> Updated log4j.properties 
> 1. Add 'logs' folder. It's better to aggregate logs into unified folder.
> 2. Log file using DailyRollingFileAppender
> 3. More useful log pattern. Add thread and code line information
> 4. Set debug level for Hadoop. If we use debug level, Hadoop log takes too 
> many lines.
> 
> 
> This addresses bug https://issues.apache.org/jira/browse/FLUME-1418.
> 
> https://issues.apache.org/jira/browse/https://issues.apache.org/jira/browse/FLUME-1418
> 
> 
> Diffs
> -
> 
>   trunk/conf/log4j.properties 1363210 
> 
> Diff: https://reviews.apache.org/r/6316/diff/
> 
> 
> Testing
> ---
> 
> That's OK in my environment
> 
> 
> Thanks,
> 
> Denny Ye
> 
>



[jira] [Created] (FLUME-1422) Fix "BarSource" Class Signature in Flume Developer Guide

2012-08-02 Thread Patrick Wendell (JIRA)
Patrick Wendell created FLUME-1422:
--

 Summary: Fix "BarSource" Class Signature in Flume Developer Guide
 Key: FLUME-1422
 URL: https://issues.apache.org/jira/browse/FLUME-1422
 Project: Flume
  Issue Type: Bug
Reporter: Patrick Wendell
Assignee: Patrick Wendell


The class signature should be:

public class BarSource extends AbstractSource implements Configurable, 
PollableSource

(where PollableSource is org.apache.flume.PollableSource)

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




[jira] [Assigned] (FLUME-1420) Exception should be thrown if we cannot instaniate an EventSerializer

2012-08-02 Thread Patrick Wendell (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Patrick Wendell reassigned FLUME-1420:
--

Assignee: Patrick Wendell

> Exception should be thrown if we cannot instaniate an EventSerializer
> -
>
> Key: FLUME-1420
> URL: https://issues.apache.org/jira/browse/FLUME-1420
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v1.2.0
>Reporter: Brock Noland
>Assignee: Patrick Wendell
>
> Currently EventSerailizerFactory returns null if it cannot instantiate the 
> class. Then the caller NPEs because they don't expect null. If we cannot 
> satisfy the caller we should throw an exception
> {noformat}
> 2012-08-02 16:38:26,489 ERROR serialization.EventSerializerFactory: Unable to 
> instantiate Builder from 
> org.apache.flume.serialization.BodyTextEventSerializer
> 2012-08-02 16:38:26,490 WARN hdfs.HDFSEventSink: HDFS IO error
> java.io.IOException: java.lang.NullPointerException
> at 
> org.apache.flume.sink.hdfs.BucketWriter.doOpen(BucketWriter.java:202)
> at 
> org.apache.flume.sink.hdfs.BucketWriter.access$000(BucketWriter.java:48)
> at 
> org.apache.flume.sink.hdfs.BucketWriter$1.run(BucketWriter.java:155)
> at 
> org.apache.flume.sink.hdfs.BucketWriter$1.run(BucketWriter.java:152)
> at 
> org.apache.flume.sink.hdfs.BucketWriter.runPrivileged(BucketWriter.java:125)
> at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:152)
> at 
> org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:307)
> at 
> org.apache.flume.sink.hdfs.HDFSEventSink$1.call(HDFSEventSink.java:717)
> at 
> org.apache.flume.sink.hdfs.HDFSEventSink$1.call(HDFSEventSink.java:714)
> 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)
> Caused by: java.lang.NullPointerException
> at 
> org.apache.flume.sink.hdfs.HDFSDataStream.open(HDFSDataStream.java:75)
> at 
> org.apache.flume.sink.hdfs.BucketWriter.doOpen(BucketWriter.java:188)
> ... 13 more
> {noformat}

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




[jira] [Commented] (FLUME-1421) PollableSourceRunner does not name it's thread

2012-08-02 Thread Hari Shreedharan (JIRA)

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

Hari Shreedharan commented on FLUME-1421:
-

+1. Looks good.

> PollableSourceRunner does not name it's thread
> --
>
> Key: FLUME-1421
> URL: https://issues.apache.org/jira/browse/FLUME-1421
> Project: Flume
>  Issue Type: Bug
>Affects Versions: v1.2.0
>Reporter: Brock Noland
>Priority: Trivial
> Attachments: FLUME-1421-0.patch
>
>


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




[jira] [Updated] (FLUME-1421) PollableSourceRunner does not name it's thread

2012-08-02 Thread Brock Noland (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brock Noland updated FLUME-1421:


Affects Version/s: v1.2.0

> PollableSourceRunner does not name it's thread
> --
>
> Key: FLUME-1421
> URL: https://issues.apache.org/jira/browse/FLUME-1421
> Project: Flume
>  Issue Type: Bug
>Affects Versions: v1.2.0
>Reporter: Brock Noland
>Priority: Trivial
> Attachments: FLUME-1421-0.patch
>
>


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




[jira] [Updated] (FLUME-1421) PollableSourceRunner does not name it's thread

2012-08-02 Thread Brock Noland (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brock Noland updated FLUME-1421:


Attachment: FLUME-1421-0.patch

Attached trival patch. Not sure it needs RB.

> PollableSourceRunner does not name it's thread
> --
>
> Key: FLUME-1421
> URL: https://issues.apache.org/jira/browse/FLUME-1421
> Project: Flume
>  Issue Type: Bug
>Affects Versions: v1.2.0
>Reporter: Brock Noland
>Priority: Trivial
> Attachments: FLUME-1421-0.patch
>
>


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




[jira] [Created] (FLUME-1421) PollableSourceRunner does not name it's thread

2012-08-02 Thread Brock Noland (JIRA)
Brock Noland created FLUME-1421:
---

 Summary: PollableSourceRunner does not name it's thread
 Key: FLUME-1421
 URL: https://issues.apache.org/jira/browse/FLUME-1421
 Project: Flume
  Issue Type: Bug
Reporter: Brock Noland
Priority: Trivial




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




Re: Flume agent called 'Application' in jps

2012-08-02 Thread Harish Mandala
Thanks, Mubarak for your reply.

On Wed, Aug 1, 2012 at 2:12 PM, Mubarak Seyed  wrote:

> Hi,
>
> Please try with -m option
>
> -bash-3.2$ sudo /usr/java/latest/bin/jps -m
> 20196 Application --conf-file /etc/flume-ng/conf/flume.properties --name
> agent
>
> Thanks,
> Mubarak
>
> On Aug 1, 2012, at 11:07 AM, Harish Mandala wrote:
>
> > Hello,
> >
> > Just out of curiosity, why is the flume agent called only 'Application'
> > when I jps, and not something more descriptive?
> >
> > - Harish
>
>


[jira] [Commented] (FLUME-1393) Create a tooling framework for Flume

2012-08-02 Thread Ralph Goers (JIRA)

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

Ralph Goers commented on FLUME-1393:


There is a rather simple answer. Leave ./flume-ng as it is and create a new 
script named ./flume that does it the new way. The old script can be modified 
to call the new script if desired.

> Create a tooling framework for Flume
> 
>
> Key: FLUME-1393
> URL: https://issues.apache.org/jira/browse/FLUME-1393
> Project: Flume
>  Issue Type: Improvement
>Reporter: Patrick Wendell
>
> Flume should have a tools framework that allows for pluggable tools which can 
> be launched from the existing Flume shell. This would let developers write 
> tools that can be launched with the Flume classpath and configuration 
> information. This would look like:
> - Have a flume-ng-tools project where individual tools create sub-projects
> - A given tool sub-project would include the source code for the tool and a 
> configuration file saying (a) the full classname of the tool and (b) a short 
> name
> - There would be a outer layer runner class called by flume-ng which can be 
> called with an arbitrary short-name and will lookup and correctly load and 
> execute the tool being referenced
> - The flume-ng script would decide whether it thinks a tool is being 
> referenced in the argument and if so would delegate to the runner class 
> (otherwise, the logic stays the same for agent/version).
> This is just a first draft of how this will look - stay tuned for further 
> comments.

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




[jira] [Commented] (FLUME-390) HBase sink should handle write fails properly when using client-side HTable buffer

2012-08-02 Thread Alex Baranau (JIRA)

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

Alex Baranau commented on FLUME-390:


Should we close it in favor of new HBase sink in Flume NG?

> HBase sink should handle write fails properly when using client-side HTable 
> buffer
> --
>
> Key: FLUME-390
> URL: https://issues.apache.org/jira/browse/FLUME-390
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v0.9.4
>Reporter: Alex Baranau
>Assignee: Alex Baranau
>
> When user uses client-side writeBuffer and writing fails (e.g. in the middle 
> of the flush) currently non-written events are lost.
> Ideally sink should "inform" about non-written events to preserve 
> reliability. 

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




[jira] [Comment Edited] (FLUME-390) HBase sink should handle write fails properly when using client-side HTable buffer

2012-08-02 Thread Alex Baranau (JIRA)

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

Alex Baranau edited comment on FLUME-390 at 8/2/12 7:02 PM:


Should we close it in favor of new HBase sink (which takes care of this) in 
Flume NG?

  was (Author: alexb):
Should we close it in favor of new HBase sink in Flume NG?
  
> HBase sink should handle write fails properly when using client-side HTable 
> buffer
> --
>
> Key: FLUME-390
> URL: https://issues.apache.org/jira/browse/FLUME-390
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v0.9.4
>Reporter: Alex Baranau
>Assignee: Alex Baranau
>
> When user uses client-side writeBuffer and writing fails (e.g. in the middle 
> of the flush) currently non-written events are lost.
> Ideally sink should "inform" about non-written events to preserve 
> reliability. 

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




[jira] [Commented] (FLUME-1417) File Channel checkpoint can be bad leading to the channel being unable to start.

2012-08-02 Thread Brock Noland (JIRA)

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

Brock Noland commented on FLUME-1417:
-

Do you have the full startup log?

> File Channel checkpoint can be bad leading to the channel being unable to 
> start.
> 
>
> Key: FLUME-1417
> URL: https://issues.apache.org/jira/browse/FLUME-1417
> Project: Flume
>  Issue Type: Bug
>Reporter: Hari Shreedharan
>
>  ERROR file.Log: Failed to initialize Log on [channel=file-channel]
> java.lang.NullPointerException
>   at org.apache.flume.channel.file.Log.writeCheckpoint(Log.java:739)
>   at org.apache.flume.channel.file.Log.replay(Log.java:261)
>   at org.apache.flume.channel.file.FileChannel.start(FileChannel.java:228)
>   at 
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:228)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>   at 
> java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:317)
>   at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:150)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(ScheduledThreadPoolExecutor.java:98)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(ScheduledThreadPoolExecutor.java:181)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:205)
>   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:619)
> ERROR file.FileChannel: Failed to start the file channel 
> [channel=file-channel]
> java.lang.NullPointerException
>   at org.apache.flume.channel.file.Log.writeCheckpoint(Log.java:739)
>   at org.apache.flume.channel.file.Log.replay(Log.java:261)
>   at org.apache.flume.channel.file.FileChannel.start(FileChannel.java:228)
>   at 
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:228)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>   at 
> java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:317)
>   at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:150)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(ScheduledThreadPoolExecutor.java:98)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(ScheduledThreadPoolExecutor.java:181)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:205)
>   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:619)

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




Re: Sending avro data from other languages

2012-08-02 Thread Brock Noland
I cannot answer what made us move to Avro. However, I prefer Avro because
you don't have to build the thrift compiler and you aren't required to do
code generation.

On Wed, Aug 1, 2012 at 11:06 PM, Juhani Connolly <
juhani_conno...@cyberagent.co.jp> wrote:

> It looks to me like this was because of the transceiver I was using.
>
> Unfortunately it seems like avro doesn't have a python implementation of a
> transceiver that fits the format expected by netty/avro(in fact it only has
> one transceiver... HTTPTransceiver).
>
> To address this, I'm thinking of putting together a thrift source(the
> legacy source doesn't seem to be usable as it returns nothing, and lacks
> batching). Does this seem like a reasonable solution to making it possible
> to send data to flume from other languages(and allowing backoff on
> failure?). Historically, what made us move away from thrift to avro?
>
>
> On 07/30/2012 05:34 PM, Juhani Connolly wrote:
>
>> I'm playing around with making a standalone tail client in python(so that
>> I can access inode data) that tracks position in a file and then sends it
>> across avro to an avro sink.
>>
>> However I'm having issues with the avro part of this and wondering if
>> anyone more familiar with it could help.
>>
>> I took the flume.avdl file and converted it using "java -jar
>> ~/Downloads/avro-tools-1.6.3.**jar idl flume.avdl flume.avpr"
>>
>> I then run it through a simple test program to see if its sending the
>> data correctly and it sends from the python client fine, but the sink end
>> OOM's because presumably the wire format is wrong:
>>
>> 2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 => /172.28.19.112:41414] OPEN
>> 2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 => /172.28.19.112:41414] BOUND: /172.28.19.112:41414
>> 2012-07-30 17:22:57,565 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 => /172.28.19.112:41414] CONNECTED: /
>> 172.22.114.32:55671
>> 2012-07-30 17:22:57,646 WARN ipc.NettyServer: Unexpected exception from
>> downstream.
>> java.lang.OutOfMemoryError: Java heap space
>> at java.util.ArrayList.(**ArrayList.java:112)
>> at org.apache.avro.ipc.**NettyTransportCodec$**NettyFrameDecoder.
>> **decodePackHeader(**NettyTransportCodec.java:154)
>> at org.apache.avro.ipc.**NettyTransportCodec$**
>> NettyFrameDecoder.decode(**NettyTransportCodec.java:131)
>> at org.jboss.netty.handler.codec.**frame.FrameDecoder.callDecode(
>> **FrameDecoder.java:282)
>> at org.jboss.netty.handler.codec.**frame.FrameDecoder.**
>> messageReceived(FrameDecoder.**java:216)
>> at org.jboss.netty.channel.**Channels.fireMessageReceived(**
>> Channels.java:274)
>> at org.jboss.netty.channel.**Channels.fireMessageReceived(**
>> Channels.java:261)
>> at org.jboss.netty.channel.**socket.nio.NioWorker.read(**
>> NioWorker.java:351)
>> at org.jboss.netty.channel.**socket.nio.NioWorker.**
>> processSelectedKeys(NioWorker.**java:282)
>> at org.jboss.netty.channel.**socket.nio.NioWorker.run(**
>> NioWorker.java:202)
>> 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:619)
>> 2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 :> /172.28.19.112:41414] DISCONNECTED
>> 2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 :> /172.28.19.112:41414] UNBOUND
>> 2012-07-30 17:22:57,647 INFO ipc.NettyServer: [id: 0x5fc6e818, /
>> 172.22.114.32:55671 :> /172.28.19.112:41414] CLOSED
>>
>> I've dumped the test program and its output
>>
>> http://pastebin.com/1DtXZyTu
>> http://pastebin.com/T9kaqKHY
>>
>>
>


-- 
Apache MRUnit - Unit testing MapReduce - http://incubator.apache.org/mrunit/


[jira] [Created] (FLUME-1420) Exception should be thrown if we cannot instaniate an EventSerializer

2012-08-02 Thread Brock Noland (JIRA)
Brock Noland created FLUME-1420:
---

 Summary: Exception should be thrown if we cannot instaniate an 
EventSerializer
 Key: FLUME-1420
 URL: https://issues.apache.org/jira/browse/FLUME-1420
 Project: Flume
  Issue Type: Bug
  Components: Sinks+Sources
Affects Versions: v1.2.0
Reporter: Brock Noland


Currently EventSerailizerFactory returns null if it cannot instantiate the 
class. Then the caller NPEs because they don't expect null. If we cannot 
satisfy the caller we should throw an exception

{noformat}
2012-08-02 16:38:26,489 ERROR serialization.EventSerializerFactory: Unable to 
instantiate Builder from org.apache.flume.serialization.BodyTextEventSerializer
2012-08-02 16:38:26,490 WARN hdfs.HDFSEventSink: HDFS IO error
java.io.IOException: java.lang.NullPointerException
at org.apache.flume.sink.hdfs.BucketWriter.doOpen(BucketWriter.java:202)
at 
org.apache.flume.sink.hdfs.BucketWriter.access$000(BucketWriter.java:48)
at org.apache.flume.sink.hdfs.BucketWriter$1.run(BucketWriter.java:155)
at org.apache.flume.sink.hdfs.BucketWriter$1.run(BucketWriter.java:152)
at 
org.apache.flume.sink.hdfs.BucketWriter.runPrivileged(BucketWriter.java:125)
at org.apache.flume.sink.hdfs.BucketWriter.open(BucketWriter.java:152)
at org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:307)
at 
org.apache.flume.sink.hdfs.HDFSEventSink$1.call(HDFSEventSink.java:717)
at 
org.apache.flume.sink.hdfs.HDFSEventSink$1.call(HDFSEventSink.java:714)
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)
Caused by: java.lang.NullPointerException
at 
org.apache.flume.sink.hdfs.HDFSDataStream.open(HDFSDataStream.java:75)
at org.apache.flume.sink.hdfs.BucketWriter.doOpen(BucketWriter.java:188)
... 13 more
{noformat}

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




[jira] Inder SIngh shared "FLUME-1415: Update Developer FAQ to build on MACOS" with you

2012-08-02 Thread Inder SIngh (JIRA)
Inder SIngh just shared FLUME-1415 with you




> Update Developer FAQ to build on MACOS
> --
>
> Key: FLUME-1415
> URL: https://issues.apache.org/jira/browse/FLUME-1415
> Project: Flume
>  Issue Type: Bug
>  Components: Build
>Reporter: Inder SIngh
>Assignee: Inder SIngh
>Priority: Minor
>
> I am creating this ticket to update the developer FAQ to ease the build 
> process due sphinx on MAC.
> we had discussed this on the dev thread a while back but it got lost with 
> some other conversations...
> if you hit the following exceptions -> Details at -> 
> http://pastebin.com/c1STnMaJ
> Failed to execute goal org.apache.maven.plugins:maven-site-plugin:3.1:site 
> (flume-site) on project flume-parent: Error during page generation: Error 
> rendering Maven report: Could not generate documentation: ValueError: unknown 
> locale: UTF-8 in 

Re: Review Request: Improvement for Log4j configuration

2012-08-02 Thread Mike Percy

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


The INFO Hadoop setting is certainly needed. Also, including the thread name is 
great. However I have heard that DailyRollingFileAppender is unreliable and can 
cause data loss at roll time. What's the reason behind that change?

- Mike Percy


On Aug. 2, 2012, 7 a.m., Denny Ye wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/6316/
> ---
> 
> (Updated Aug. 2, 2012, 7 a.m.)
> 
> 
> Review request for Flume and Hari Shreedharan.
> 
> 
> Description
> ---
> 
> Updated log4j.properties 
> 1. Add 'logs' folder. It's better to aggregate logs into unified folder.
> 2. Log file using DailyRollingFileAppender
> 3. More useful log pattern. Add thread and code line information
> 4. Set debug level for Hadoop. If we use debug level, Hadoop log takes too 
> many lines.
> 
> 
> This addresses bug https://issues.apache.org/jira/browse/FLUME-1418.
> 
> https://issues.apache.org/jira/browse/https://issues.apache.org/jira/browse/FLUME-1418
> 
> 
> Diffs
> -
> 
>   trunk/conf/log4j.properties 1363210 
> 
> Diff: https://reviews.apache.org/r/6316/diff/
> 
> 
> Testing
> ---
> 
> That's OK in my environment
> 
> 
> Thanks,
> 
> Denny Ye
> 
>



[jira] [Updated] (FLUME-1419) Using system time if 'timestamp' property is absent in event header

2012-08-02 Thread Denny Ye (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denny Ye updated FLUME-1419:


Attachment: (was: FLUME-1418.patch)

> Using system time if 'timestamp' property is absent in event header
> ---
>
> Key: FLUME-1419
> URL: https://issues.apache.org/jira/browse/FLUME-1419
> Project: Flume
>  Issue Type: Improvement
>  Components: Sinks+Sources
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
> Fix For: v1.3.0
>
> Attachments: FLUME-1419.patch
>
>
> If we want to use pattern for generating HDFS path but the 'timestamp' 
> property is absent in event header, it may cause the exception...
> Event may comes from several Sources and doesn't use Interceptor(pupulate 
> 'timestamp' property). Thus, event may be haven't this property normally.
> Use the local system time to replace it.
> java.lang.RuntimeException: Flume wasn't able to parse timestamp header in 
> the event to resolve time based bucketing. Please check that you're correctly 
> populating timestamp header (for example using TimestampInterceptor source 
> interceptor).
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:149)
>   at 
> org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:318)
>   at 
> org.apache.flume.formatter.output.TestBucketPath.testDateFormatHours(TestBucketPath.java:46)
> Caused by: java.lang.NumberFormatException: null
>   at java.lang.Long.parseLong(Unknown Source)
>   at java.lang.Long.valueOf(Unknown Source)
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:147)
>   ... 26 more

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




[jira] [Updated] (FLUME-1419) Using system time if 'timestamp' property is absent in event header

2012-08-02 Thread Denny Ye (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denny Ye updated FLUME-1419:


Attachment: FLUME-1419.patch

> Using system time if 'timestamp' property is absent in event header
> ---
>
> Key: FLUME-1419
> URL: https://issues.apache.org/jira/browse/FLUME-1419
> Project: Flume
>  Issue Type: Improvement
>  Components: Sinks+Sources
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
> Fix For: v1.3.0
>
> Attachments: FLUME-1419.patch
>
>
> If we want to use pattern for generating HDFS path but the 'timestamp' 
> property is absent in event header, it may cause the exception...
> Event may comes from several Sources and doesn't use Interceptor(pupulate 
> 'timestamp' property). Thus, event may be haven't this property normally.
> Use the local system time to replace it.
> java.lang.RuntimeException: Flume wasn't able to parse timestamp header in 
> the event to resolve time based bucketing. Please check that you're correctly 
> populating timestamp header (for example using TimestampInterceptor source 
> interceptor).
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:149)
>   at 
> org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:318)
>   at 
> org.apache.flume.formatter.output.TestBucketPath.testDateFormatHours(TestBucketPath.java:46)
> Caused by: java.lang.NumberFormatException: null
>   at java.lang.Long.parseLong(Unknown Source)
>   at java.lang.Long.valueOf(Unknown Source)
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:147)
>   ... 26 more

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




[jira] [Updated] (FLUME-1419) Using system time if 'timestamp' property is absent in event header

2012-08-02 Thread Denny Ye (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denny Ye updated FLUME-1419:


Attachment: FLUME-1418.patch

> Using system time if 'timestamp' property is absent in event header
> ---
>
> Key: FLUME-1419
> URL: https://issues.apache.org/jira/browse/FLUME-1419
> Project: Flume
>  Issue Type: Improvement
>  Components: Sinks+Sources
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
> Fix For: v1.3.0
>
> Attachments: FLUME-1418.patch
>
>
> If we want to use pattern for generating HDFS path but the 'timestamp' 
> property is absent in event header, it may cause the exception...
> Event may comes from several Sources and doesn't use Interceptor(pupulate 
> 'timestamp' property). Thus, event may be haven't this property normally.
> Use the local system time to replace it.
> java.lang.RuntimeException: Flume wasn't able to parse timestamp header in 
> the event to resolve time based bucketing. Please check that you're correctly 
> populating timestamp header (for example using TimestampInterceptor source 
> interceptor).
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:149)
>   at 
> org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:318)
>   at 
> org.apache.flume.formatter.output.TestBucketPath.testDateFormatHours(TestBucketPath.java:46)
> Caused by: java.lang.NumberFormatException: null
>   at java.lang.Long.parseLong(Unknown Source)
>   at java.lang.Long.valueOf(Unknown Source)
>   at 
> org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:147)
>   ... 26 more

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




[jira] [Created] (FLUME-1419) Using system time if 'timestamp' property is absent in event header

2012-08-02 Thread Denny Ye (JIRA)
Denny Ye created FLUME-1419:
---

 Summary: Using system time if 'timestamp' property is absent in 
event header
 Key: FLUME-1419
 URL: https://issues.apache.org/jira/browse/FLUME-1419
 Project: Flume
  Issue Type: Improvement
  Components: Sinks+Sources
Affects Versions: v1.2.0
Reporter: Denny Ye
Assignee: Denny Ye
 Fix For: v1.3.0


If we want to use pattern for generating HDFS path but the 'timestamp' property 
is absent in event header, it may cause the exception...

Event may comes from several Sources and doesn't use Interceptor(pupulate 
'timestamp' property). Thus, event may be haven't this property normally.
Use the local system time to replace it.

java.lang.RuntimeException: Flume wasn't able to parse timestamp header in the 
event to resolve time based bucketing. Please check that you're correctly 
populating timestamp header (for example using TimestampInterceptor source 
interceptor).
at 
org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:149)
at 
org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:318)
at 
org.apache.flume.formatter.output.TestBucketPath.testDateFormatHours(TestBucketPath.java:46)
Caused by: java.lang.NumberFormatException: null
at java.lang.Long.parseLong(Unknown Source)
at java.lang.Long.valueOf(Unknown Source)
at 
org.apache.flume.formatter.output.BucketPath.replaceShorthand(BucketPath.java:147)
... 26 more

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




Re: flume-ng-core compiling exception

2012-08-02 Thread Juhani Connolly
Since I presume you're not developpping this custom source for 
contribution, you might want to just compile your own additions 
separately from the main project. You can add them to the classpath in 
flume-env.sh and then add them to the configuration by the classname.


On 08/02/2012 03:10 PM, JP wrote:

After adding this licence in my custom class it is working fine

/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * regarding copyright ownership.  The ASF licenses this file
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
  */

On Thu, Aug 2, 2012 at 9:53 AM, JP  wrote:




Hi,

Can you please tell me where im going wrong.


I have added the attached CustomAvroSerializer  class to send avro source
to the hdfs sink.

after im trying to compile im getting the following exception.

Even, If i try to compile parent also  im getting:
org.apache.maven.BuildFailureException: Too many unapproved licenses: 3
exception.


Please suggest me how can i over come this problem.

Im getting the following exception:


DEBUG] Trace
org.apache.maven.BuildFailureException: Too many unapproved licenses: 3
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeGoals(DefaultLifecycleExecutor.java:715)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeGoalWithLifecycle(DefaultLifecycleExecutor.java:556)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeGoal(DefaultLifecycleExecutor.java:535)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeGoalAndHandleFailures(DefaultLifecycleExecutor.java:387)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeTaskSegments(DefaultLifecycleExecutor.java:348)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.execute(DefaultLifecycleExecutor.java:180)
 at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:328)
 at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:138)
 at org.apache.maven.cli.MavenCli.main(MavenCli.java:362)
 at
org.apache.maven.cli.compat.CompatibleMain.main(CompatibleMain.java:60)
 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
 at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
 at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
 at java.lang.reflect.Method.invoke(Method.java:597)
 at org.codehaus.classworlds.Launcher.launchEnhanced(Launcher.java:315)
 at org.codehaus.classworlds.Launcher.launch(Launcher.java:255)
 at
org.codehaus.classworlds.Launcher.mainWithExitCode(Launcher.java:430)
 at org.codehaus.classworlds.Launcher.main(Launcher.java:375)
Caused by: org.apache.rat.mp.RatCheckException: Too many unapproved
licenses: 3
 at org.apache.rat.mp.RatCheckMojo.check(RatCheckMojo.java:167)
 at org.apache.rat.mp.RatCheckMojo.execute(RatCheckMojo.java:159)
 at
org.apache.maven.plugin.DefaultPluginManager.executeMojo(DefaultPluginManager.java:490)
 at
org.apache.maven.lifecycle.DefaultLifecycleExecutor.executeGoals(DefaultLifecycleExecutor.java:694)
 ... 17 more
[INFO]

[INFO] Total time: 4 minutes 43 seconds
[INFO] Finished at: Wed Aug 01 23:06:08 IST 2012
[INFO] Final Memory: 33M/80M
[INFO]


--
JP



--
JP








Re: Deleting branch master on git-wip-us

2012-08-02 Thread Hari Shreedharan
Seems like it is not possible to delete master. I will just update the website 
mentioning this. 

Thanks
Hari

-- 
Hari Shreedharan


On Thursday, August 2, 2012 at 12:28 AM, Hari Shreedharan wrote:

> It seems like a branch was created which has the same commits as trunk. I am 
> deleting the branch, to avoid confusion of that being the main development 
> branch - as in git convention. 
> 
> Thanks,
> Hari
> 
> -- 
> Hari Shreedharan
> 



[jira] [Commented] (FLUME-1416) Version Info should have hardcoded git repo address

2012-08-02 Thread Mubarak Seyed (JIRA)

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

Mubarak Seyed commented on FLUME-1416:
--

Committed. Thanks for the patch Hari.

> Version Info should have hardcoded git repo address
> ---
>
> Key: FLUME-1416
> URL: https://issues.apache.org/jira/browse/FLUME-1416
> Project: Flume
>  Issue Type: Bug
>Reporter: Hari Shreedharan
>Assignee: Hari Shreedharan
> Fix For: v1.3.0
>
> Attachments: FLUME-1416.patch
>
>


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




[jira] [Resolved] (FLUME-1416) Version Info should have hardcoded git repo address

2012-08-02 Thread Mubarak Seyed (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Mubarak Seyed resolved FLUME-1416.
--

   Resolution: Fixed
Fix Version/s: v1.3.0

> Version Info should have hardcoded git repo address
> ---
>
> Key: FLUME-1416
> URL: https://issues.apache.org/jira/browse/FLUME-1416
> Project: Flume
>  Issue Type: Bug
>Reporter: Hari Shreedharan
>Assignee: Hari Shreedharan
> Fix For: v1.3.0
>
> Attachments: FLUME-1416.patch
>
>


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




Deleting branch master on git-wip-us

2012-08-02 Thread Hari Shreedharan
It seems like a branch was created which has the same commits as trunk. I am 
deleting the branch, to avoid confusion of that being the main development 
branch - as in git convention. 

Thanks,
Hari

-- 
Hari Shreedharan



[jira] [Updated] (FLUME-1418) Improvement for Log4j configuration

2012-08-02 Thread Denny Ye (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-1418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denny Ye updated FLUME-1418:


Attachment: FLUME-1418.patch

> Improvement for Log4j configuration
> ---
>
> Key: FLUME-1418
> URL: https://issues.apache.org/jira/browse/FLUME-1418
> Project: Flume
>  Issue Type: Improvement
>  Components: Build
>Affects Versions: v1.2.0
>Reporter: Denny Ye
>Assignee: Denny Ye
>  Labels: log4j
> Fix For: v1.3.0
>
> Attachments: FLUME-1418.patch
>
>
> Updated log4j.properties 
> 1. Add 'logs' folder. It's better to aggregate logs into unified folder.
> 2. Log file using DailyRollingFileAppender
> 3. More useful log pattern. Add thread and code line information
> 4. Set debug level for Hadoop. If we use debug level, Hadoop log takes too 
> many lines.

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




Re: Review Request: Improvement for Log4j configuration

2012-08-02 Thread Denny Ye

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

(Updated Aug. 2, 2012, 7 a.m.)


Review request for Flume and Hari Shreedharan.


Changes
---

Remove the tailing whitespace


Description
---

Updated log4j.properties 
1. Add 'logs' folder. It's better to aggregate logs into unified folder.
2. Log file using DailyRollingFileAppender
3. More useful log pattern. Add thread and code line information
4. Set debug level for Hadoop. If we use debug level, Hadoop log takes too many 
lines.


This addresses bug https://issues.apache.org/jira/browse/FLUME-1418.

https://issues.apache.org/jira/browse/https://issues.apache.org/jira/browse/FLUME-1418


Diffs (updated)
-

  trunk/conf/log4j.properties 1363210 

Diff: https://reviews.apache.org/r/6316/diff/


Testing
---

That's OK in my environment


Thanks,

Denny Ye