Return-Path: Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: (qmail 39085 invoked from network); 4 Mar 2011 04:34:49 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 4 Mar 2011 04:34:49 -0000 Received: (qmail 52915 invoked by uid 500); 4 Mar 2011 04:34:49 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 52671 invoked by uid 500); 4 Mar 2011 04:34:49 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 52587 invoked by uid 99); 4 Mar 2011 04:34:49 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Mar 2011 04:34:49 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Mar 2011 04:34:46 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 1784F2388C27; Fri, 4 Mar 2011 04:34:02 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1077597 [6/6] - in /hadoop/common/branches/branch-0.20-security-patches: ./ conf/ ivy/ src/core/org/apache/hadoop/ipc/ src/core/org/apache/hadoop/ipc/metrics/ src/core/org/apache/hadoop/log/ src/core/org/apache/hadoop/metrics/ src/core/org... Date: Fri, 04 Mar 2011 04:34:00 -0000 To: common-commits@hadoop.apache.org From: omalley@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110304043402.1784F2388C27@eris.apache.org> Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,107 @@ +/** + * 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.impl; + +import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong; +import org.apache.hadoop.metrics2.lib.MetricMutableStat; +import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong; +import org.apache.hadoop.metrics2.lib.AbstractMetricsSource; +import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.runners.MockitoJUnitRunner; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import static org.apache.hadoop.test.MoreAsserts.*; +import org.apache.hadoop.metrics2.Metric; +import org.apache.hadoop.metrics2.MetricsRecord; +import org.apache.hadoop.metrics2.MetricsSink; +import org.apache.hadoop.metrics2.MetricsTag; + +/** + * Test the MetricsSystemImpl class + */ +@RunWith(MockitoJUnitRunner.class) +public class TestMetricsSystemImpl { + private static final Log LOG = LogFactory.getLog(TestMetricsSystemImpl.class); + @Captor private ArgumentCaptor r1; + @Captor private ArgumentCaptor r2; + private static String hostname = MetricsSystemImpl.getHostname(); + + @Test public void testInitFirst() throws Exception { + ConfigBuilder cb = new ConfigBuilder().add("default.period", 8) + .add("Test.sink.0.class", "org.apache.hadoop.metrics.sink.FileSink") + .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test")); + MetricsSystemImpl ms = new MetricsSystemImpl("Test"); + ms.start(); + TestSource s1 = ms.register("s1", "s1 desc", new TestSource("s1rec")); + s1.s1.add(0); + MetricsSink sink1 = mock(MetricsSink.class); + MetricsSink sink2 = mock(MetricsSink.class); + ms.register("sink1", "sink1 desc", sink1); + ms.register("sink2", "sink2 desc", sink2); + ms.onTimerEvent(); // trigger something interesting + ms.stop(); + + verify(sink1, times(2)).putMetrics(r1.capture()); + List mr1 = r1.getAllValues(); + verify(sink2, times(2)).putMetrics(r2.capture()); + List mr2 = r2.getAllValues(); + checkMetricsRecords(mr1); + assertEquals("output", mr1, mr2); + } + + static void checkMetricsRecords(List recs) { + LOG.debug(recs); + MetricsRecord r = recs.get(0); + assertEquals("name", "s1rec", r.name()); + assertEquals("tags", new MetricsTag[] { + new MetricsTag("context", "Metrics context", "test"), + new MetricsTag("hostName", "Local hostname", hostname)}, r.tags()); + assertEquals("metrics", new Metric[] { + new MetricCounterLong("c1", "c1 desc", 1), + new MetricGaugeLong("g1", "g1 desc", 2), + new MetricCounterLong("s1_num_ops", "Number of ops for s1 desc", 1), + new MetricGaugeDouble("s1_avg_time", "Average time for s1 desc", 0)}, + r.metrics()); + + // Skip the system metrics for now. + // MetricsRecord r1 = recs.get(1); + } + + private static class TestSource extends AbstractMetricsSource { + final MetricMutableCounterLong c1; + final MetricMutableGaugeLong g1; + final MetricMutableStat s1; + + TestSource(String name) { + super(name); + registry.setContext("test"); + c1 = registry.newCounter("c1", "c1 desc", 1L); + g1 = registry.newGauge("g1", "g1 desc", 2L); + s1 = registry.newStat("s1", "s1 desc", "ops", "time"); + } + } + +} Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsVisitor.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsVisitor.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsVisitor.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestMetricsVisitor.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,87 @@ +/** + * 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.impl; + +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.hadoop.metrics2.MetricsVisitor; +import org.apache.hadoop.metrics2.Metric; +import org.apache.hadoop.metrics2.MetricCounter; +import org.apache.hadoop.metrics2.MetricGauge; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.runners.MockitoJUnitRunner; + +/** + * Test the metric visitor interface + */ +@RunWith(MockitoJUnitRunner.class) +public class TestMetricsVisitor { + + @Captor private ArgumentCaptor> c1; + @Captor private ArgumentCaptor> c2; + @Captor private ArgumentCaptor> g1; + @Captor private ArgumentCaptor> g2; + @Captor private ArgumentCaptor> g3; + @Captor private ArgumentCaptor> g4; + + /** + * Test the common use cases + */ + @Test public void testCommon() { + MetricsVisitor visitor = mock(MetricsVisitor.class); + List metrics = Arrays.asList( + new MetricCounterInt("c1", "int counter", 1), + new MetricCounterLong("c2", "long counter", 2L), + new MetricGaugeInt("g1", "int gauge", 5), + new MetricGaugeLong("g2", "long gauge", 6L), + new MetricGaugeFloat("g3", "float gauge", 7f), + new MetricGaugeDouble("g4", "double gauge", 8d)); + + for (Metric metric : metrics) { + metric.visit(visitor); + } + + verify(visitor).counter(c1.capture(), eq(1)); + assertEquals("c1 name", "c1", c1.getValue().name()); + assertEquals("c1 description", "int counter", c1.getValue().description()); + verify(visitor).counter(c2.capture(), eq(2L)); + assertEquals("c2 name", "c2", c2.getValue().name()); + assertEquals("c2 description", "long counter", c2.getValue().description()); + verify(visitor).gauge(g1.capture(), eq(5)); + assertEquals("g1 name", "g1", g1.getValue().name()); + assertEquals("g1 description", "int gauge", g1.getValue().description()); + verify(visitor).gauge(g2.capture(), eq(6L)); + assertEquals("g2 name", "g2", g2.getValue().name()); + assertEquals("g2 description", "long gauge", g2.getValue().description()); + verify(visitor).gauge(g3.capture(), eq(7f)); + assertEquals("g3 name", "g3", g3.getValue().name()); + assertEquals("g3 description", "float gauge", g3.getValue().description()); + verify(visitor).gauge(g4.capture(), eq(8d)); + assertEquals("g4 name", "g4", g4.getValue().name()); + assertEquals("g4 description", "double gauge", g4.getValue().description()); + } + +} Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestSinkQueue.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestSinkQueue.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestSinkQueue.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/impl/TestSinkQueue.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,265 @@ +/** + * 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.impl; + +import java.util.ConcurrentModificationException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +/** + * Test the half-blocking metrics sink queue + */ +public class TestSinkQueue { + + private final Log LOG = LogFactory.getLog(TestSinkQueue.class); + + /** + * Test common use case + * @throws Exception + */ + @Test public void testCommon() throws Exception { + final SinkQueue q = new SinkQueue(2); + q.enqueue(1); + assertEquals("queue front", 1, (int) q.front()); + assertEquals("queue back", 1, (int) q.back()); + assertEquals("element", 1, (int) q.dequeue()); + + assertTrue("should enqueue", q.enqueue(2)); + q.consume(new Consumer() { + public void consume(Integer e) { + assertEquals("element", 2, (int) e); + } + }); + assertTrue("should enqueue", q.enqueue(3)); + assertEquals("element", 3, (int) q.dequeue()); + assertEquals("queue size", 0, q.size()); + assertEquals("queue front", null, q.front()); + assertEquals("queue back", null, q.back()); + } + + /** + * Test blocking when queue is empty + * @throws Exception + */ + @Test public void testEmptyBlocking() throws Exception { + final SinkQueue q = new SinkQueue(2); + final Runnable trigger = mock(Runnable.class); + // try consuming emtpy equeue and blocking + Thread t = new Thread() { + @Override public void run() { + try { + assertEquals("element", 1, (int) q.dequeue()); + q.consume(new Consumer() { + public void consume(Integer e) { + assertEquals("element", 2, (int) e); + trigger.run(); + } + }); + } + catch (InterruptedException e) { + LOG.warn("Interrupted", e); + } + } + }; + t.start(); + Thread.yield(); // Let the other block + q.enqueue(1); + q.enqueue(2); + t.join(); + verify(trigger).run(); + } + + /** + * Test nonblocking enqueue when queue is full + * @throws Exception + */ + @Test public void testFull() throws Exception { + final SinkQueue q = new SinkQueue(1); + q.enqueue(1); + + assertTrue("should drop", !q.enqueue(2)); + assertEquals("element", 1, (int) q.dequeue()); + + q.enqueue(3); + q.consume(new Consumer() { + public void consume(Integer e) { + assertEquals("element", 3, (int) e); + } + }); + assertEquals("queue size", 0, q.size()); + } + + /** + * Test the consumeAll method + * @throws Exception + */ + @Test public void testConsumeAll() throws Exception { + final int capacity = 64; // arbitrary + final SinkQueue q = new SinkQueue(capacity); + + for (int i = 0; i < capacity; ++i) { + assertTrue("should enqueue", q.enqueue(i)); + } + assertTrue("should not enqueue", !q.enqueue(capacity)); + + final Runnable trigger = mock(Runnable.class); + q.consumeAll(new Consumer() { + private int expected = 0; + public void consume(Integer e) { + assertEquals("element", expected++, (int) e); + trigger.run(); + } + }); + + verify(trigger, times(capacity)).run(); + } + + /** + * Test the consumer throwing exceptions + * @throws Exception + */ + @Test public void testConsumerException() throws Exception { + final SinkQueue q = new SinkQueue(1); + final RuntimeException ex = new RuntimeException("expected"); + q.enqueue(1); + + try { + q.consume(new Consumer() { + public void consume(Integer e) { + throw ex; + } + }); + } + catch (Exception expected) { + assertSame("consumer exception", ex, expected); + } + // The queue should be in consistent state after exception + assertEquals("queue size", 1, q.size()); + assertEquals("element", 1, (int) q.dequeue()); + } + + /** + * Test the clear method + */ + @Test public void testClear() { + final SinkQueue q = new SinkQueue(128); + for (int i = 0; i < q.capacity() + 97; ++i) { + q.enqueue(i); + } + assertEquals("queue size", q.capacity(), q.size()); + q.clear(); + assertEquals("queue size", 0, q.size()); + } + + /** + * Test consumers that take their time. + * @throws Exception + */ + @Test public void testHangingConsumer() throws Exception { + SinkQueue q = newSleepingConsumerQueue(2, 1, 2); + assertEquals("queue back", 2, (int) q.back()); + assertTrue("should drop", !q.enqueue(3)); // should not block + assertEquals("queue size", 2, q.size()); + assertEquals("queue head", 1, (int) q.front()); + assertEquals("queue back", 2, (int) q.back()); + } + + /** + * Test concurrent consumer access, which is illegal + * @throws Exception + */ + @Test public void testConcurrentConsumers() throws Exception { + final SinkQueue q = newSleepingConsumerQueue(2, 1); + assertTrue("should enqueue", q.enqueue(2)); + assertEquals("queue back", 2, (int) q.back()); + assertTrue("should drop", !q.enqueue(3)); // should not block + shouldThrowCME(new Fun() { + public void run() { + q.clear(); + } + }); + shouldThrowCME(new Fun() { + public void run() throws Exception { + q.consume(null); + } + }); + shouldThrowCME(new Fun() { + public void run() throws Exception { + q.consumeAll(null); + } + }); + shouldThrowCME(new Fun() { + public void run() throws Exception { + q.dequeue(); + } + }); + // The queue should still be in consistent state after all the exceptions + assertEquals("queue size", 2, q.size()); + assertEquals("queue front", 1, (int) q.front()); + assertEquals("queue back", 2, (int) q.back()); + } + + private void shouldThrowCME(Fun callback) throws Exception { + try { + callback.run(); + } + catch (ConcurrentModificationException e) { + LOG.info(e); + return; + } + fail("should've thrown"); + } + + private SinkQueue + newSleepingConsumerQueue(int capacity, int... values) { + final SinkQueue q = new SinkQueue(capacity); + for (int i : values) { + q.enqueue(i); + } + Thread t = new Thread() { + @Override public void run() { + try { + q.consume(new Consumer() { + public void consume(Integer e) throws InterruptedException { + LOG.info("sleeping"); + Thread.sleep(1000 * 86400); // a long time + } + }); + } + catch (InterruptedException ex) { + LOG.warn("Interrupted", ex); + } + } + }; + t.setName("Sleeping consumer"); + t.setDaemon(true); // so jvm can exit + t.start(); + Thread.yield(); // Let the consumer consume + LOG.debug("Returning new sleeping consumer queue"); + return q; + } + + static interface Fun { + void run() throws Exception; + } + +} Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricMutables.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricMutables.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricMutables.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricMutables.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,92 @@ +/** + * 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 java.util.Arrays; +import java.util.List; + +import org.junit.Test; +import static org.mockito.Mockito.*; +import static org.mockito.AdditionalMatchers.*; + +import org.apache.hadoop.metrics2.MetricsRecordBuilder; + + +/** + * Test metrics record builder interface and mutable metrics + */ +public class TestMetricMutables { + + private final double EPSILON = 1e-42; + + /** + * Test the snapshot method + */ + @Test public void testSnapshot() { + MetricsRecordBuilder mb = mock(MetricsRecordBuilder.class); + MetricMutableStat stat = + new MetricMutableStat("s1", "stat", "ops", "time", true); + stat.add(0); + MetricMutableStat stat2 = + new MetricMutableStat("s2", "stat", "ops", "time"); + stat2.add(0); + List metrics = Arrays.asList( + new MetricMutableCounterInt("c1", "int counter", 1), + new MetricMutableCounterLong("c2", "long counter", 2L), + new MetricMutableGaugeInt("g1", "int gauge", 3), + new MetricMutableGaugeLong("g2", "long gauge", 4L), + stat, stat2); + + for (MetricMutable metric : metrics) { + metric.snapshot(mb, true); + } + stat2.snapshot(mb, true); // should get the same back. + stat2.add(1); + stat2.snapshot(mb, true); // should get new interval values back + + verify(mb).addCounter("c1", "int counter", 1); + verify(mb).addCounter("c2", "long counter", 2L); + verify(mb).addGauge("g1", "int gauge", 3); + verify(mb).addGauge("g2", "long gauge", 4L); + verify(mb).addCounter("s1_num_ops", "Number of ops for stat", 1L); + verify(mb).addGauge(eq("s1_avg_time"), eq("Average time for stat"), + eq(0.0, EPSILON)); + verify(mb).addGauge(eq("s1_stdev_time"), + eq("Standard deviation of time for stat"), + eq(0.0, EPSILON)); + verify(mb).addGauge(eq("s1_imin_time"), + eq("Interval min time for stat"), + eq(0.0, EPSILON)); + verify(mb).addGauge(eq("s1_imax_time"), + eq("Interval max time for stat"), + eq(0.0, EPSILON)); + verify(mb).addGauge(eq("s1_min_time"), eq("Min time for stat"), + eq(0.0, EPSILON)); + verify(mb).addGauge(eq("s1_max_time"), eq("Max time for stat"), + eq(0.0, EPSILON)); + verify(mb, times(2)).addCounter("s2_num_ops", "Number of ops for stat", 1L); + verify(mb, times(2)).addGauge(eq("s2_avg_time"), + eq("Average time for stat"), + eq(0.0, EPSILON)); + verify(mb).addCounter("s2_num_ops", "Number of ops for stat", 2L); + verify(mb).addGauge(eq("s2_avg_time"), eq("Average time for stat"), + eq(1.0, EPSILON)); + } + +} Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,173 @@ +/** + * 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; + +/** + * Test the metric registry class + */ +public class TestMetricsRegistry { + + /** + * Test various factory methods + */ + @Test public void testNewMetrics() { + MetricMutableFactory factory = spy(new MetricMutableFactory()); + final MetricsRegistry r = new MetricsRegistry("test", factory); + 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()); + verify(factory).newCounter("c1", "c1 desc", 1); + verify(factory).newCounter("c2", "c2 desc", 2L); + verify(factory).newGauge("g1", "g1 desc", 3); + verify(factory).newGauge("g2", "g2 desc", 4L); + verify(factory).newStat("s1", "s1 desc", "ops", "time", false); + assertTrue("c1 found", r.get("c1") instanceof MetricMutableCounterInt); + assertTrue("c2 found", r.get("c2") instanceof MetricMutableCounterLong); + assertTrue("g1 found", r.get("g1") instanceof MetricMutableGaugeInt); + assertTrue("g2 found", r.get("g2") instanceof MetricMutableGaugeLong); + assertTrue("s1 found", r.get("s1") instanceof MetricMutableStat); + + expectMetricsException("Metric name c1 already exists", new Runnable() { + public void run() { r.newCounter("c1", "test dup", 0); } + }); + } + + /** + * Test the incr by name method + */ + @Test public void testIncrByName() { + MetricsRecordBuilder builder = mock(MetricsRecordBuilder.class); + MetricMutableFactory factory = new MetricMutableFactory() { + @Override public MetricMutable newMetric(String name) { + return new MetricMutableGaugeInt(name, "test incr", 1); + } + }; + final MetricsRegistry r = new MetricsRegistry("test", factory); + r.incr("g1"); + r.get("g1").snapshot(builder); + verify(builder).addGauge("g1", "test incr", 2); + + r.incr("c1", new MetricMutableFactory() { + @Override public MetricMutable newMetric(String name) { + return new MetricMutableCounterInt(name, "test incr", 2); + } + }); + r.get("c1").snapshot(builder); + verify(builder).addCounter("c1", "test incr", 3); + + r.newStat("s1", "test incr", "ops", "time"); + expectMetricsException("Unsupported incr", new Runnable() { + public void run() { r.incr("s1"); } + }); + + expectMetricsException("Metric n1 doesn't exist", new Runnable() { + public void run() { r.incr("n1", null); } + }); + } + + /** + * Test the decr by name method + */ + @Test public void testDecrByName() { + MetricsRecordBuilder builder = mock(MetricsRecordBuilder.class); + MetricMutableFactory factory = new MetricMutableFactory() { + @Override public MetricMutable newMetric(String name) { + return new MetricMutableGaugeInt(name, "test decr", 1); + } + }; + final MetricsRegistry r = new MetricsRegistry("test", factory); + r.decr("g1"); + r.get("g1").snapshot(builder); + verify(builder).addGauge("g1", "test decr", 0); + + expectMetricsException("Unsupported decr", new Runnable() { + public void run() { + r.decr("c1", new MetricMutableFactory() { + @Override public MetricMutable newMetric(String name) { + return new MetricMutableCounterInt(name, "test decr", 2); + } + }); + } + }); + + r.newStat("s1", "test decr", "ops", "time"); + expectMetricsException("Unsupported decr", new Runnable() { + public void run() { r.decr("s1"); } + }); + + expectMetricsException("Metric n1 doesn't exist", new Runnable() { + public void run() { r.decr("n1", null); } + }); + } + + /** + * Test the add by name method + */ + @Test public void testAddByName() { + MetricsRecordBuilder builder = mock(MetricsRecordBuilder.class); + MetricMutableFactory factory = new MetricMutableFactory() { + @Override public MetricMutableStat newStat(String name) { + return new MetricMutableStat(name, "test add", "ops", "time"); + } + }; + final MetricsRegistry r = new MetricsRegistry("test", factory); + r.add("s1", 42); + r.get("s1").snapshot(builder); + verify(builder).addCounter("s1_num_ops", "Number of ops for test add", 1L); + verify(builder).addGauge("s1_avg_time", "Average time for test add", 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); } + }); + + expectMetricsException("Metric n1 doesn't exist", new Runnable() { + public void run() { r.add("n1", 42, null); } + }); + } + + 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/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestSampleStat.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestSampleStat.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestSampleStat.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestSampleStat.java Fri Mar 4 04:33:55 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/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestTryIterator.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestTryIterator.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestTryIterator.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/metrics2/util/TestTryIterator.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,166 @@ +/** + * 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.Iterator; +import java.util.NoSuchElementException; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class TestTryIterator { + + /** + * Test a common use case + */ + @Test public void testCommonIteration() { + Iterator it = new TryIterator() { + private int count = 0; + @Override protected Integer tryNext() { + switch (count++) { + case 0: return 0; + case 1: return 1; + case 2: return done(); + default: fail("Should not reach here"); + } + return null; + } + }; + + assertTrue("has next", it.hasNext()); + assertEquals("next", 0, (int) it.next()); + + assertTrue("has next", it.hasNext()); + assertTrue("has next", it.hasNext()); // should be idempotent + + assertEquals("current", 1, (int) ((TryIterator) it).current()); + assertEquals("current 1", 1, (int) ((TryIterator) it).current()); + assertEquals("next", 1, (int) it.next()); + + assertTrue("no next", !it.hasNext()); + assertTrue("no next", !it.hasNext()); // ditto + + try { + it.next(); + fail("Should throw exception"); + } + catch (NoSuchElementException expected) { + expected.getCause(); + } + } + + /** + * Test empty conditions + */ + @Test public void testEmptyIteration() { + TryIterator it = new TryIterator() { + private boolean doneDone = false; + @Override public Integer tryNext() { + if (doneDone) { + fail("Should not be called again"); + } + doneDone = true; + return done(); + } + }; + + assertTrue("should not has next", !it.hasNext()); + + try { + it.current(); + fail("should throw"); + } + catch (NoSuchElementException expected) { + expected.getCause(); + } + + try { + it.next(); + fail("should throw"); + } + catch (NoSuchElementException expected) { + expected.getCause(); + } + } + + /** + * Test tryNext throwing exceptions + */ + @Test public void testExceptionInTryNext() { + final RuntimeException exception = new RuntimeException("expected"); + + Iterator it = new TryIterator() { + @Override public Integer tryNext() { + throw exception; + } + }; + + try { + it.hasNext(); + fail("should throw"); + } + catch (Exception expected) { + assertSame(exception, expected); + } + } + + /** + * Test remove method on the iterator, which should throw + */ + @Test public void testRemove() { + Iterator it = new TryIterator() { + private boolean called = false; + @Override public Integer tryNext() { + if (called) { + return done(); + } + called = true; + return 0; + } + }; + + assertEquals("should be 0", 0, (int) it.next()); + + try { + it.remove(); + } + catch (UnsupportedOperationException expected) { + expected.getCause(); + } + } + + /** + * Test calling hasNext in tryNext, which is illegal + */ + @Test public void testHasNextInTryNext() { + Iterator it = new TryIterator() { + @Override public Integer tryNext() { + hasNext(); + return null; + } + }; + + try { + it.hasNext(); + fail("should throw"); + } catch (IllegalStateException expected) { + expected.getCause(); + } + } +} Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestUserGroupInformation.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestUserGroupInformation.java?rev=1077597&r1=1077596&r2=1077597&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestUserGroupInformation.java (original) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/security/TestUserGroupInformation.java Fri Mar 4 04:33:55 2011 @@ -34,10 +34,12 @@ import java.util.List; import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.junit.Test; +import static org.apache.hadoop.test.MetricsAsserts.*; public class TestUserGroupInformation { final private static String USER_NAME = "user1@HADOOP.APACHE.ORG"; @@ -307,15 +309,16 @@ public class TestUserGroupInformation { public static void verifyLoginMetrics(int success, int failure) throws IOException { // Ensure metrics related to kerberos login is updated. - UserGroupInformation.UgiMetrics metrics = UserGroupInformation.metrics; - metrics.doUpdates(null); + UgiInstrumentation metrics = UserGroupInformation.metrics; + MetricsRecordBuilder rb = getMetrics(metrics); + if (success > 0) { - assertEquals(success, metrics.loginSuccess.getPreviousIntervalNumOps()); - assertTrue(metrics.loginSuccess.getPreviousIntervalAverageTime() > 0); + assertCounter("loginSuccess_num_ops", success, rb); + assertGaugeGt("loginSuccess_avg_time", 0, rb); } if (failure > 0) { - assertEquals(failure, metrics.loginFailure.getPreviousIntervalNumOps()); - assertTrue(metrics.loginFailure.getPreviousIntervalAverageTime() > 0); + assertEquals("loginFailure_num_ops", failure, rb); + assertGaugeGt("loginFailure_avg_time", 0, rb); } } } Added: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MetricsAsserts.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MetricsAsserts.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MetricsAsserts.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MetricsAsserts.java Fri Mar 4 04:33:55 2011 @@ -0,0 +1,198 @@ +/** + * 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 org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import static org.mockito.Mockito.*; +import org.mockito.stubbing.Answer; +import org.mockito.invocation.InvocationOnMock; +import static org.mockito.AdditionalMatchers.*; + +import org.apache.hadoop.metrics2.MetricsBuilder; +import org.apache.hadoop.metrics2.MetricsSource; +import org.apache.hadoop.metrics2.MetricsRecordBuilder; +import org.apache.hadoop.util.StringUtils; + +/** + * Helpers for metrics source tests + */ +public class MetricsAsserts { + + final static Log LOG = LogFactory.getLog(MetricsAsserts.class); + + /** + * Call getMetrics on source and get a record builder mock to verify + * @param source the metrics source + * @return the record builder mock to verify + */ + public static MetricsRecordBuilder getMetrics(MetricsSource source) { + MetricsBuilder mb = mock(MetricsBuilder.class); + final MetricsRecordBuilder rb = mock(MetricsRecordBuilder.class, + new Answer() { + @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)); + } + LOG.debug(invocation.getMethod().getName() +": "+ sb); + return invocation.getMock(); + } + }); + when(mb.addRecord(anyString())).thenReturn(rb); + source.getMetrics(mb, true); + return rb; + } + + /** + * 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(eq(name), anyString(), 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(eq(name), anyString(), 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(eq(name), anyString(), 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(eq(name), anyString(), 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(eq(name), anyString(), 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(eq(name), anyString(), 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/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MoreAsserts.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MoreAsserts.java?rev=1077597&view=auto ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MoreAsserts.java (added) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/test/MoreAsserts.java Fri Mar 4 04:33:55 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 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 void assertEquals(String s, T[] expected, + Iterable actual) { + Iterator 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 the type of the elements + * @param s + * @param expected + * @param actual + */ + public static void assertEquals(String s, Iterable expected, + Iterable actual) { + Iterator ite = expected.iterator(); + Iterator 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()); + } + +} Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/util/TestStringUtils.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/util/TestStringUtils.java?rev=1077597&r1=1077596&r2=1077597&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/util/TestStringUtils.java (original) +++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/util/TestStringUtils.java Fri Mar 4 04:33:55 2011 @@ -124,13 +124,20 @@ public class TestStringUtils extends Tes public void testJoin() { List s = new ArrayList(); + s.add(""); s.add("a"); s.add("b"); s.add("c"); - assertEquals("", StringUtils.join(":", s.subList(0, 0))); - assertEquals("a", StringUtils.join(":", s.subList(0, 1))); - assertEquals("a:b", StringUtils.join(":", s.subList(0, 2))); - assertEquals("a:b:c", StringUtils.join(":", s.subList(0, 3))); + checkJoin("", s.subList(0, 0)); + checkJoin(":a", s.subList(0, 2)); + checkJoin(":a:b", s.subList(0, 3)); + checkJoin(":a:b:c", s.subList(0, 4)); + } + + private void checkJoin(String result, List list) { + String[] a = new String[list.size()]; + assertEquals(result, StringUtils.join(":", list)); + assertEquals(result, StringUtils.join(":", list.toArray(a))); } public void testCamelize() {