You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2008/12/03 22:03:11 UTC

svn commit: r723070 - in /hadoop/hbase/branches/0.19_on_hadoop_0.18: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/mapred/ src/java/org/apache/hadoop/hbase/master/ src/java/org/apache/had...

Author: apurtell
Date: Wed Dec  3 13:03:10 2008
New Revision: 723070

URL: http://svn.apache.org/viewvc?rev=723070&view=rev
Log:
merge up to trunk (revision 723059)

Modified:
    hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/master/HMaster.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt Wed Dec  3 13:03:10 2008
@@ -88,8 +88,13 @@
    HBASE-1036  HBASE-1028 broke Thrift
    HBASE-1037  Some test cases failing on Windows/Cygwin but not UNIX/Linux
    HBASE-1041  Migration throwing NPE
-   HBASE-1042  OOME but we don't abort
+   HBASE-1042  OOME but we don't abort; two part commit.
    HBASE-927   We don't recover if HRS hosting -ROOT-/.META. goes down
+   HBASE-1043  Removing @Override attributes where they are no longer needed.
+               (Ryan Smith via Jim Kellerman)
+   HBASE-927   We don't recover if HRS hosting -ROOT-/.META. goes down -
+               (fix bug in createTable which caused tests to fail)
+
       
   IMPROVEMENTS
    HBASE-901   Add a limit to key length, check key and value length on client side

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java Wed Dec  3 13:03:10 2008
@@ -34,6 +34,27 @@
   private RemoteExceptionHandler() {super();}
   
   /**
+   * Examine passed Throwable.  See if its carrying a RemoteException. If so,
+   * run {@link #decodeRemoteException(RemoteException)} on it.  Otherwise,
+   * pass back <code>t</code> unaltered.
+   * @param t Throwable to examine.
+   * @return Decoded RemoteException carried by <code>t</code> or
+   * <code>t</code> unaltered.
+   */
+  public static Throwable checkThrowable(final Throwable t) {
+    Throwable result = t;
+    if (t instanceof RemoteException) {
+      try {
+        result =
+          RemoteExceptionHandler.decodeRemoteException((RemoteException)t);
+      } catch (Throwable tt) {
+        result = tt;
+      }
+    }
+    return result;
+  }
+  
+  /**
    * Examine passed IOException.  See if its carrying a RemoteException. If so,
    * run {@link #decodeRemoteException(RemoteException)} on it.  Otherwise,
    * pass back <code>e</code> unaltered.
@@ -42,16 +63,8 @@
    * <code>e</code> unaltered.
    */
   public static IOException checkIOException(final IOException e) {
-    IOException result = e;
-    if (e instanceof RemoteException) {
-      try {
-        result = RemoteExceptionHandler.decodeRemoteException(
-            (RemoteException) e);
-      } catch (IOException ex) {
-        result = ex;
-      }
-    }
-    return result;
+    Throwable t = checkThrowable(e);
+    return t instanceof IOException? (IOException)t: new IOException(t);
   }
   
   /**

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java Wed Dec  3 13:03:10 2008
@@ -65,19 +65,16 @@
     this.pattern = Pattern.compile(expr);
   }
 
-  @Override
   public int compareTo(byte[] value) {
     // Use find() for subsequence match instead of matches() (full sequence
     // match) to adhere to the principle of least surprise.
     return pattern.matcher(Bytes.toString(value)).find() ? 1 : 0;
   }
 
-  @Override
   public void readFields(DataInput in) throws IOException {
     this.pattern = Pattern.compile(in.readUTF());
   }
 
-  @Override
   public void write(DataOutput out) throws IOException {
     out.writeUTF(pattern.toString());
   }

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java Wed Dec  3 13:03:10 2008
@@ -56,17 +56,14 @@
     this.substr = substr.toLowerCase();
   }
 
-  @Override
   public int compareTo(byte[] value) {
     return Bytes.toString(value).toLowerCase().contains(substr) ? 1 : 0;
   }
 
-  @Override
   public void readFields(DataInput in) throws IOException {
     substr = in.readUTF();
   }
 
-  @Override
   public void write(DataOutput out) throws IOException {
     out.writeUTF(substr);
   }

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java Wed Dec  3 13:03:10 2008
@@ -35,6 +35,9 @@
  * This is used to partition the output keys into groups of keys.
  * Keys are grouped according to the regions that currently exist
  * so that each reducer fills a single region so load is distributed.
+ * 
+ * @param <K2>
+ * @param <V2>
  */
 public class HRegionPartitioner<K2,V2> 
 implements Partitioner<ImmutableBytesWritable, V2> {
@@ -57,9 +60,8 @@
     }
   }
 
