You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2009/04/08 00:00:07 UTC

svn commit: r762987 - in /hadoop/core/trunk: ./ src/core/org/apache/hadoop/http/ src/core/org/apache/hadoop/metrics/ src/core/org/apache/hadoop/metrics/file/ src/core/org/apache/hadoop/metrics/ganglia/ src/core/org/apache/hadoop/metrics/spi/ src/test/o...

Author: cutting
Date: Tue Apr  7 22:00:07 2009
New Revision: 762987

URL: http://svn.apache.org/viewvc?rev=762987&view=rev
Log:
HADOOP-5469.  Add /metrics servlet to daemons, providing metrics over HTTP as either text or JSON.  Contributed by Philip Zeyliger.

Added:
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsServlet.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java
    hadoop/core/trunk/src/test/org/apache/hadoop/metrics/
    hadoop/core/trunk/src/test/org/apache/hadoop/metrics/TestMetricsServlet.java
    hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/
    hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/TestOutputRecord.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/core/org/apache/hadoop/http/HttpServer.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ContextFactory.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsContext.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/file/FileContext.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ganglia/GangliaContext.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java
    hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/OutputRecord.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue Apr  7 22:00:07 2009
@@ -80,6 +80,9 @@
     HADOOP-5518. Add contrib/mrunit, a MapReduce unit test framework.
     (Aaron Kimball via cutting)
 
+    HADOOP-5469.  Add /metrics servlet to daemons, providing metrics
+    over HTTP as either text or JSON.  (Philip Zeyliger via cutting)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/http/HttpServer.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/http/HttpServer.java Tue Apr  7 22:00:07 2009
@@ -37,6 +37,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.log.LogLevel;
+import org.apache.hadoop.metrics.MetricsServlet;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import org.mortbay.jetty.Connector;
@@ -188,6 +189,7 @@
     // set up default servlets
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
+    addServlet("metrics", "/metrics", MetricsServlet.class);
   }
 
   public void addContext(Context ctxt, boolean isFiltered)

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ContextFactory.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ContextFactory.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ContextFactory.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ContextFactory.java Tue Apr  7 22:00:07 2009
@@ -22,6 +22,8 @@
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -142,6 +144,14 @@
            IllegalAccessException {
     return getContext(contextName, contextName);
   }
+  
+  /** 
+   * Returns all MetricsContexts built by this factory.
+   */
+  public synchronized Collection<MetricsContext> getAllContexts() {
+    // Make a copy to avoid race conditions with creating new contexts.
+    return new ArrayList<MetricsContext>(contextMap.values());
+  }
     
   /**
    * Returns a "null" context - one which does nothing.

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsContext.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsContext.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsContext.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsContext.java Tue Apr  7 22:00:07 2009
@@ -21,6 +21,10 @@
 package org.apache.hadoop.metrics;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.metrics.spi.OutputRecord;
 
 /**
  * The main interface to the metrics package. 
@@ -103,5 +107,12 @@
    * Returns the timer period.
    */
   public abstract int getPeriod();
-    
+  
+  /**
+   * Retrieves all the records managed by this MetricsContext.
+   * Useful for monitoring systems that are polling-based.
+   * 
+   * @return A non-null map from all record names to the records managed.
+   */
+   Map<String, Collection<OutputRecord>> getAllRecords();
 }

