You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/03/24 21:25:10 UTC

svn commit: r1085114 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Author: stack
Date: Thu Mar 24 20:25:10 2011
New Revision: 1085114

URL: http://svn.apache.org/viewvc?rev=1085114&view=rev
Log:
HBASE-3654 Weird blocking between getOnlineRegion and createRegionLoad

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1085114&r1=1085113&r2=1085114&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Mar 24 20:25:10 2011
@@ -171,6 +171,8 @@ Release 0.90.2 - Unreleased
    HBASE-3627  NPE in EventHandler when region already reassigned
    HBASE-3660  HMaster will exit when starting with stale data in cached locations
                such as -ROOT- or .META.
+   HBASE-3654  Weird blocking between getOnlineRegion and createRegionLoad
+               (Subbu M Iyer via Stack)
 
   IMPROVEMENTS
    HBASE-3542  MultiGet methods in Thrift

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1085114&r1=1085113&r2=1085114&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Mar 24 20:25:10 2011
@@ -182,7 +182,7 @@ public class HRegionServer implements HR
    * encoded region name.  All access should be synchronized.
    */
   protected final Map<String, HRegion> onlineRegions =
-    new HashMap<String, HRegion>();
+    new ConcurrentHashMap<String, HRegion>();
 
   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
   private final LinkedBlockingQueue<HMsg> outboundMsgs = new LinkedBlockingQueue<HMsg>();
@@ -688,11 +688,9 @@ public class HRegionServer implements HR
 
   String getOnlineRegionsAsPrintableString() {
     StringBuilder sb = new StringBuilder();
-    synchronized (this.onlineRegions) {
-      for (HRegion r: this.onlineRegions.values()) {
-        if (sb.length() > 0) sb.append(", ");
-        sb.append(r.getRegionInfo().getEncodedName());
-      }
+    for (HRegion r: this.onlineRegions.values()) {
+      if (sb.length() > 0) sb.append(", ");
+      sb.append(r.getRegionInfo().getEncodedName());
     }
     return sb.toString();
   }
@@ -712,9 +710,7 @@ public class HRegionServer implements HR
         // Only print out regions still closing if a small number else will
         // swamp the log.
         if (count < 10 && LOG.isDebugEnabled()) {
-          synchronized (this.onlineRegions) {
-            LOG.debug(this.onlineRegions);
-          }
+          LOG.debug(this.onlineRegions);
         }
       }
       Threads.sleep(1000);
@@ -756,10 +752,8 @@ public class HRegionServer implements HR
     HServerLoad hsl = new HServerLoad(requestCount.get(),
       (int)(memory.getUsed() / 1024 / 1024),
       (int) (memory.getMax() / 1024 / 1024));
-    synchronized (this.onlineRegions) {
-      for (HRegion r : this.onlineRegions.values()) {
-        hsl.addRegionInfo(createRegionLoad(r));
-      }
+    for (HRegion r : this.onlineRegions.values()) {
+      hsl.addRegionInfo(createRegionLoad(r));
     }
     return hsl;
   }
@@ -923,9 +917,7 @@ public class HRegionServer implements HR
    */
   public HServerLoad.RegionLoad createRegionLoad(final String encodedRegionName) {
     HRegion r = null;
-    synchronized (this.onlineRegions) {
-      r = this.onlineRegions.get(encodedRegionName);
-    }
+    r = this.onlineRegions.get(encodedRegionName);
     return createRegionLoad(r);
   }
 
@@ -1042,17 +1034,15 @@ public class HRegionServer implements HR
 
     @Override
     protected void chore() {
-      synchronized (this.instance.onlineRegions) {
-        for (HRegion r : this.instance.onlineRegions.values()) {
-          try {
-            if (r != null && r.isMajorCompaction()) {
-              // Queue a compaction. Will recognize if major is needed.
-              this.instance.compactSplitThread.requestCompaction(r, getName()
-                + " requests major compaction");
-            }
-          } catch (IOException e) {
-            LOG.warn("Failed major compaction check on " + r, e);
+      for (HRegion r : this.instance.onlineRegions.values()) {
+        try {
+          if (r != null && r.isMajorCompaction()) {
+            // Queue a compaction. Will recognize if major is needed.
+            this.instance.compactSplitThread.requestCompaction(r, getName()
+              + " requests major compaction");
           }
+        } catch (IOException e) {
+          LOG.warn("Failed major compaction check on " + r, e);
         }
       }
     }
@@ -1154,8 +1144,7 @@ public class HRegionServer implements HR
     long memstoreSize = 0;
     long requestsCount = 0;
     long storefileIndexSize = 0;
-    synchronized (this.onlineRegions) {
-      for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
+    for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
         HRegion r = e.getValue();
         memstoreSize += r.memstoreSize.get();
         requestsCount += r.requestsCount.get();
@@ -1168,7 +1157,6 @@ public class HRegionServer implements HR
           }
         }
       }
-    }
     this.metrics.stores.set(stores);
     this.metrics.storefiles.set(storefiles);
     this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