-  @Override
-  public int getPartition(ImmutableBytesWritable key, V2 value,
-    int numPartitions) {
+  public int getPartition(ImmutableBytesWritable key,
+      @SuppressWarnings("unused") V2 value, int numPartitions) {
     byte[] region = null;
     // Only one region return 0
     if (this.startKeys.length == 1){
@@ -78,9 +80,8 @@
           // cover if we have less reduces then regions.
           return (Integer.toString(i).hashCode() 
               & Integer.MAX_VALUE) % numPartitions;
-        } else {
-          return i;
         }
+        return i;
       }
     }
     // if above fails to find start key that match we need to return something

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/master/HMaster.java Wed Dec  3 13:03:10 2008
@@ -590,14 +590,27 @@
     if (!isMasterRunning()) {
       throw new MasterNotRunningException();
     }
-    // We can not create a table unless meta regions have already been
-    // assigned and scanned.
-    if (!regionManager.areAllMetaRegionsOnline()) {
-      throw new NotAllMetaRegionsOnlineException();
-    }
     HRegionInfo newRegion = new HRegionInfo(desc, null, null);
-    createTable(newRegion);
-    LOG.info("created table " + desc.getNameAsString());
+
+    for (int tries = 0; tries < numRetries; tries++) {
+      try {
+        // We can not create a table unless meta regions have already been
+        // assigned and scanned.
+        if (!regionManager.areAllMetaRegionsOnline()) {
+          throw new NotAllMetaRegionsOnlineException();
+        }
+        createTable(newRegion);
+        LOG.info("created table " + desc.getNameAsString());
+        break;
+      } catch (TableExistsException e) {
+        throw e;
+      } catch (IOException e) {
+        if (tries == numRetries - 1) {
+          throw RemoteExceptionHandler.checkIOException(e);
+        }
+        sleeper.sleep();
+      }
+    }
   }
 
   private synchronized void createTable(final HRegionInfo newRegion) 

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=723070&r1=723069&r2=723070&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Wed Dec  3 13:03:10 2008
@@ -170,9 +170,10 @@
   /** region server process name */
   public static final String REGIONSERVER = "regionserver";
   
-  /**
+  /*
    * Space is reserved in HRS constructor and then released when aborting
-   * to recover from an OOME. See HBASE-706.
+   * to recover from an OOME. See HBASE-706.  TODO: Make this percentage of the
+   * heap or a minimum.
    */
   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte []>();
   
@@ -197,6 +198,9 @@
   // flag set after we're done setting up server threads (used for testing)
   protected volatile boolean isOnline;
 