Added: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsServlet.java?rev=762987&view=auto
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsServlet.java (added)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/MetricsServlet.java Tue Apr  7 22:00:07 2009
@@ -0,0 +1,160 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.metrics;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.metrics.spi.OutputRecord;
+import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
+import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
+import org.mortbay.util.ajax.JSON;
+import org.mortbay.util.ajax.JSON.Output;
+
+/**
+ * A servlet to print out metrics data.  By default, the servlet returns a 
+ * textual representation (no promises are made for parseability), and
+ * users can use "?format=json" for parseable output.
+ */
+public class MetricsServlet extends HttpServlet {
+  
+  /**
+   * A helper class to hold a TagMap and MetricMap.
+   */
+  static class TagsMetricsPair implements JSON.Convertible {
+    final TagMap tagMap;
+    final MetricMap metricMap;
+    
+    public TagsMetricsPair(TagMap tagMap, MetricMap metricMap) {
+      this.tagMap = tagMap;
+      this.metricMap = metricMap;
+    }
+
+    @SuppressWarnings("unchecked")
+    public void fromJSON(Map map) {
+      throw new UnsupportedOperationException();
+    }
+
+    /** Converts to JSON by providing an array. */
+    public void toJSON(Output out) {
+      out.add(new Object[] { tagMap, metricMap });
+    }
+  }
+  
+  /**
+   * Collects all metric data, and returns a map:
+   *   contextName -> recordName -> [ (tag->tagValue), (metric->metricValue) ].
+   * The values are either String or Number.  The final value is implemented
+   * as a list of TagsMetricsPair.
+   */
+   Map<String, Map<String, List<TagsMetricsPair>>> makeMap(
+       Collection<MetricsContext> contexts) throws IOException {
+    Map<String, Map<String, List<TagsMetricsPair>>> map = 
+      new TreeMap<String, Map<String, List<TagsMetricsPair>>>();
+
+    for (MetricsContext context : contexts) {
+      Map<String, List<TagsMetricsPair>> records = 
+        new TreeMap<String, List<TagsMetricsPair>>();
+      map.put(context.getContextName(), records);
+    
+      for (Map.Entry<String, Collection<OutputRecord>> r : 
+          context.getAllRecords().entrySet()) {
+        List<TagsMetricsPair> metricsAndTags = 
+          new ArrayList<TagsMetricsPair>();
+        records.put(r.getKey(), metricsAndTags);
+        for (OutputRecord outputRecord : r.getValue()) {
+          TagMap tagMap = outputRecord.getTagsCopy();
+          MetricMap metricMap = outputRecord.getMetricsCopy();
+          metricsAndTags.add(new TagsMetricsPair(tagMap, metricMap));
+        }
+      }
+    }
+    return map;
+  }
+  
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+    PrintWriter out = new PrintWriter(response.getOutputStream());
+    String format = request.getParameter("format");
+    Collection<MetricsContext> allContexts = 
+      ContextFactory.getFactory().getAllContexts();
+    if ("json".equals(format)) {
+      // Uses Jetty's built-in JSON support to convert the map into JSON.
+      out.print(new JSON().toJSON(makeMap(allContexts)));
+    } else {
+      printMap(out, makeMap(allContexts));
+    }
+    out.close();
+  }
+  
+  /**
+   * Prints metrics data in a multi-line text form.
+   */
+  void printMap(PrintWriter out, Map<String, Map<String, List<TagsMetricsPair>>> map) {
+    for (Map.Entry<String, Map<String, List<TagsMetricsPair>>> context : map.entrySet()) {
+      out.println(context.getKey());
+      for (Map.Entry<String, List<TagsMetricsPair>> record : context.getValue().entrySet()) {
+        indent(out, 1);
+        out.println(record.getKey());
+        for (TagsMetricsPair pair : record.getValue()) {
+          indent(out, 2);
+          // Prints tag values in the form "{key=value,key=value}:"
+          out.print("{");
+          boolean first = true;
+          for (Map.Entry<String, Object> tagValue : pair.tagMap.entrySet()) {
+            if (first) {
+              first = false;
+            } else {
+              out.print(",");
+            }
+            out.print(tagValue.getKey());
+            out.print("=");
+            out.print(tagValue.getValue().toString());
+          }
+          out.println("}:");
+          
+          // Now print metric values, one per line
+          for (Map.Entry<String, Number> metricValue : 
+              pair.metricMap.entrySet()) {
+            indent(out, 3);
+            out.print(metricValue.getKey());
+            out.print("=");
+            out.println(metricValue.getValue().toString());
+          }
+        }
+      }
+    }    
+  }
+  
+  private void indent(PrintWriter out, int indent) {
+    for (int i = 0; i < indent; ++i) {
+      out.append("  ");
+    }
+  }
+}

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/file/FileContext.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/file/FileContext.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/file/FileContext.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/file/FileContext.java Tue Apr  7 22:00:07 2009
@@ -27,7 +27,6 @@
 import java.io.PrintWriter;
 
 import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsException;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext;
 import org.apache.hadoop.metrics.spi.OutputRecord;
 
