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 om...@apache.org on 2011/03/08 05:39:02 UTC

svn commit: r1079141 [5/5] - in /hadoop/common/branches/yahoo-merge: ./ conf/ ivy/ src/java/org/apache/hadoop/metrics/file/ src/java/org/apache/hadoop/metrics2/ src/java/org/apache/hadoop/metrics2/annotation/ src/java/org/apache/hadoop/metrics2/filter/...

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestInterns.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestInterns.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestInterns.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestInterns.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,79 @@
+/*
+ * 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.metrics2.lib;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsTag;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+
+public class TestInterns {
+
+  @Test public void testInfo() {
+    MetricsInfo info = info("m", "m desc");
+    assertSame("same info", info, info("m", "m desc"));
+  }
+
+  @Test public void testTag() {
+    MetricsTag tag = tag("t", "t desc", "t value");
+    assertSame("same tag", tag, tag("t", "t desc", "t value"));
+  }
+
+  @Test public void testInfoOverflow() {
+    MetricsInfo i0 = info("m0", "m desc");
+    for (int i = 0; i < MAX_INFO_NAMES + 1; ++i) {
+      info("m"+ i, "m desc");
+      if (i < MAX_INFO_NAMES) {
+        assertSame("m0 is still there", i0, info("m0", "m desc"));
+      }
+    }
+    assertNotSame("m0 is gone", i0, info("m0", "m desc"));
+
+    MetricsInfo i1 = info("m1", "m desc");
+    for (int i = 0; i < MAX_INFO_DESCS; ++i) {
+      info("m1", "m desc"+ i);
+      if (i < MAX_INFO_DESCS - 1) {
+        assertSame("i1 is still there", i1, info("m1", "m desc"));
+      }
+    }
+    assertNotSame("i1 is gone", i1,  info("m1", "m desc"));
+  }
+
+  @Test public void testTagOverflow() {
+    MetricsTag t0 = tag("t0", "t desc", "t value");
+    for (int i = 0; i < MAX_TAG_NAMES + 1; ++i) {
+      tag("t"+ i, "t desc", "t value");
+      if (i < MAX_TAG_NAMES) {
+        assertSame("t0 still there", t0, tag("t0", "t desc", "t value"));
+      }
+    }
+    assertNotSame("t0 is gone", t0, tag("t0", "t desc", "t value"));
+
+    MetricsTag t1 = tag("t1", "t desc", "t value");
+    for (int i = 0; i < MAX_TAG_VALUES; ++i) {
+      tag("t1", "t desc", "t value"+ i);
+      if (i < MAX_TAG_VALUES -1) {
+        assertSame("t1 is still there", t1, tag("t1", "t desc", "t value"));
+      }
+    }
+    assertNotSame("t1 is gone", t1, tag("t1", "t desc", "t value"));
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsAnnotations.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsAnnotations.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsAnnotations.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsAnnotations.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,202 @@
+/*
+ * 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.metrics2.lib;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import static org.apache.hadoop.test.MockitoMaker.*;
+
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metric.*;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.impl.MsInfo;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+import static org.apache.hadoop.test.MetricsAsserts.*;
+
+public class TestMetricsAnnotations {
+
+  static class MyMetrics {
+    @Metric MutableCounterInt c1;
+    @Metric({"Counter2", "Counter2 desc"}) MutableCounterLong c2;
+    @Metric MutableGaugeInt g1, g2;
+    @Metric("g3 desc") MutableGaugeLong g3;
+    @Metric MutableRate r1;
+    @Metric MutableStat s1;
+    @Metric MutableRates rs1;
+  }
+
+  @Test public void testFields() {
+    MyMetrics metrics = new MyMetrics();
+    MetricsSource source = MetricsAnnotations.makeSource(metrics);
+
+    metrics.c1.incr();
+    metrics.c2.incr();
+    metrics.g1.incr();
+    metrics.g2.incr();
+    metrics.g3.incr();
+    metrics.r1.add(1);
+    metrics.s1.add(1);
+    metrics.rs1.add("rs1", 1);
+
+    MetricsRecordBuilder rb = getMetrics(source);
+
+    verify(rb).addCounter(info("C1", "C1"), 1);
+    verify(rb).addCounter(info("Counter2", "Counter2 desc"), 1L);
+    verify(rb).addGauge(info("G1", "G1"), 1);
+    verify(rb).addGauge(info("G2", "G2"), 1);
+    verify(rb).addGauge(info("G3", "g3 desc"), 1L);
+    verify(rb).addCounter(info("R1NumOps", "Number of ops for r1"), 1L);
+    verify(rb).addGauge(info("R1AvgTime", "Average time for r1"), 1.0);
+    verify(rb).addCounter(info("S1NumOps", "Number of ops for s1"), 1L);
+    verify(rb).addGauge(info("S1AvgTime", "Average time for s1"), 1.0);
+    verify(rb).addCounter(info("Rs1NumOps", "Number of ops for rs1"), 1L);
+    verify(rb).addGauge(info("Rs1AvgTime", "Average time for rs1"), 1.0);
+  }
+
+  static class BadMetrics {
+    @Metric Integer i0;
+  }
+
+  @Test(expected=MetricsException.class) public void testBadFields() {
+    MetricsAnnotations.makeSource(new BadMetrics());
+  }
+
+  static class MyMetrics2 {
+    @Metric int getG1() { return 1; }
+    @Metric long getG2() { return 2; }
+    @Metric float getG3() { return 3; }
+    @Metric double getG4() { return 4; }
+    @Metric(type=Type.COUNTER) int getC1() { return 1; }
+    @Metric(type=Type.COUNTER) long getC2() { return 2; }
+    @Metric(type=Type.TAG) String getT1() { return "t1"; }
+  }
+
+  @Test public void testMethods() {
+    MyMetrics2 metrics = new MyMetrics2();
+    MetricsSource source = MetricsAnnotations.makeSource(metrics);
+    MetricsRecordBuilder rb = getMetrics(source);
+
+    verify(rb).addGauge(info("G1", "G1"), 1);
+    verify(rb).addGauge(info("G2", "G2"), 2L);
+    verify(rb).addGauge(info("G3", "G3"), 3.0f);
+    verify(rb).addGauge(info("G4", "G4"), 4.0);
+    verify(rb).addCounter(info("C1", "C1"), 1);
+    verify(rb).addCounter(info("C2", "C2"), 2L);
+    verify(rb).tag(info("T1", "T1"), "t1");
+  }
+
+  static class BadMetrics2 {
+    @Metric int foo(int i) { return i; }
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testBadMethodWithArgs() {
+    MetricsAnnotations.makeSource(new BadMetrics2());
+  }
+
+  static class BadMetrics3 {
+    @Metric boolean foo() { return true; }
+  }
+
+  @Test(expected=MetricsException.class)
+  public void testBadMethodReturnType() {
+    MetricsAnnotations.makeSource(new BadMetrics3());
+  }
+
+  @Metrics(about="My metrics", context="foo")
+  static class MyMetrics3 {
+    @Metric int getG1() { return 1; }
+  }
+
+  @Test public void testClasses() {
+    MetricsRecordBuilder rb = getMetrics(
+        MetricsAnnotations.makeSource(new MyMetrics3()));
+    MetricsCollector collector = rb.parent();
+
+    verify(collector).addRecord(info("MyMetrics3", "My metrics"));
+    verify(rb).add(tag(MsInfo.Context, "foo"));
+  }
+
+  static class HybridMetrics implements MetricsSource {
+    final MetricsRegistry registry = new MetricsRegistry("HybridMetrics")
+        .setContext("hybrid");
+    @Metric("C0 desc") MutableCounterInt C0;
+
+    @Metric int getG0() { return 0; }
+
+    public void getMetrics(MetricsCollector collector, boolean all) {
+      collector.addRecord("foo")
+                  .setContext("foocontext")
+                  .addCounter(info("C1", "C1 desc"), 1)
+                  .endRecord()
+               .addRecord("bar")
+                  .setContext("barcontext")
+                  .addGauge(info("G1", "G1 desc"), 1);
+      registry.snapshot(collector.addRecord(registry.info()), all);
+    }
+  }
+
+  @Test public void testHybrid() {
+    HybridMetrics metrics = new HybridMetrics();
+    MetricsSource source = MetricsAnnotations.makeSource(metrics);
+
+    assertSame(metrics, source);
+    metrics.C0.incr();
+    MetricsRecordBuilder rb = getMetrics(source);
+    MetricsCollector collector = rb.parent();
+
+    verify(collector).addRecord("foo");
+    verify(collector).addRecord("bar");
+    verify(collector).addRecord(info("HybridMetrics", "HybridMetrics"));
+    verify(rb).setContext("foocontext");
+    verify(rb).addCounter(info("C1", "C1 desc"), 1);
+    verify(rb).setContext("barcontext");
+    verify(rb).addGauge(info("G1", "G1 desc"), 1);
+    verify(rb).add(tag(MsInfo.Context, "hybrid"));
+    verify(rb).addCounter(info("C0", "C0 desc"), 1);
+    verify(rb).addGauge(info("G0", "G0"), 0);
+  }
+
+  @Metrics(context="hybrid")
+  static class BadHybridMetrics implements MetricsSource {
+
+    @Metric MutableCounterInt c1;
+
+    public void getMetrics(MetricsCollector collector, boolean all) {
+      collector.addRecord("foo");
+    }
+  }
+
+  @Test(expected=MetricsException.class) public void testBadHybrid() {
+    MetricsAnnotations.makeSource(new BadHybridMetrics());
+  }
+
+  static class EmptyMetrics {
+    int foo;
+  }
+
+  @Test(expected=MetricsException.class) public void testEmptyMetrics() {
+    MetricsAnnotations.makeSource(new EmptyMetrics());
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,91 @@
+/**
+ * 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.metrics2.lib;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+import static org.apache.hadoop.test.MetricsAsserts.*;
+
+/**
+ * Test the metric registry class
+ */
+public class TestMetricsRegistry {
+
+  /**
+   * Test various factory methods
+   */
+  @Test public void testNewMetrics() {
+    final MetricsRegistry r = new MetricsRegistry("test");
+    r.newCounter("c1", "c1 desc", 1);
+    r.newCounter("c2", "c2 desc", 2L);
+    r.newGauge("g1", "g1 desc", 3);
+    r.newGauge("g2", "g2 desc", 4L);
+    r.newStat("s1", "s1 desc", "ops", "time");
+
+    assertEquals("num metrics in registry", 5, r.metrics().size());
+    assertTrue("c1 found", r.get("c1") instanceof MutableCounterInt);
+    assertTrue("c2 found", r.get("c2") instanceof MutableCounterLong);
+    assertTrue("g1 found", r.get("g1") instanceof MutableGaugeInt);
+    assertTrue("g2 found", r.get("g2") instanceof MutableGaugeLong);
+    assertTrue("s1 found", r.get("s1") instanceof MutableStat);
+
+    expectMetricsException("Metric name c1 already exists", new Runnable() {
+      public void run() { r.newCounter("c1", "test dup", 0); }
+    });
+  }
+
+  /**
+   * Test the add by name method
+   */
+  @Test public void testAddByName() {
+    MetricsRecordBuilder rb = mockMetricsRecordBuilder();
+    final MetricsRegistry r = new MetricsRegistry("test");
+    r.add("s1", 42);
+    r.get("s1").snapshot(rb);
+    verify(rb).addCounter(info("S1NumOps", "Number of ops for s1"), 1L);
+    verify(rb).addGauge(info("S1AvgTime", "Average time for s1"), 42.0);
+
+    r.newCounter("c1", "test add", 1);
+    r.newGauge("g1", "test add", 1);
+
+    expectMetricsException("Unsupported add", new Runnable() {
+      public void run() { r.add("c1", 42); }
+    });
+
+    expectMetricsException("Unsupported add", new Runnable() {
+      public void run() { r.add("g1", 42); }
+    });
+  }
+
+  private void expectMetricsException(String prefix, Runnable fun) {
+    try {
+      fun.run();
+    }
+    catch (MetricsException e) {
+      assertTrue("expected exception", e.getMessage().startsWith(prefix));
+      return;
+    }
+    fail("should've thrown '"+ prefix +"...'");
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,106 @@
+/**
+ * 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.metrics2.lib;
+
+import org.junit.Test;
+import static org.mockito.Mockito.*;
+import static org.mockito.AdditionalMatchers.*;
+
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+import static org.apache.hadoop.test.MetricsAsserts.*;
+
+/**
+ * Test metrics record builder interface and mutable metrics
+ */
+public class TestMutableMetrics {
+
+  private final double EPSILON = 1e-42;
+
+  /**
+   * Test the snapshot method
+   */
+  @Test public void testSnapshot() {
+    MetricsRecordBuilder mb = mockMetricsRecordBuilder();
+
+    MetricsRegistry registry = new MetricsRegistry("test");
+    registry.newCounter("c1", "int counter", 1);
+    registry.newCounter("c2", "long counter", 2L);
+    registry.newGauge("g1", "int gauge", 3);
+    registry.newGauge("g2", "long gauge", 4L);
+    registry.newStat("s1", "stat", "Ops", "Time", true).add(0);
+    registry.newRate("s2", "stat", false).add(0);
+
+    registry.snapshot(mb, true);
+
+    MutableStat s2 = (MutableStat) registry.get("s2");
+
+    s2.snapshot(mb, true); // should get the same back.
+    s2.add(1);
+    s2.snapshot(mb, true); // should get new interval values back
+
+    verify(mb).addCounter(info("c1", "int counter"), 1);
+    verify(mb).addCounter(info("c2", "long counter"), 2L);
+    verify(mb).addGauge(info("g1", "int gauge"), 3);
+    verify(mb).addGauge(info("g2", "long gauge"), 4L);
+    verify(mb).addCounter(info("S1NumOps", "Number of ops for stat"), 1L);
+    verify(mb).addGauge(eq(info("S1AvgTime", "Average time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb).addGauge(eq(info("S1StdevTime",
+                                "Standard deviation of time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb).addGauge(eq(info("S1IMinTime",
+                                "Interval min time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb).addGauge(eq(info("S1IMaxTime",
+                                "Interval max time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb).addGauge(eq(info("S1MinTime","Min time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb).addGauge(eq(info("S1MaxTime","Max time for stat")),
+                           eq(0.0, EPSILON));
+    verify(mb, times(2))
+        .addCounter(info("S2NumOps", "Number of ops for stat"), 1L);
+    verify(mb, times(2)).addGauge(eq(info("S2AvgTime",
+                                          "Average time for stat")),
+                                  eq(0.0, EPSILON));
+    verify(mb).addCounter(info("S2NumOps", "Number of ops for stat"), 2L);
+    verify(mb).addGauge(eq(info("S2AvgTime", "Average time for stat")),
+                           eq(1.0, EPSILON));
+  }
+
+  interface TestProtocol {
+    void foo();
+    void bar();
+  }
+
+  @Test public void testMutableRates() {
+    MetricsRecordBuilder rb = mockMetricsRecordBuilder();
+    MetricsRegistry registry = new MetricsRegistry("test");
+    MutableRates rates = new MutableRates(registry);
+
+    rates.init(TestProtocol.class);
+    registry.snapshot(rb, false);
+
+    assertCounter("FooNumOps", 0L, rb);
+    assertGauge("FooAvgTime", 0.0, rb);
+    assertCounter("BarNumOps", 0L, rb);
+    assertGauge("BarAvgTime", 0.0, rb);
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestUniqNames.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestUniqNames.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestUniqNames.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/lib/TestUniqNames.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,42 @@
+/*
+ * 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.metrics2.lib;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestUniqNames {
+
+  @Test public void testCommonCases() {
+    UniqueNames u = new UniqueNames();
+
+    assertEquals("foo", u.uniqueName("foo"));
+    assertEquals("foo-1", u.uniqueName("foo"));
+  }
+
+  @Test public void testCollisions() {
+    UniqueNames u = new UniqueNames();
+    u.uniqueName("foo");
+
+    assertEquals("foo-1", u.uniqueName("foo-1"));
+    assertEquals("foo-2", u.uniqueName("foo"));
+    assertEquals("foo-1-1", u.uniqueName("foo-1"));
+    assertEquals("foo-2-1", u.uniqueName("foo-2"));
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestMetricsCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestMetricsCache.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestMetricsCache.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestMetricsCache.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,139 @@
+/*
+ * 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.metrics2.util;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsTag;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+
+public class TestMetricsCache {
+  private static final Log LOG = LogFactory.getLog(TestMetricsCache.class);
+
+  @Test public void testUpdate() {
+    MetricsCache cache = new MetricsCache();
+    MetricsRecord mr = makeRecord("r",
+        Arrays.asList(makeTag("t", "tv")),
+        Arrays.asList(makeMetric("m", 0), makeMetric("m1", 1)));
+
+    MetricsCache.Record cr = cache.update(mr);
+    verify(mr).name();
+    verify(mr).tags();
+    verify(mr).metrics();
+    assertEquals("same record size", cr.metrics().size(),
+                 ((Collection<AbstractMetric>)mr.metrics()).size());
+    assertEquals("same metric value", 0, cr.getMetric("m"));
+
+    MetricsRecord mr2 = makeRecord("r",
+        Arrays.asList(makeTag("t", "tv")),
+        Arrays.asList(makeMetric("m", 2), makeMetric("m2", 42)));
+    cr = cache.update(mr2);
+    assertEquals("contains 3 metric", 3, cr.metrics().size());
+    assertEquals("updated metric value", 2, cr.getMetric("m"));
+    assertEquals("old metric value", 1, cr.getMetric("m1"));
+    assertEquals("new metric value", 42, cr.getMetric("m2"));
+
+    MetricsRecord mr3 = makeRecord("r",
+        Arrays.asList(makeTag("t", "tv3")), // different tag value
+        Arrays.asList(makeMetric("m3", 3)));
+    cr = cache.update(mr3); // should get a new record
+    assertEquals("contains 1 metric", 1, cr.metrics().size());
+    assertEquals("updated metric value", 3, cr.getMetric("m3"));
+    // tags cache should be empty so far
+    assertEquals("no tags", 0, cr.tags().size());
+    // until now
+    cr = cache.update(mr3, true);
+    assertEquals("Got 1 tag", 1, cr.tags().size());
+    assertEquals("Tag value", "tv3", cr.getTag("t"));
+    assertEquals("Metric value", 3, cr.getMetric("m3"));
+  }
+
+  @Test public void testGet() {
+    MetricsCache cache = new MetricsCache();
+    assertNull("empty", cache.get("r", Arrays.asList(makeTag("t", "t"))));
+    MetricsRecord mr = makeRecord("r",
+        Arrays.asList(makeTag("t", "t")),
+        Arrays.asList(makeMetric("m", 1)));
+    cache.update(mr);
+    MetricsCache.Record cr = cache.get("r", mr.tags());
+    LOG.debug("tags="+ mr.tags() +" cr="+ cr);
+
+    assertNotNull("Got record", cr);
+    assertEquals("contains 1 metric", 1, cr.metrics().size());
+    assertEquals("new metric value", 1, cr.getMetric("m"));
+  }
+
+  /**
+   * Make sure metrics tag has a sane hashCode impl
+   */
+  @Test public void testNullTag() {
+    MetricsCache cache = new MetricsCache();
+    MetricsRecord mr = makeRecord("r",
+        Arrays.asList(makeTag("t", null)),
+        Arrays.asList(makeMetric("m", 0), makeMetric("m1", 1)));
+
+    MetricsCache.Record cr = cache.update(mr);
+    assertTrue("t value should be null", null == cr.getTag("t"));
+  }
+
+  @Test public void testOverflow() {
+    MetricsCache cache = new MetricsCache();
+    MetricsCache.Record cr;
+    Collection<MetricsTag> t0 = Arrays.asList(makeTag("t0", "0"));
+    for (int i = 0; i < MetricsCache.MAX_RECS_PER_NAME_DEFAULT + 1; ++i) {
+      cr = cache.update(makeRecord("r",
+          Arrays.asList(makeTag("t"+ i, ""+ i)),
+          Arrays.asList(makeMetric("m", i))));
+      assertEquals("new metrics value", i, cr.getMetric("m"));
+      if (i < MetricsCache.MAX_RECS_PER_NAME_DEFAULT) {
+        assertNotNull("t0 is still there", cache.get("r", t0));
+      }
+    }
+    assertNull("t0 is gone", cache.get("r", t0));
+  }
+
+  private MetricsRecord makeRecord(String name, Collection<MetricsTag> tags,
+                                   Collection<AbstractMetric> metrics) {
+    MetricsRecord mr = mock(MetricsRecord.class);
+    when(mr.name()).thenReturn(name);
+    when(mr.tags()).thenReturn(tags);
+    when(mr.metrics()).thenReturn(metrics);
+    return mr;
+  }
+
+  private MetricsTag makeTag(String name, String value) {
+    return new MetricsTag(info(name, ""), value);
+  }
+
+  private AbstractMetric makeMetric(String name, Number value) {
+    AbstractMetric metric = mock(AbstractMetric.class);
+    when(metric.name()).thenReturn(name);
+    when(metric.value()).thenReturn(value);
+    return metric;
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestSampleStat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestSampleStat.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestSampleStat.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/metrics2/util/TestSampleStat.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,67 @@
+/**
+ * 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.metrics2.util;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test the running sample stat computation
+ */
+public class TestSampleStat {
+  private static final double EPSILON = 1e-42;
+
+  /**
+   * Some simple use cases
+   */
+  @Test public void testSimple() {
+    SampleStat stat = new SampleStat();
+    assertEquals("num samples", 0, stat.numSamples());
+    assertEquals("mean", 0.0, stat.mean(), EPSILON);
+    assertEquals("variance", 0.0, stat.variance(), EPSILON);
+    assertEquals("stddev", 0.0, stat.stddev(), EPSILON);
+    assertEquals("min", Double.MAX_VALUE, stat.min(), EPSILON);
+    assertEquals("max", Double.MIN_VALUE, stat.max(), EPSILON);
+
+    stat.add(3);
+    assertEquals("num samples", 1L, stat.numSamples());
+    assertEquals("mean", 3.0, stat.mean(), EPSILON);
+    assertEquals("variance", 0.0, stat.variance(), EPSILON);
+    assertEquals("stddev", 0.0, stat.stddev(), EPSILON);
+    assertEquals("min", 3.0, stat.min(), EPSILON);
+    assertEquals("max", 3.0, stat.max(), EPSILON);
+
+    stat.add(2).add(1);
+    assertEquals("num samples", 3L, stat.numSamples());
+    assertEquals("mean", 2.0, stat.mean(), EPSILON);
+    assertEquals("variance", 1.0, stat.variance(), EPSILON);
+    assertEquals("stddev", 1.0, stat.stddev(), EPSILON);
+    assertEquals("min", 1.0, stat.min(), EPSILON);
+    assertEquals("max", 3.0, stat.max(), EPSILON);
+
+    stat.reset();
+    assertEquals("num samples", 0, stat.numSamples());
+    assertEquals("mean", 0.0, stat.mean(), EPSILON);
+    assertEquals("variance", 0.0, stat.variance(), EPSILON);
+    assertEquals("stddev", 0.0, stat.stddev(), EPSILON);
+    assertEquals("min", Double.MAX_VALUE, stat.min(), EPSILON);
+    assertEquals("max", Double.MIN_VALUE, stat.max(), EPSILON);
+  }
+
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MetricsAsserts.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MetricsAsserts.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MetricsAsserts.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MetricsAsserts.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,276 @@
+/**
+ * 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.test;
+
+import static com.google.common.base.Preconditions.*;
+
+import org.hamcrest.Description;
+
+import static org.mockito.Mockito.*;
+import org.mockito.stubbing.Answer;
+import org.mockito.invocation.InvocationOnMock;
+import static org.mockito.AdditionalMatchers.*;
+import org.mockito.ArgumentMatcher;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import static org.apache.hadoop.metrics2.lib.Interns.*;
+
+/**
+ * Helpers for metrics source tests
+ */
+public class MetricsAsserts {
+
+  final static Log LOG = LogFactory.getLog(MetricsAsserts.class);
+
+  public static MetricsSystem mockMetricsSystem() {
+    MetricsSystem ms = mock(MetricsSystem.class);
+    DefaultMetricsSystem.setInstance(ms);
+    return ms;
+  }
+
+  public static MetricsRecordBuilder mockMetricsRecordBuilder() {
+    final MetricsCollector mc = mock(MetricsCollector.class);
+    MetricsRecordBuilder rb = mock(MetricsRecordBuilder.class,
+        new Answer<Object>() {
+      @Override
+      public Object answer(InvocationOnMock invocation) {
+        Object[] args = invocation.getArguments();
+        StringBuilder sb = new StringBuilder();
+        for (Object o : args) {
+          if (sb.length() > 0) sb.append(", ");
+          sb.append(String.valueOf(o));
+        }
+        String methodName = invocation.getMethod().getName();
+        LOG.debug(methodName +": "+ sb);
+        return methodName.equals("parent") || methodName.equals("endRecord") ?
+               mc : invocation.getMock();
+      }
+    });
+    when(mc.addRecord(anyString())).thenReturn(rb);
+    when(mc.addRecord(anyInfo())).thenReturn(rb);
+    return rb;
+  }
+
+  /**
+   * Call getMetrics on source and get a record builder mock to verify
+   * @param source  the metrics source
+   * @param all     if true, return all metrics even if not changed
+   * @return the record builder mock to verify
+   */
+  public static MetricsRecordBuilder getMetrics(MetricsSource source,
+                                                boolean all) {
+    MetricsRecordBuilder rb = mockMetricsRecordBuilder();
+    MetricsCollector mc = rb.parent();
+    source.getMetrics(mc, all);
+    return rb;
+  }
+
+  public static MetricsRecordBuilder getMetrics(String name) {
+    return getMetrics(DefaultMetricsSystem.instance().getSource(name));
+  }
+
+  public static MetricsRecordBuilder getMetrics(MetricsSource source) {
+    return getMetrics(source, true);
+  }
+
+  private static class InfoWithSameName extends ArgumentMatcher<MetricsInfo> {
+    private final String expected;
+
+    InfoWithSameName(MetricsInfo info) {
+      expected = checkNotNull(info.name(), "info name");
+    }
+
+    @Override public boolean matches(Object info) {
+      return expected.equals(((MetricsInfo)info).name());
+    }
+
+    @Override public void describeTo(Description desc) {
+      desc.appendText("Info with name="+ expected);
+    }
+  }
+
+  /**
+   * MetricInfo with the same name
+   * @param info to match
+   * @return <code>null</code>
+   */
+  public static MetricsInfo eqName(MetricsInfo info) {
+    return argThat(new InfoWithSameName(info));
+  }
+
+  private static class AnyInfo extends ArgumentMatcher<MetricsInfo> {
+    @Override public boolean matches(Object info) {
+      return info instanceof MetricsInfo;  // not null as well
+    }
+  }
+
+  public static MetricsInfo anyInfo() {
+    return argThat(new AnyInfo());
+  }
+
+  /**
+   * Assert an int gauge metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertGauge(String name, int expected,
+                                 MetricsRecordBuilder rb) {
+    verify(rb).addGauge(eqName(info(name, "")), eq(expected));
+  }
+
+  /**
+   * Assert an int counter metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertCounter(String name, int expected,
+                                   MetricsRecordBuilder rb) {
+    verify(rb).addCounter(eqName(info(name, "")), eq(expected));
+  }
+
+  /**
+   * Assert a long gauge metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertGauge(String name, long expected,
+                                 MetricsRecordBuilder rb) {
+    verify(rb).addGauge(eqName(info(name, "")), eq(expected));
+  }
+
+   /**
+   * Assert a double gauge metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertGauge(String name, double expected,
+                                 MetricsRecordBuilder rb) {
+    verify(rb).addGauge(eqName(info(name, "")), eq(expected));
+  }
+
+  /**
+   * Assert a long counter metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertCounter(String name, long expected,
+                                   MetricsRecordBuilder rb) {
+    verify(rb).addCounter(eqName(info(name, "")), eq(expected));
+  }
+
+  /**
+   * Assert an int gauge metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param source  to get metrics from
+   */
+  public static void assertGauge(String name, int expected,
+                                 MetricsSource source) {
+    assertGauge(name, expected, getMetrics(source));
+  }
+
+  /**
+   * Assert an int counter metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param source  to get metrics from
+   */
+  public static void assertCounter(String name, int expected,
+                                   MetricsSource source) {
+    assertCounter(name, expected, getMetrics(source));
+  }
+
+  /**
+   * Assert a long gauge metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param source  to get metrics from
+   */
+  public static void assertGauge(String name, long expected,
+                                 MetricsSource source) {
+    assertGauge(name, expected, getMetrics(source));
+  }
+
+  /**
+   * Assert a long counter metric as expected
+   * @param name  of the metric
+   * @param expected  value of the metric
+   * @param source  to get metrics from
+   */
+  public static void assertCounter(String name, long expected,
+                                   MetricsSource source) {
+    assertCounter(name, expected, getMetrics(source));
+  }
+
+  /**
+   * Assert that a long counter metric is greater than a value
+   * @param name  of the metric
+   * @param greater value of the metric should be greater than this
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertCounterGt(String name, long greater,
+                                     MetricsRecordBuilder rb) {
+    verify(rb).addCounter(eqName(info(name, "")), gt(greater));
+  }
+
+  /**
+   * Assert that a long counter metric is greater than a value
+   * @param name  of the metric
+   * @param greater value of the metric should be greater than this
+   * @param source  the metrics source
+   */
+  public static void assertCounterGt(String name, long greater,
+                                     MetricsSource source) {
+    assertCounterGt(name, greater, getMetrics(source));
+  }
+
+  /**
+   * Assert that a double gauge metric is greater than a value
+   * @param name  of the metric
+   * @param greater value of the metric should be greater than this
+   * @param rb  the record builder mock used to getMetrics
+   */
+  public static void assertGaugeGt(String name, double greater,
+                                   MetricsRecordBuilder rb) {
+    verify(rb).addGauge(eqName(info(name, "")), gt(greater));
+  }
+
+  /**
+   * Assert that a double gauge metric is greater than a value
+   * @param name  of the metric
+   * @param greater value of the metric should be greater than this
+   * @param source  the metrics source
+   */
+  public static void assertGaugeGt(String name, double greater,
+                                   MetricsSource source) {
+    assertGaugeGt(name, greater, getMetrics(source));
+  }
+}

Added: hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MoreAsserts.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MoreAsserts.java?rev=1079141&view=auto
==============================================================================
--- hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MoreAsserts.java (added)
+++ hadoop/common/branches/yahoo-merge/src/test/core/org/apache/hadoop/test/MoreAsserts.java Tue Mar  8 04:38:59 2011
@@ -0,0 +1,66 @@
+/**
+ * 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.test;
+
+import java.util.Iterator;
+import org.junit.Assert;
+
+/**
+ * A few more asserts
+ */
+public class MoreAsserts {
+
+  /**
+   * Assert equivalence for array and iterable
+   * @param <T> the type of the elements
+   * @param s the name/message for the collection
+   * @param expected  the expected array of elements
+   * @param actual    the actual iterable of elements
+   */
+  public static <T> void assertEquals(String s, T[] expected,
+                                      Iterable<T> actual) {
+    Iterator<T> it = actual.iterator();
+    int i = 0;
+    for (; i < expected.length && it.hasNext(); ++i) {
+      Assert.assertEquals("Element "+ i +" for "+ s, expected[i], it.next());
+    }
+    Assert.assertTrue("Expected more elements", i == expected.length);
+    Assert.assertTrue("Expected less elements", !it.hasNext());
+  }
+
+  /**
+   * Assert equality for two iterables
+   * @param <T> the type of the elements
+   * @param s
+   * @param expected
+   * @param actual
+   */
+  public static <T> void assertEquals(String s, Iterable<T> expected,
+                                      Iterable<T> actual) {
+    Iterator<T> ite = expected.iterator();
+    Iterator<T> ita = actual.iterator();
+    int i = 0;
+    while (ite.hasNext() && ita.hasNext()) {
+      Assert.assertEquals("Element "+ i +" for "+s, ite.next(), ita.next());
+    }
+    Assert.assertTrue("Expected more elements", !ite.hasNext());
+    Assert.assertTrue("Expected less elements", !ita.hasNext());
+  }
+
+}