+  final Map<String, InternalScanner> scanners =
+    new ConcurrentHashMap<String, InternalScanner>();
+
   /**
    * Starts a HRegionServer at the default location
    * @param conf
@@ -443,9 +447,10 @@
         sleeper.sleep(lastMsg);
       } // for
     } catch (Throwable t) {
-      checkOOME(t);
-      LOG.fatal("Unhandled exception. Aborting...", t);
-      abort();
+      if (!checkOOME(t)) {
+        LOG.fatal("Unhandled exception. Aborting...", t);
+        abort();
+      }
     }
     RegionHistorian.getInstance().offline();
     this.leases.closeAfterLeasesExpire();
@@ -476,9 +481,9 @@
             this.log.close();
             LOG.info("On abort, closed hlog");
           }
-        } catch (IOException e) {
+        } catch (Throwable e) {
           LOG.error("Unable to close log in abort",
-              RemoteExceptionHandler.checkIOException(e));
+            RemoteExceptionHandler.checkThrowable(e));
         }
         closeAllRegions(); // Don't leave any open file handles
       }
@@ -488,9 +493,9 @@
       ArrayList<HRegion> closedRegions = closeAllRegions();
       try {
         log.closeAndDelete();
-      } catch (IOException e) {
+      } catch (Throwable e) {
         LOG.error("Close and delete failed",
-            RemoteExceptionHandler.checkIOException(e));
+          RemoteExceptionHandler.checkThrowable(e));
       }
       try {
         HMsg[] exitMsg = new HMsg[closedRegions.size() + 1];
@@ -505,9 +510,9 @@
         LOG.info("telling master that region server is shutting down at: " +
             serverInfo.getServerAddress().toString());
         hbaseMaster.regionServerReport(serverInfo, exitMsg, (HRegionInfo[])null);
-      } catch (IOException e) {
+      } catch (Throwable e) {
         LOG.warn("Failed to send exiting message to master: ",
-            RemoteExceptionHandler.checkIOException(e));
+          RemoteExceptionHandler.checkThrowable(e));
       }
       LOG.info("stopping server at: " +
         serverInfo.getServerAddress().toString());
@@ -554,15 +559,11 @@
       this.metrics = new RegionServerMetrics();
       startServiceThreads();
       isOnline = true;
-    } catch (IOException e) {
+    } catch (Throwable e) {
+      this.isOnline = false;
       this.stopRequested.set(true);
-      checkOOME(e);
-      isOnline = false;
-      e = RemoteExceptionHandler.checkIOException(e); 
-      LOG.fatal("Failed init", e);
-      IOException ex = new IOException("region server startup failed");
-      ex.initCause(e);
-      throw ex;
+      throw convertThrowableToIOE(cleanup(e, "Failed init"),
+        "Region server startup failed");
     }
   }
 
@@ -601,20 +602,88 @@
   }
 
   /*
+   * Cleanup after Throwable caught invoking method.  Converts <code>t</code>
+   * to IOE if it isn't already.
+   * @param t Throwable
+   * @return Throwable converted to an IOE; methods can only let out IOEs.
+   */
+  private Throwable cleanup(final Throwable t) {
+    return cleanup(t, null);
+  }
+
+  /*
+   * Cleanup after Throwable caught invoking method.  Converts <code>t</code>
+   * to IOE if it isn't already.
+   * @param t Throwable
+   * @param msg Message to log in error.  Can be null.
+   * @return Throwable converted to an IOE; methods can only let out IOEs.
+   */
+  private Throwable cleanup(final Throwable t, final String msg) {
+    if (msg == null) {
+      LOG.error(RemoteExceptionHandler.checkThrowable(t));
+    } else {
+      LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
+    }
+    if (!checkOOME(t)) {
+      checkFileSystem();
+    }
+    return t;
+  }
+
+  /*
+   * @param t
+   * @return Make <code>t</code> an IOE if it isn't already.
+   */
+  private IOException convertThrowableToIOE(final Throwable t) {
+    return convertThrowableToIOE(t, null);
+  }
+
+  /*
+   * @param t
+   * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
+   * @return Make <code>t</code> an IOE if it isn't already.
+   */
+  private IOException convertThrowableToIOE(final Throwable t,
+      final String msg) {
+    return (t instanceof IOException? (IOException)t:
+      msg == null || msg.length() == 0?
+        new IOException(t): new IOException(msg, t));
+  }
+  /*
    * Check if an OOME and if so, call abort.
    * @param e
    * @return True if we OOME'd and are aborting.
    */
   private boolean checkOOME(final Throwable e) {
-    boolean aborting = false;
+    boolean stop = false;
     if (e instanceof OutOfMemoryError ||
         (e.getCause()!= null && e.getCause() instanceof OutOfMemoryError) ||
         e.getMessage().contains("java.lang.OutOfMemoryError")) {
       LOG.fatal("OutOfMemoryError, aborting.", e);
       abort();
-      aborting = true;
+      stop = true;
+    }
+    return stop;
+  }
+  
+  
+  /**
+   * Checks to see if the file system is still accessible.
+   * If not, sets abortRequested and stopRequested
+   * 
+   * @return false if file system is not available
+   */
+  protected boolean checkFileSystem() {
+    if (this.fsOk && this.fs != null) {
+      try {
+        FSUtils.checkFileSystemAvailable(this.fs);
+      } catch (IOException e) {
+        LOG.fatal("Shutting down HRegionServer: file system not available", e);
+        abort();
+        this.fsOk = false;
+      }
     }
-    return aborting;
+    return this.fsOk;
   }
 
   /*
@@ -1088,8 +1157,9 @@
           }
         }
       } catch(Throwable t) {
-        checkOOME(t);
-        LOG.fatal("Unhandled exception", t);
+        if (!checkOOME(t)) {
+          LOG.fatal("Unhandled exception", t);
+        }
       } finally {
         LOG.info("worker thread exiting");
       }
@@ -1110,14 +1180,13 @@
         // Startup a compaction early if one is needed.
         this.compactSplitThread.
           compactionRequested(region, "Region open check");
-      } catch (IOException e) {
-        checkOOME(e);
-        LOG.error("error opening region " + regionInfo.getRegionNameAsString(),
-          e);
+      } catch (Throwable e) {
+        Throwable t = cleanup(e,
+          "Error opening " + regionInfo.getRegionNameAsString());
         // TODO: add an extra field in HRegionInfo to indicate that there is
         // an error. We can't do that now because that would be an incompatible
         // change that would require a migration
-        reportClose(regionInfo, StringUtils.stringifyException(e).getBytes());
+        reportClose(regionInfo, StringUtils.stringifyException(t).getBytes());
         return;
       }
       this.lock.writeLock().lock();
@@ -1182,11 +1251,8 @@
       }
       try {
         region.close(abortRequested);
-      } catch (IOException e) {
-        LOG.error("error closing region " +
-            Bytes.toString(region.getRegionName()),
-          RemoteExceptionHandler.checkIOException(e));
-        checkOOME(e);
+      } catch (Throwable e) {
+        cleanup(e, "Error closing " + Bytes.toString(region.getRegionName()));
       }
     }
     return regionsToClose;
@@ -1210,9 +1276,9 @@
           LOG.debug("Closing region " + r.toString());
         }
         r.close();
-      } catch (IOException e) {
+      } catch (Throwable e) {
         LOG.error("Error closing region " + r.toString(),
-          RemoteExceptionHandler.checkIOException(e));
+          RemoteExceptionHandler.checkThrowable(e));
       }
     }
   }
@@ -1280,9 +1346,8 @@
     requestCount.incrementAndGet();
     try {
       return getRegion(regionName).get(row, column, timestamp, numVersions);
-    } catch (IOException e) {
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
 
@@ -1307,10 +1372,8 @@
       if (result == null || result.isEmpty())
         return null;
       return new RowResult(row, result);
-    } catch (IOException e) {
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
 
@@ -1325,10 +1388,8 @@
       // ask the region for all the data 
       RowResult rr = region.getClosestRowBefore(row, columnFamily);
       return rr;
-    } catch (IOException e) {
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
   
@@ -1362,10 +1423,8 @@
         }
       }
       return resultSets.toArray(new RowResult[resultSets.size()]);
-    } catch (IOException e) {
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
 
@@ -1382,10 +1441,8 @@
     try {
       cacheFlusher.reclaimMemcacheMemory();
       region.batchUpdate(b, getLockFromId(b.getRowLock()));
-    } catch (IOException e) {
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
   
@@ -1407,10 +1464,8 @@
       return i;
     } catch (NotServingRegionException ex) {
       return i;
-    } catch (IOException e) {
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
     return -1;
   }
@@ -1458,9 +1513,7 @@
       npe = new NullPointerException("firstRow for scanner is null");
     }
     if (npe != null) {
-      IOException io = new IOException("Invalid arguments to openScanner");
-      io.initCause(npe);
-      throw io;
+      throw new IOException("Invalid arguments to openScanner", npe);
     }
     requestCount.incrementAndGet();
     try {
@@ -1469,12 +1522,8 @@
         r.getScanner(cols, firstRow, timestamp, filter);
       long scannerId = addScanner(s);
       return scannerId;
-    } catch (IOException e) {
-      LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")",
-        RemoteExceptionHandler.checkIOException(e));
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
     }
   }
   
@@ -1491,9 +1540,9 @@
   }
   
   public void close(final long scannerId) throws IOException {
-    checkOpen();
-    requestCount.incrementAndGet();
     try {
+      checkOpen();
+      requestCount.incrementAndGet();
       String scannerName = String.valueOf(scannerId);
       InternalScanner s = null;
       synchronized(scanners) {
@@ -1504,18 +1553,11 @@
       }
       s.close();
       this.leases.cancelLease(scannerName);
-    } catch (IOException e) {
-      // TODO: Should we even be returning an exception out of a close?
-      // What can the client do with an exception in close?
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
 
-  Map<String, InternalScanner> scanners =
-    new ConcurrentHashMap<String, InternalScanner>();
-
   /** 
    * Instantiated as a scanner lease.
    * If the lease times out, the scanner is closed
@@ -1561,7 +1603,6 @@
     region.deleteAll(row, timestamp, getLockFromId(lockId));
   }
 
-  @Override
   public void deleteAllByRegex(byte[] regionName, byte[] row, String colRegex,
       long timestamp, long lockId) throws IOException {
     getRegion(regionName).deleteAllByRegex(row, colRegex, timestamp, 
@@ -1575,7 +1616,6 @@
         getLockFromId(lockId));
   }
 
-  @Override
   public void deleteFamilyByRegex(byte[] regionName, byte[] row, String familyRegex,
       long timestamp, long lockId) throws IOException {
     getRegion(regionName).deleteFamilyByRegex(row, familyRegex, timestamp, 
@@ -1603,12 +1643,9 @@
       long lockId = addRowLock(r,region);
       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
       return lockId;
-    } catch (IOException e) {
-      LOG.error("Error obtaining row lock (fsOk: " + this.fsOk + ")",
-        RemoteExceptionHandler.checkIOException(e));
-      checkOOME(e);
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t,
+        "Error obtaining row lock (fsOk: " + this.fsOk + ")"));
     }
   }
 
@@ -1676,9 +1713,8 @@
       region.releaseRowLock(r);
       this.leases.cancelLease(lockName);
       LOG.debug("Row lock " + lockId + " has been explicitly released by client");
-    } catch (IOException e) {
-      checkFileSystem();
-      throw e;
+    } catch (Throwable t) {
+      throw convertThrowableToIOE(cleanup(t));
     }
   }
 
@@ -1878,25 +1914,6 @@
       throw new IOException("File system not available");
     }
   }
-  
-  /**
-   * Checks to see if the file system is still accessible.
-   * If not, sets abortRequested and stopRequested
-   * 
-   * @return false if file system is not available
-   */
-  protected boolean checkFileSystem() {
-    if (this.fsOk && fs != null) {
-      try {
-        FSUtils.checkFileSystemAvailable(fs);
-      } catch (IOException e) {
-        LOG.fatal("Shutting down HRegionServer: file system not available", e);
-        abort();
-        fsOk = false;
-      }
-    }
-    return this.fsOk;
-  }
  
   /**
    * @return Returns list of non-closed regions hosted on this server.  If no
@@ -1922,7 +1939,7 @@
   }
 
   public long getProtocolVersion(final String protocol, 
-      final long clientVersion)
+      @SuppressWarnings("unused") final long clientVersion)
   throws IOException {  
     if (protocol.equals(HRegionInterface.class.getName())) {
       return HBaseRPCProtocolVersion.versionID;
@@ -2054,4 +2071,4 @@
         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
     doMain(args, regionServerClass);
   }
-}
+}
\ No newline at end of file