@@ -62,18 +61,7 @@
       file = new File(fileName);
     }
         
-    String periodStr = getAttribute(PERIOD_PROPERTY);
-    if (periodStr != null) {
-      int period = 0;
-      try {
-        period = Integer.parseInt(periodStr);
-      } catch (NumberFormatException nfe) {
-      }
-      if (period <= 0) {
-        throw new MetricsException("Invalid period: " + periodStr);
-      }
-      setPeriod(period);
-    }
+    parseAndSetPeriod(PERIOD_PROPERTY);
   }
 
   /**

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ganglia/GangliaContext.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ganglia/GangliaContext.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ganglia/GangliaContext.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/ganglia/GangliaContext.java Tue Apr  7 22:00:07 2009
@@ -88,19 +88,7 @@
     
   public void init(String contextName, ContextFactory factory) {
     super.init(contextName, factory);
-        
-    String periodStr = getAttribute(PERIOD_PROPERTY);
-    if (periodStr != null) {
-      int period = 0;
-      try {
-        period = Integer.parseInt(periodStr);
-      } catch (NumberFormatException nfe) {
-      }
-      if (period <= 0) {
-        throw new MetricsException("Invalid period: " + periodStr);
-      }
-      setPeriod(period);
-    }
+    parseAndSetPeriod(PERIOD_PROPERTY);
         
     metricsServers = 
       Util.parse(getAttribute(SERVERS_PROPERTY), DEFAULT_PORT); 

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java Tue Apr  7 22:00:07 2009
@@ -26,6 +26,7 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Timer;
@@ -60,7 +61,7 @@
   private ContextFactory factory = null;
   private String contextName = null;
     
-  static class TagMap extends TreeMap<String,Object> {
+  public static class TagMap extends TreeMap<String,Object> {
     private static final long serialVersionUID = 3546309335061952993L;
     TagMap() {
       super();
@@ -83,8 +84,14 @@
     }
   }
   
-  static class MetricMap extends TreeMap<String,Number> {
+  public static class MetricMap extends TreeMap<String,Number> {
     private static final long serialVersionUID = -7495051861141631609L;
+    MetricMap() {
+      super();
+    }
+    MetricMap(MetricMap orig) {
+      super(orig);
+    }
   }
             
   static class RecordMap extends HashMap<TagMap,MetricMap> {
@@ -309,6 +316,28 @@
     }
     flush();
   }
+  
+  /**
+   * Retrieves all the records managed by this MetricsContext.
+   * Useful for monitoring systems that are polling-based.
+   * @return A non-null collection of all monitoring records.
+   */
+  public synchronized Map<String, Collection<OutputRecord>> getAllRecords() {
+    Map<String, Collection<OutputRecord>> out = new TreeMap<String, Collection<OutputRecord>>();
+    for (String recordName : bufferedData.keySet()) {
+      RecordMap recordMap = bufferedData.get(recordName);
+      synchronized (recordMap) {
+        List<OutputRecord> records = new ArrayList<OutputRecord>();
+        Set<Entry<TagMap, MetricMap>> entrySet = recordMap.entrySet();
+        for (Entry<TagMap, MetricMap> entry : entrySet) {
+          OutputRecord outRec = new OutputRecord(entry.getKey(), entry.getValue());
+          records.add(outRec);
+        }
+        out.put(recordName, records);
+      }
+    }
+    return out;
+  }
 
   /**
    * Sends a record to the metrics system.
@@ -424,4 +453,23 @@
   protected void setPeriod(int period) {
     this.period = period;
   }
+  
+  /**
+   * If a period is set in the attribute passed in, override
+   * the default with it.
+   */
+  protected void parseAndSetPeriod(String attributeName) {
+    String periodStr = getAttribute(attributeName);
+    if (periodStr != null) {
+      int period = 0;
+      try {
+        period = Integer.parseInt(periodStr);
+      } catch (NumberFormatException nfe) {
+      }
+      if (period <= 0) {
+        throw new MetricsException("Invalid period: " + periodStr);
+      }
+      setPeriod(period);
+    }
+  }
 }

