Updated Branches:
  refs/heads/flume-1.3.0 dc99a7455 -> 5ad522c1e

FLUME-1671: Add support for custom components to MonitoredCounterGroup

(Hari Shreedharan via Brock Noland)


Project: http://git-wip-us.apache.org/repos/asf/flume/repo
Commit: http://git-wip-us.apache.org/repos/asf/flume/commit/5ad522c1
Tree: http://git-wip-us.apache.org/repos/asf/flume/tree/5ad522c1
Diff: http://git-wip-us.apache.org/repos/asf/flume/diff/5ad522c1

Branch: refs/heads/flume-1.3.0
Commit: 5ad522c1e392dcbb07aca5d0015e5302ce29a203
Parents: dc99a74
Author: Brock Noland <[email protected]>
Authored: Wed Oct 31 14:02:15 2012 -0500
Committer: Brock Noland <[email protected]>
Committed: Wed Oct 31 14:02:36 2012 -0500

----------------------------------------------------------------------
 .../instrumentation/MonitoredCounterGroup.java     |    5 +-
 flume-ng-doc/sphinx/FlumeUserGuide.rst             |   72 ++++++++++++--
 2 files changed, 65 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flume/blob/5ad522c1/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java
----------------------------------------------------------------------
diff --git 
a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java
 
b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java
index 6bc31ef..1d0c3ce 100644
--- 
a/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java
+++ 
b/flume-ng-core/src/main/java/org/apache/flume/instrumentation/MonitoredCounterGroup.java
@@ -137,7 +137,10 @@ public abstract class MonitoredCounterGroup {
     CHANNEL_PROCESSOR,
     CHANNEL,
     SINK_PROCESSOR,
-    SINK
+    SINK,
+    INTERCEPTOR,
+    SERIALIZER,
+    OTHER
   };
 
   public String getType(){

http://git-wip-us.apache.org/repos/asf/flume/blob/5ad522c1/flume-ng-doc/sphinx/FlumeUserGuide.rst
----------------------------------------------------------------------
diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst 
b/flume-ng-doc/sphinx/FlumeUserGuide.rst
index d62e6b9..08ef220 100644
--- a/flume-ng-doc/sphinx/FlumeUserGuide.rst
+++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst
@@ -2037,17 +2037,6 @@ We can start Flume with Ganglia support as follows::
 
   $ bin/flume-ng agent --conf-file example.conf --name agent1 
-Dflume.monitoring.type=GANGLIA 
-Dflume.monitoring.hosts=com.example:1234,com.example2:5455
 
-Any custom flume components should use Java MBean ObjectNames which begin
-with ``org.apache.flume`` for Flume to report the metrics to Ganglia. This can
-be done by adding the ObjectName as follows(the name can be anything provided 
it
-starts with ``org.apache.flume``):
-
-.. code-block:: java
-
-  ObjectName objName = new ObjectName("org.apache.flume." + myClassName + 
":type=" + name);
-
-  ManagementFactory.getPlatformMBeanServer().registerMBean(this, objName);
-
 JSON Reporting
 --------------
 Flume can also report metrics in a JSON format. To enable reporting in JSON 
format, Flume hosts
@@ -2116,6 +2105,67 @@ Property Name            Default  Description
 **type**                 --       The component type name, has to be FQCN
 =======================  =======  ========================================
 
+Reporting metrics from custom components
+----------------------------------------
+Any custom flume components should inherit from the
+``org.apache.flume.instrumentation.MonitoredCounterGroup`` class. The class
+should then provide getter methods for each of the metrics it exposes. See
+the code below. The MonitoredCounterGroup expects a list of attributes whose
+metrics are exposed by this class. As of now, this class only supports exposing
+metrics as long values.
+
+.. code-block:: java
+
+  public class SinkCounter extends MonitoredCounterGroup implements
+      SinkCounterMBean {
+
+    private static final String COUNTER_CONNECTION_CREATED =
+      "sink.connection.creation.count";
+
+    private static final String COUNTER_CONNECTION_CLOSED =
+      "sink.connection.closed.count";
+
+    private static final String COUNTER_CONNECTION_FAILED =
+      "sink.connection.failed.count";
+
+    private static final String COUNTER_BATCH_EMPTY =
+      "sink.batch.empty";
+
+    private static final String COUNTER_BATCH_UNDERFLOW =
+        "sink.batch.underflow";
+
+    private static final String COUNTER_BATCH_COMPLETE =
+      "sink.batch.complete";
+
+    private static final String COUNTER_EVENT_DRAIN_ATTEMPT =
+      "sink.event.drain.attempt";
+
+    private static final String COUNTER_EVENT_DRAIN_SUCCESS =
+      "sink.event.drain.sucess";
+
+    private static final String[] ATTRIBUTES = {
+      COUNTER_CONNECTION_CREATED, COUNTER_CONNECTION_CLOSED,
+      COUNTER_CONNECTION_FAILED, COUNTER_BATCH_EMPTY,
+      COUNTER_BATCH_UNDERFLOW, COUNTER_BATCH_COMPLETE,
+      COUNTER_EVENT_DRAIN_ATTEMPT, COUNTER_EVENT_DRAIN_SUCCESS
+    };
+
+
+    public SinkCounter(String name) {
+      super(MonitoredCounterGroup.Type.SINK, name, ATTRIBUTES);
+    }
+
+    @Override
+    public long getConnectionCreatedCount() {
+      return get(COUNTER_CONNECTION_CREATED);
+    }
+
+    public long incrementConnectionCreatedCount() {
+      return increment(COUNTER_CONNECTION_CREATED);
+    }
+
+  }
+
 
 Topology Design Considerations
 ==============================

Reply via email to