I think it is best to replace the current system rather than having multiple 
metrics systems. We should implement it such that it is transparent to user 
code, and the changes only go into MonitoredCounterGroup (the CounterGroup 
class is obsolete - dont bother).


Thanks,
Hari

On Mon, Nov 10, 2014 at 5:36 PM, Ashish <[email protected]> wrote:

> Codahale does have lot more features like the rate/sec one, which is
> most needed metrics. It provides a lot of reporters out of box
> (Nagios, HTTP, Ganglia, Graphite, Console etc) so we just need to
> enable it, rather than writing custom components.
> As of now we have to hide it within MonitoredCounterGroup and its
> sub-classes, that's what I did for running it. It was running parallel
> to existing system. Let me work more on it and see if I can simplify
> the same.
> On Tue, Nov 11, 2014 at 4:50 AM, Hari Shreedharan
> <[email protected]> wrote:
>> Is it easier to use than the current one and/or does it give better 
>> performance? You’d need to support the current metrics API 
>> (MonitoredCounterGroup, SourceCounter, SinkCounter, ChannelCounter etc).
>>
>>
>> Thanks,
>> Hari
>>
>> On Sat, Nov 8, 2014 at 8:21 PM, Ashish <[email protected]> wrote:
>>
>>> Hi,
>>> Have hacked a bit into our existing instrumentation package and piggy
>>> backed cohahale metrics package. Here is one sample for Spooled
>>> Directory source (with instrumentation only for Source and Channel ),
>>> using console reporter
>>> -- Gauges 
>>> ----------------------------------------------------------------------
>>> org.apache.flume.instrumentation.ChannelCounter.channel.current.size
>>>              value = 200
>>> org.apache.flume.instrumentation.ChannelCounter.channel.fill.percentage
>>>              value = 2.0
>>> org.apache.flume.instrumentation.SourceCounter.src.open-connection.count
>>>              value = 0
>>> -- Counters 
>>> --------------------------------------------------------------------
>>> org.apache.flume.instrumentation.ChannelCounter.channel.event.put.attempt
>>>              count = 1138800
>>> org.apache.flume.instrumentation.ChannelCounter.channel.event.put.success
>>>              count = 1138800
>>> org.apache.flume.instrumentation.ChannelCounter.channel.event.take.attempt
>>>              count = 1138601
>>> org.apache.flume.instrumentation.ChannelCounter.channel.event.take.success
>>>              count = 1138600
>>> org.apache.flume.instrumentation.SourceCounter.src.events.accepted
>>>              count = 1138800
>>> org.apache.flume.instrumentation.SourceCounter.src.events.received
>>>              count = 1138800
>>> src.append-batch.accepted
>>>              count = 11388
>>> src.append-batch.received
>>>              count = 11388
>>> src.append.accepted
>>>              count = 0
>>> src.append.received
>>>              count = 0
>>> -- Meters 
>>> ----------------------------------------------------------------------
>>> eventAcceptedMeter
>>>              count = 1138800
>>>          mean rate = 106478.56 events/second
>>>      1-minute rate = 93420.18 events/second
>>>      5-minute rate = 91389.24 events/second
>>>     15-minute rate = 91037.40 events/second
>>> eventReceivedMeter
>>>              count = 1138800
>>>          mean rate = 106462.14 events/second
>>>      1-minute rate = 93420.18 events/second
>>>      5-minute rate = 91389.24 events/second
>>>     15-minute rate = 91037.40 events/second
>>> If there is interest in the community, can raise a jira and continue
>>> to work on it.
>>> --
>>> thanks
>>> ashish
>>> Blog: http://www.ashishpaliwal.com/blog
>>> My Photo Galleries: http://www.pbase.com/ashishpaliwal
> -- 
> thanks
> ashish
> Blog: http://www.ashishpaliwal.com/blog
> My Photo Galleries: http://www.pbase.com/ashishpaliwal

Reply via email to