Added: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java?rev=762987&view=auto
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java (added)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java Tue Apr  7 22:00:07 2009
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.metrics.spi;
+
+import org.apache.hadoop.metrics.ContextFactory;
+import org.apache.hadoop.metrics.MetricsServlet;
+
+/** 
+ * A MetricsContext that does not emit data, but, unlike NullContextWithUpdate,
+ * does save it for retrieval with getAllRecords().
+ * 
+ * This is useful if you want to support {@link MetricsServlet}, but
+ * not emit metrics in any other way.
+ */
+public class NoEmitMetricsContext extends AbstractMetricsContext {
+    
+    private static final String PERIOD_PROPERTY = "period";
+      
+    /** Creates a new instance of NullContextWithUpdateThread */
+    public NoEmitMetricsContext() {
+    }
+    
+    public void init(String contextName, ContextFactory factory) {
+      super.init(contextName, factory);
+      parseAndSetPeriod(PERIOD_PROPERTY);
+    }
+     
+    /**
+     * Do-nothing version of emitRecord
+     */
+    protected void emitRecord(String contextName, String recordName,
+                              OutputRecord outRec) {
+    }
+}

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java Tue Apr  7 22:00:07 2009
@@ -44,21 +44,7 @@
   
   public void init(String contextName, ContextFactory factory) {
     super.init(contextName, factory);
-    
-    // If period is specified, use it, otherwise the default is good enough
-        
-    String periodStr = getAttribute(PERIOD_PROPERTY);
-    if (periodStr != null) {
-      int period = 0;
-      try {
-        period = Integer.parseInt(periodStr);
-      } catch (NumberFormatException nfe) {
-      }
-      if (period <= 0) {
-        throw new MetricsException("Invalid period: " + periodStr);
-      }
-      setPeriod(period);
-    }
+    parseAndSetPeriod(PERIOD_PROPERTY);
   }
    
     

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/OutputRecord.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/OutputRecord.java?rev=762987&r1=762986&r2=762987&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/OutputRecord.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/metrics/spi/OutputRecord.java Tue Apr  7 22:00:07 2009
@@ -21,7 +21,11 @@
 package org.apache.hadoop.metrics.spi;
 
 import java.util.Collections;
+import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
 
@@ -68,5 +72,19 @@
   public Number getMetric(String name) {
     return metricMap.get(name);
   }
-    
+  
+
+  /**
+   * Returns a copy of this record's tags.
+   */
+  public TagMap getTagsCopy() {
+    return new TagMap(tagMap);
+  }
+  
+  /**
+   * Returns a copy of this record's metrics.
+   */
+  public MetricMap getMetricsCopy() {
+    return new MetricMap(metricMap);
+  }
 }