@@ -1547,16 +1535,14 @@ public class HRegionServer implements HR
     HRegion root = null;
     this.lock.writeLock().lock();
     try {
-      synchronized (this.onlineRegions) {
-        for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
-          HRegionInfo hri = e.getValue().getRegionInfo();
-          if (hri.isRootRegion()) {
-            root = e.getValue();
-          } else if (hri.isMetaRegion()) {
-            meta = e.getValue();
-          }
-          if (meta != null && root != null) break;
+      for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
+        HRegionInfo hri = e.getValue().getRegionInfo();
+        if (hri.isRootRegion()) {
+          root = e.getValue();
+        } else if (hri.isMetaRegion()) {
+          meta = e.getValue();
         }
+        if (meta != null && root != null) break;
       }
     } finally {
       this.lock.writeLock().unlock();
@@ -1572,13 +1558,11 @@ public class HRegionServer implements HR
   void closeUserRegions(final boolean abort) {
     this.lock.writeLock().lock();
     try {
-      synchronized (this.onlineRegions) {
-        for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
-          HRegion r = e.getValue();
-          if (!r.getRegionInfo().isMetaRegion()) {
-            // Don't update zk with this close transition; pass false.
-            closeRegion(r.getRegionInfo(), abort, false);
-          }
+      for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
+        HRegion r = e.getValue();
+        if (!r.getRegionInfo().isMetaRegion()) {
+          // Don't update zk with this close transition; pass false.
+          closeRegion(r.getRegionInfo(), abort, false);
         }
       }
     } finally {
@@ -2267,14 +2251,12 @@ public class HRegionServer implements HR
   public boolean closeRegion(HRegionInfo region, final boolean zk)
   throws NotServingRegionException {
     LOG.info("Received close region: " + region.getRegionNameAsString());
-    synchronized (this.onlineRegions) {
-      boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
-      if (!hasit) {
-        LOG.warn("Received close for region we are not serving; " +
-          region.getEncodedName());
-        throw new NotServingRegionException("Received close for "
-          + region.getRegionNameAsString() + " but we are not serving it");
-      }
+    boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
+    if (!hasit) {
+      LOG.warn("Received close for region we are not serving; " +
+        region.getEncodedName());
+      throw new NotServingRegionException("Received close for "
+        + region.getRegionNameAsString() + " but we are not serving it");
     }
     return closeRegion(region, false, zk);
   }
@@ -2376,10 +2358,8 @@ public class HRegionServer implements HR
   @QosPriority(priority=HIGH_QOS)
   public List<HRegionInfo> getOnlineRegions() {
     List<HRegionInfo> list = new ArrayList<HRegionInfo>();
-    synchronized(this.onlineRegions) {
-      for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
-        list.add(e.getValue().getRegionInfo());
-      }
+    for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
+      list.add(e.getValue().getRegionInfo());
     }
     Collections.sort(list);
     return list;
@@ -2387,16 +2367,12 @@ public class HRegionServer implements HR
 
   public int getNumberOfOnlineRegions() {
     int size = -1;
-    synchronized (this.onlineRegions) {
-      size = this.onlineRegions.size();
-    }
+    size = this.onlineRegions.size();
     return size;
   }
 
   boolean isOnlineRegionsEmpty() {
-    synchronized (this.onlineRegions) {
-      return this.onlineRegions.isEmpty();
-    }
+    return this.onlineRegions.isEmpty();
   }
 
   /**
@@ -2406,35 +2382,19 @@ public class HRegionServer implements HR
    * @see #getOnlineRegions()
    */
   public Collection<HRegion> getOnlineRegionsLocalContext() {
-    synchronized (this.onlineRegions) {
-      Collection<HRegion> regions = this.onlineRegions.values();
-      return Collections.unmodifiableCollection(regions);
-    }
+    Collection<HRegion> regions = this.onlineRegions.values();
+    return Collections.unmodifiableCollection(regions);
   }
 
   @Override
   public void addToOnlineRegions(HRegion region) {
-    lock.writeLock().lock();
-    try {
-      synchronized (this.onlineRegions) {
-        this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
-      }
-    } finally {
-      lock.writeLock().unlock();
-    }
+    this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
   }
 
   @Override
   public boolean removeFromOnlineRegions(final String encodedName) {
-    this.lock.writeLock().lock();
     HRegion toReturn = null;
-    try {
-      synchronized (this.onlineRegions) {
-        toReturn = this.onlineRegions.remove(encodedName);
-      }
-    } finally {
-      this.lock.writeLock().unlock();
-    }
+    toReturn = this.onlineRegions.remove(encodedName);
     return toReturn != null;
   }
 
@@ -2451,10 +2411,8 @@ public class HRegionServer implements HR
           }
         });
     // Copy over all regions. Regions are sorted by size with biggest first.
-    synchronized (this.onlineRegions) {
-      for (HRegion region : this.onlineRegions.values()) {
-        sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
-      }
+    for (HRegion region : this.onlineRegions.values()) {
+      sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
     }
     return sortedRegions;
   }
@@ -2462,9 +2420,7 @@ public class HRegionServer implements HR
   @Override
   public HRegion getFromOnlineRegions(final String encodedRegionName) {
     HRegion r = null;
-    synchronized (this.onlineRegions) {
-      r = this.onlineRegions.get(encodedRegionName);
-    }
+    r = this.onlineRegions.get(encodedRegionName);
     return r;
   }
 
@@ -2498,17 +2454,12 @@ public class HRegionServer implements HR
   protected HRegion getRegion(final byte[] regionName)
       throws NotServingRegionException {
     HRegion region = null;
-    this.lock.readLock().lock();
-    try {
-      region = getOnlineRegion(regionName);
-      if (region == null) {
-        throw new NotServingRegionException("Region is not online: " +
-          Bytes.toStringBinary(regionName));
-      }
-      return region;
-    } finally {
-      this.lock.readLock().unlock();
+    region = getOnlineRegion(regionName);
+    if (region == null) {
+      throw new NotServingRegionException("Region is not online: " +
+        Bytes.toStringBinary(regionName));
     }
+    return region;
   }
 
   /**
@@ -2519,16 +2470,14 @@ public class HRegionServer implements HR
    */
   protected HRegionInfo[] getMostLoadedRegions() {
     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
-    synchronized (onlineRegions) {
-      for (HRegion r : onlineRegions.values()) {
-        if (r.isClosed() || r.isClosing()) {
-          continue;
-        }
-        if (regions.size() < numRegionsToReport) {
-          regions.add(r.getRegionInfo());
-        } else {
-          break;
-        }
+    for (HRegion r : onlineRegions.values()) {
+      if (r.isClosed() || r.isClosing()) {
+        continue;
+      }
+      if (regions.size() < numRegionsToReport) {
+        regions.add(r.getRegionInfo());
+      } else {
+        break;
       }
     }
     return regions.toArray(new HRegionInfo[regions.size()]);
@@ -2573,10 +2522,8 @@ public class HRegionServer implements HR
    */
   public long getGlobalMemStoreSize() {
     long total = 0;
-    synchronized (onlineRegions) {
-      for (HRegion region : onlineRegions.values()) {
-        total += region.memstoreSize.get();
-      }
+    for (HRegion region : onlineRegions.values()) {
+      total += region.memstoreSize.get();
     }
     return total;
   }
@@ -2675,14 +2622,12 @@ public class HRegionServer implements HR
   }
 
   public HRegionInfo[] getRegionsAssignment() throws IOException {
-    synchronized (this.onlineRegions) {
-      HRegionInfo [] regions = new HRegionInfo[getNumberOfOnlineRegions()];
-      Iterator<HRegion> ite = onlineRegions.values().iterator();
-      for (int i = 0; ite.hasNext(); i++) {
-        regions[i] = ite.next().getRegionInfo();
-      }
-      return regions;
+    HRegionInfo [] regions = new HRegionInfo[getNumberOfOnlineRegions()];
+    Iterator<HRegion> ite = onlineRegions.values().iterator();
+    for (int i = 0; ite.hasNext(); i++) {
+      regions[i] = ite.next().getRegionInfo();
     }
+    return regions;
   }
 
   /** {@inheritDoc} */