You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "Ding Yuan (JIRA)" <ji...@apache.org> on 2014/02/02 20:40:08 UTC

[jira] [Created] (HBASE-10452) Potential bugs in exception handlers

Ding Yuan created HBASE-10452:
---------------------------------

             Summary: Potential bugs in exception handlers
                 Key: HBASE-10452
                 URL: https://issues.apache.org/jira/browse/HBASE-10452
             Project: HBase
          Issue Type: Bug
          Components: Client, master, regionserver, util
    Affects Versions: 0.96.1
            Reporter: Ding Yuan


Hi HBase developers,
We are a group of researchers on software reliability. Recently we did a study and found that majority of the most severe failures in HBase are caused by bugs in exception handling logic -- that it is hard to anticipate all the possible real-world error scenarios. Therefore we built a simple checking tool that automatically detects some bug patterns that have caused some very severe real-world failures. I am reporting some of the results here. Any feedback is much appreciated!

=========================
Case 1:
  Line: 134, File: "org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java"

{noformat}
  protected void releaseTableLock() {
    if (this.tableLock != null) {
      try {
        this.tableLock.release();
      } catch (IOException ex) {
        LOG.warn("Could not release the table lock", ex);
        //TODO: if we get here, and not abort RS, this lock will never be released
      }
    }
{noformat}

The lock is not released if the exception occurs, causing potential deadlock or starvation.

Similar code pattern can be found at:
  Line: 135, File: "org/apache/hadoop/hbase/regionserver/SplitRequest.java"
==========================================

=========================
Case 2:
  Line: 252, File: "org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java"

{noformat}
    try {
      Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
      fEnd.setAccessible(true);
      end = fEnd.getLong(this.reader);
    } catch(Exception e) { /* reflection fail. keep going */ }
{noformat}

The caught Exception seems to be too general.
While reflection-related errors might be harmless, the try block can throw
other exceptions including "SecurityException", "IllegalAccessException", etc. Currently
all those exceptions are ignored. Maybe
the safe way is to ignore the specific reflection-related errors while logging and
handling other types of unexpected exceptions.
==========================================
=========================
Case 3:
  Line: 148, File: "org/apache/hadoop/hbase/HBaseConfiguration.java"

{noformat}
    try {
      if (Class.forName("org.apache.hadoop.conf.ConfServlet") != null) {
        isShowConf = true;
      }
    } catch (Exception e) {
    }
{noformat}

Similar to the previous case, the exception handling is too general. While ClassNotFound error might be the normal case and ignored, Class.forName can also throw other exceptions (e.g., LinkageError) under some unexpected and rare error cases. If that happens, the error will be lost. So maybe change it to below:

{noformat}
    try {
      if (Class.forName("org.apache.hadoop.conf.ConfServlet") != null) {
        isShowConf = true;
      }
    } catch (LinkageError e) {
      LOG.warn("..");
      // handle linkage error
    } catch (ExceptionInInitializerError e) {
      LOG.warn("..");
      // handle Initializer error
    } catch (ClassNotFoundException e) {
     LOG.debug("..");
     // ignore
    }
{noformat}
==========================================
=========================
Case 4:
  Line: 163, File: "org/apache/hadoop/hbase/client/Get.java"

{noformat}
  public Get setTimeStamp(long timestamp) {
    try {
      tr = new TimeRange(timestamp, timestamp+1);
    } catch(IOException e) {
      // Will never happen
    }
    return this;
  }
{noformat}

Even if the IOException never happens right now, is it possible to happen in the future due to code change?
At least there should be a log message. The current behavior is dangerous since if the exception ever happens
in any unexpected scenario, it will be silently swallowed.

Similar code pattern can be found at:
  Line: 300, File: "org/apache/hadoop/hbase/client/Scan.java"
==========================================

=========================
Case 5:
  Line: 207, File: "org/apache/hadoop/hbase/util/JVM.java"

{noformat}
   if (input != null){
        try {
          input.close();
        } catch (IOException ignored) {
        }
      }
{noformat}

Any exception encountered in close is completely ignored, not even logged.
In particular, the same exception scenario was handled differently in other methods in the same file:
Line: 154, same file
{noformat}
       if (in != null){
         try {
           in.close();
         } catch (IOException e) {
           LOG.warn("Not able to close the InputStream", e);
         }
       }
{noformat}

Line: 248, same file

{noformat}
      if (in != null){
        try {
          in.close();
        } catch (IOException e) {
          LOG.warn("Not able to close the InputStream", e);
        }
      }
{noformat}
==========================================

=========================
Case 6: empty handler for exception: java.io.IOException
  Line: 312, File: "org/apache/hadoop/hbase/rest/RowResource.java"

{noformat}
    } finally {
      if (table != null) try {
        table.close();
      } catch (IOException ioe) { }
    }
{noformat}

IOException is completely ignored. This behavior is inconsistent with the same
code snippet at line 249 in the same file, where the IOExceptions was logged:

{noformat}
   } finally {
      if (table != null) try {
        table.close();
      } catch (IOException ioe) {
        LOG.debug("Exception received while closing the table", ioe);
      }
    }
{noformat}
==========================================
=========================
Case 7:
  Line: 95, File: "org/apache/hadoop/hbase/master/handler/EnableTableHandler.java"

{noformat}
        try {
          this.assignmentManager.getZKTable().removeEnablingTable(tableName, true);
        } catch (KeeperException e) {
          // TODO : Use HBCK to clear such nodes
          LOG.warn("Failed to delete the ENABLING node for the table " + tableName
              + ".  The table will remain unusable. Run HBCK to manually fix the problem.");
        }
{noformat}

The log message in the exception handler and the comment seem to suggest that such nodes should be cleared using HBCK.
==========================================


=========================
Case 8:
  Line: 463, File: "org/apache/hadoop/hbase/client/ClientScanner.java"

{noformat}
        try {
          this.caller.callWithRetries(callable);
        } catch (IOException e) {
          // We used to catch this error, interpret, and rethrow. However, we
          // have since decided that it's not nice for a scanner's close to
          // throw exceptions. Chances are it was just an UnknownScanner
          // exception due to lease time out.
        }
{noformat}

Currently the handler is empty because it may be caused by "just an UnknownScanner exception".
But what if it has other causes? Maybe the catch block can differentiate the exception into
different causes, ignoring the ones caused by UnknownScanner while handle others differently?
==========================================



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)