You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2013/05/17 02:28:51 UTC

svn commit: r1483617 - in /hbase/trunk: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/ hbase-hadoop-compat/src/test/resources/ hbase-hadoop-compat/src/test/resources/META-INF/ hbase...

Author: eclark
Date: Fri May 17 00:28:50 2013
New Revision: 1483617

URL: http://svn.apache.org/r1483617
Log:
HBASE-8564 TestMetricsRegionServer depends on test order

Added:
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGenerator.java
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGeneratorImpl.java
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java
    hbase/trunk/hbase-hadoop-compat/src/test/resources/
    hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/
    hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/
    hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.RandomStringGenerator
Modified:
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java (original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java Fri May 17 00:28:50 2013
@@ -33,8 +33,8 @@ import java.util.ServiceLoader;
 public class CompatibilitySingletonFactory extends CompatibilityFactory {
   public static enum SingletonStorage {
     INSTANCE;
-    Object lock = new Object();
-    private static final Map<Class, Object> instances = new HashMap<Class, Object>();
+    private final Object lock = new Object();
+    private final Map<Class, Object> instances = new HashMap<Class, Object>();
   }
   private static final Log LOG = LogFactory.getLog(CompatibilitySingletonFactory.class);
 

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGenerator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGenerator.java?rev=1483617&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGenerator.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGenerator.java Fri May 17 00:28:50 2013
@@ -0,0 +1,23 @@
+/**
+ * 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.hbase;
+
+public interface RandomStringGenerator {
+  String getRandString();
+}

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGeneratorImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGeneratorImpl.java?rev=1483617&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGeneratorImpl.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/RandomStringGeneratorImpl.java Fri May 17 00:28:50 2013
@@ -0,0 +1,36 @@
+/**
+ * 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.hbase;
+
+
+import java.util.UUID;
+
+public class RandomStringGeneratorImpl implements RandomStringGenerator {
+
+  private final String s;
+
+  public RandomStringGeneratorImpl() {
+    s = UUID.randomUUID().toString();
+  }
+
+  @Override
+  public String getRandString() {
+    return s;
+  }
+}

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java?rev=1483617&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java Fri May 17 00:28:50 2013
@@ -0,0 +1,84 @@
+/**
+ * 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.hbase;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class TestCompatibilitySingletonFactory {
+
+  private static final int ITERATIONS = 100000;
+  private static final Random RANDOM = new Random();
+
+  private class TestCompatibilitySingletonFactoryCallable implements Callable<String> {
+
+    @Override
+    public String call() throws Exception {
+      Thread.sleep(RANDOM.nextInt(10));
+      RandomStringGenerator
+          instance =
+          CompatibilitySingletonFactory.getInstance(RandomStringGenerator.class);
+      return instance.getRandString();
+    }
+  }
+
+  @Test
+  public void testGetInstance() throws Exception {
+    List<TestCompatibilitySingletonFactoryCallable> callables =
+        new ArrayList<TestCompatibilitySingletonFactoryCallable>(ITERATIONS);
+    List<String> resultStrings = new ArrayList<String>(ITERATIONS);
+
+
+    // Create the callables.
+    for (int i = 0; i < ITERATIONS; i++) {
+      callables.add(new TestCompatibilitySingletonFactoryCallable());
+    }
+
+    // Now run the callables.
+    ExecutorService executorService = Executors.newFixedThreadPool(100);
+    List<Future<String>> futures = executorService.invokeAll(callables);
+
+    // Wait for them all to finish.
+    for (Future<String> f : futures) {
+      resultStrings.add(f.get());
+    }
+
+    // Get the first string.
+    String firstString = resultStrings.get(0);
+
+
+    // Assert that all the strings are equal to the fist.
+    for (String s : resultStrings) {
+      assertEquals(firstString, s);
+    }
+
+    // an assert to make sure that RandomStringGeneratorImpl is generating random strings.
+    assertNotEquals(new RandomStringGeneratorImpl().getRandString(), firstString);
+  }
+}

Added: hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.RandomStringGenerator
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.RandomStringGenerator?rev=1483617&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.RandomStringGenerator (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.RandomStringGenerator Fri May 17 00:28:50 2013
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.RandomStringGeneratorImpl
\ No newline at end of file

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java Fri May 17 00:28:50 2013
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase.regionserver;
 
 /**
- * Factory to create MetricsRegionServerSource when given a  MetricsRegionServerWrapper
+ * Factory to create MetricsRegionServerSource when given a  MetricsRegionServerWrapper.
  */
 public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServerSourceFactory {
   public static enum FactoryStorage {
     INSTANCE;
     private Object aggLock = new Object();
-    private Object serverLock = new Object();
-    private MetricsRegionServerSource serverSource;
     private MetricsRegionAggregateSourceImpl aggImpl;
   }
 
@@ -41,14 +39,8 @@ public class MetricsRegionServerSourceFa
 
 
   @Override
-  public synchronized MetricsRegionServerSource createServer(MetricsRegionServerWrapper regionServerWrapper) {
-    synchronized (FactoryStorage.INSTANCE.serverLock) {
-      if (FactoryStorage.INSTANCE.serverSource == null) {
-        FactoryStorage.INSTANCE.serverSource = new MetricsRegionServerSourceImpl(
-            regionServerWrapper);
-      }
-      return FactoryStorage.INSTANCE.serverSource;
-    }
+  public MetricsRegionServerSource createServer(MetricsRegionServerWrapper regionServerWrapper) {
+    return new MetricsRegionServerSourceImpl(regionServerWrapper);
   }
 
   @Override

Modified: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java Fri May 17 00:28:50 2013
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -30,9 +28,8 @@ import java.util.concurrent.locks.Reentr
 public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
     implements MetricsRegionAggregateSource {
 
-  private final Log LOG = LogFactory.getLog(this.getClass());
   // lock to guard against concurrent access to regionSources
-  final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
   private final TreeSet<MetricsRegionSourceImpl> regionSources =
       new TreeSet<MetricsRegionSourceImpl>();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegion.java Fri May 17 00:28:50 2013
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionse
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CompatibilityFactory;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 
 
 /**
@@ -29,10 +30,10 @@ import org.apache.hadoop.hbase.Compatibi
 @InterfaceAudience.Private
 public class MetricsRegion {
 
-  private MetricsRegionSource source;
+  private final MetricsRegionSource source;
 
-  public MetricsRegion(MetricsRegionWrapper wrapper) {
-    source = CompatibilityFactory.getInstance(MetricsRegionServerSourceFactory.class)
+  public MetricsRegion(final MetricsRegionWrapper wrapper) {
+    source = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
                                              .createRegion(wrapper);
   }
 
@@ -48,11 +49,11 @@ public class MetricsRegion {
     source.updateDelete();
   }
 
-  public void updateGet(long getSize) {
+  public void updateGet(final long getSize) {
     source.updateGet(getSize);
   }
 
-  public void updateScanNext(long scanSize) {
+  public void updateScanNext(final long scanSize) {
     source.updateScan(scanSize);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java Fri May 17 00:28:50 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
@@ -26,22 +24,27 @@ import org.apache.hadoop.hbase.Compatibi
 /**
  * This class is for maintaining the various regionserver statistics
  * and publishing them through the metrics interfaces.
- * <p>
+ * <p/>
  * This class has a number of metrics variables that are publicly accessible;
  * these variables (objects) have methods to update their values.
  */
 @InterfaceStability.Evolving
 @InterfaceAudience.Private
 public class MetricsRegionServer {
-  private final Log LOG = LogFactory.getLog(this.getClass());
   private MetricsRegionServerSource serverSource;
   private MetricsRegionServerWrapper regionServerWrapper;
 
   public MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper) {
+    this(regionServerWrapper,
+        CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
+            .createServer(regionServerWrapper));
+
+  }
+
+  MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper,
+                      MetricsRegionServerSource serverSource) {
     this.regionServerWrapper = regionServerWrapper;
-    serverSource =
-            CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
-            .createServer(regionServerWrapper);
+    this.serverSource = serverSource;
   }
 
   // for unit-test usage
@@ -53,35 +56,35 @@ public class MetricsRegionServer {
     return regionServerWrapper;
   }
 
-  public void updatePut(long t){
+  public void updatePut(long t) {
     if (t > 1000) {
       serverSource.incrSlowPut();
     }
     serverSource.updatePut(t);
   }
 
-  public void updateDelete(long t){
+  public void updateDelete(long t) {
     if (t > 1000) {
       serverSource.incrSlowDelete();
     }
     serverSource.updateDelete(t);
   }
 
-  public void updateGet(long t){
+  public void updateGet(long t) {
     if (t > 1000) {
       serverSource.incrSlowGet();
     }
     serverSource.updateGet(t);
   }
 
-  public void updateIncrement(long t){
+  public void updateIncrement(long t) {
     if (t > 1000) {
       serverSource.incrSlowIncrement();
     }
     serverSource.updateIncrement(t);
   }
 
-  public void updateAppend(long t){
+  public void updateAppend(long t) {
     if (t > 1000) {
       serverSource.incrSlowAppend();
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java?rev=1483617&r1=1483616&r2=1483617&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java Fri May 17 00:28:50 2013
@@ -38,7 +38,11 @@ public class TestMetricsRegionServer {
 
   @Test
   public void testWrapperSource() {
-    MetricsRegionServer rsm = new MetricsRegionServer(new MetricsRegionServerWrapperStub());
+    MetricsRegionServerWrapperStub wrapper = new MetricsRegionServerWrapperStub();
+    MetricsRegionServerSource source =
+        CompatibilityFactory.getInstance(MetricsRegionServerSourceFactory.class)
+            .createServer(wrapper);
+    MetricsRegionServer rsm = new MetricsRegionServer(wrapper, source);
     MetricsRegionServerSource serverSource = rsm.getMetricsSource();
     HELPER.assertTag("serverName", "test", serverSource);
     HELPER.assertTag("clusterId", "tClusterId", serverSource);