Added: hadoop/core/trunk/src/test/org/apache/hadoop/metrics/TestMetricsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/metrics/TestMetricsServlet.java?rev=762987&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/metrics/TestMetricsServlet.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/metrics/TestMetricsServlet.java Tue Apr  7 22:00:07 2009
@@ -0,0 +1,110 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.metrics;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.metrics.MetricsServlet.TagsMetricsPair;
+import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
+import org.apache.hadoop.metrics.spi.OutputRecord;
+import org.mortbay.util.ajax.JSON;
+
+public class TestMetricsServlet extends TestCase {
+  MetricsContext nc1;
+  MetricsContext nc2;
+  // List containing nc1 and nc2.
+  List<MetricsContext> contexts;
+  OutputRecord outputRecord;
+  
+  /**
+   * Initializes, for testing, two NoEmitMetricsContext's, and adds one value 
+   * to the first of them.
+   */
+  public void setUp() throws IOException {
+    nc1 = new NoEmitMetricsContext();
+    nc1.init("test1", ContextFactory.getFactory());
+    nc2 = new NoEmitMetricsContext();
+    nc2.init("test2", ContextFactory.getFactory());
+    contexts = new ArrayList<MetricsContext>();
+    contexts.add(nc1);
+    contexts.add(nc2);
+
+    MetricsRecord r = nc1.createRecord("testRecord");
+    
+    r.setTag("testTag1", "testTagValue1");
+    r.setTag("testTag2", "testTagValue2");
+    r.setMetric("testMetric1", 1);
+    r.setMetric("testMetric2", 33);
+    r.update();
+
+    Map<String, Collection<OutputRecord>> m = nc1.getAllRecords();
+    assertEquals(1, m.size());
+    assertEquals(1, m.values().size());
+    Collection<OutputRecord> outputRecords = m.values().iterator().next();
+    assertEquals(1, outputRecords.size());
+    outputRecord = outputRecords.iterator().next();
+  }
+  
+ 
+  
+  public void testTagsMetricsPair() throws IOException {
+    TagsMetricsPair pair = new TagsMetricsPair(outputRecord.getTagsCopy(), 
+        outputRecord.getMetricsCopy());
+    String s = JSON.toString(pair);
+    assertEquals(
+        "[{\"testTag1\":\"testTagValue1\",\"testTag2\":\"testTagValue2\"},"+
+        "{\"testMetric1\":1,\"testMetric2\":33}]", s);
+  }
+  
+  public void testGetMap() throws IOException {
+    MetricsServlet servlet = new MetricsServlet();
+    Map<String, Map<String, List<TagsMetricsPair>>> m = servlet.makeMap(contexts);
+    assertEquals("Map missing contexts", 2, m.size());
+    assertTrue(m.containsKey("test1"));
+   
+    Map<String, List<TagsMetricsPair>> m2 = m.get("test1");
+    
+    assertEquals("Missing records", 1, m2.size());
+    assertTrue(m2.containsKey("testRecord"));
+    assertEquals("Wrong number of tags-values pairs.", 1, m2.get("testRecord").size());
+  }
+  
+  public void testPrintMap() throws IOException {
+    StringWriter sw = new StringWriter();
+    PrintWriter out = new PrintWriter(sw);
+    MetricsServlet servlet = new MetricsServlet();
+    servlet.printMap(out, servlet.makeMap(contexts));
+    
+    String EXPECTED = "" +
+      "test1\n" +
+      "  testRecord\n" +
+      "    {testTag1=testTagValue1,testTag2=testTagValue2}:\n" +
+      "      testMetric1=1\n" +
+      "      testMetric2=33\n" +
+      "test2\n";
+    assertEquals(EXPECTED, sw.toString());
+  }
+}

Added: hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/TestOutputRecord.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/TestOutputRecord.java?rev=762987&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/TestOutputRecord.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/metrics/spi/TestOutputRecord.java Tue Apr  7 22:00:07 2009
@@ -0,0 +1,38 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.metrics.spi;
+
+import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
+import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
+
+import junit.framework.TestCase;
+
+public class TestOutputRecord extends TestCase {
+  public void testCopy() {
+    TagMap tags = new TagMap();
+    tags.put("tagkey", "tagval");
+    MetricMap metrics = new MetricMap();
+    metrics.put("metrickey", 123.4);
+    OutputRecord r = new OutputRecord(tags, metrics);
+    
+    assertEquals(tags, r.getTagsCopy());    
+    assertNotSame(tags, r.getTagsCopy());
+    assertEquals(metrics, r.getMetricsCopy());
+    assertNotSame(metrics, r.getMetricsCopy());
+  } 
+}