hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Anoop Sam John (JIRA)" <j...@apache.org>
Subject [jira] [Created] (HBASE-11145) Issue with HLog sync
Date Sat, 10 May 2014 22:15:04 GMT
Anoop Sam John created HBASE-11145:
--------------------------------------

             Summary: Issue with HLog sync
                 Key: HBASE-11145
                 URL: https://issues.apache.org/jira/browse/HBASE-11145
             Project: HBase
          Issue Type: Bug
            Reporter: Anoop Sam John


Got the below Exceptions Log in case of a write heavy test

{code}
2014-05-07 11:29:56,417 ERROR [main.append-pool1-t1] wal.FSHLog$RingBufferEventHandler(1882):
UNEXPECTED!!!
java.lang.IllegalStateException: Queue full
 at java.util.AbstractQueue.add(Unknown Source)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$SyncRunner.offer(FSHLog.java:1227)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1878)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1)
 at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:133)
 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
2014-05-07 11:29:56,418 ERROR [main.append-pool1-t1] wal.FSHLog$RingBufferEventHandler(1882):
UNEXPECTED!!!
java.lang.ArrayIndexOutOfBoundsException: 5
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1838)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1)
 at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:133)
 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
2014-05-07 11:29:56,419 ERROR [main.append-pool1-t1] wal.FSHLog$RingBufferEventHandler(1882):
UNEXPECTED!!!
java.lang.ArrayIndexOutOfBoundsException: 6
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1838)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1)
 at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:133)
 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)
2014-05-07 11:29:56,419 ERROR [main.append-pool1-t1] wal.FSHLog$RingBufferEventHandler(1882):
UNEXPECTED!!!
java.lang.ArrayIndexOutOfBoundsException: 7
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1838)
 at org.apache.hadoop.hbase.regionserver.wal.FSHLog$RingBufferEventHandler.onEvent(FSHLog.java:1)
 at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:133)
 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
 at java.lang.Thread.run(Unknown Source)

 {code}

In FSHLog$SyncRunner.offer we do BlockingQueue.add() which throws Exception as it is full.
The problem is the below shown catch() we do not do any cleanup.
{code}
this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount);
        attainSafePoint(sequence);
        this.syncFuturesCount = 0;
      } catch (Throwable t) {
        LOG.error("UNEXPECTED!!!", t);
      }
{code}
syncFuturesCount is not getting reset to 0 and so the subsequent onEvent() handling throws
ArrayIndexOutOfBoundsException.

I think we should do the below 
1. Handle the Exception and call cleanupOutstandingSyncsOnException() as in other cases of
Exception handling
2. Instead of BlockingQueue.add() use offer() (?)




--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message