hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Enis Soztutar (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5623) Race condition when rolling the HLog and hlogFlush
Date Fri, 23 Mar 2012 18:27:30 GMT

    [ https://issues.apache.org/jira/browse/HBASE-5623?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13236889#comment-13236889
] 

Enis Soztutar commented on HBASE-5623:
--------------------------------------

@Stack, thanks for the unit test. I'll test the the unit test with and without the patch today.
Normally, it is very easy to replicate this bug. I just set up a 4 node cluster with Hadoop-1.0.1,
and run ycsb with -load -threads 30.  

@Lars
Yes, it is the same with HBASE-5336. I had searched for the exception but did not find that
issue. Anyway I'll close that one as duplicate. 
bq. If we want maximum concurrency here, we need to do the same we do for writer.sync(). I.e.
try without lock held, if failing get the lock and retry, along with SequenceFileLogWriter.getLength()
catching Exception and retrow as IOException.
Yes we can do that, but we cannot do this double-checking for the logSyncerThread.hlogFlush(this.writer)
since it is not idempotent, so in any case we should be holding the updateLock, no? From my
understanding you are suggesting that we sync on the hlogFlush, and do the retry mechanism
in writer.sync and getLength(), catching all the exceptions in the first try, right? 
                
> Race condition when rolling the HLog and hlogFlush
> --------------------------------------------------
>
>                 Key: HBASE-5623
>                 URL: https://issues.apache.org/jira/browse/HBASE-5623
>             Project: HBase
>          Issue Type: Bug
>          Components: wal
>    Affects Versions: 0.94.0
>            Reporter: Enis Soztutar
>            Assignee: Enis Soztutar
>            Priority: Critical
>         Attachments: 5623v2.txt, HBASE-5623_v0.patch
>
>
> When doing a ycsb test with a large number of handlers (regionserver.handler.count=60),
I get the following exceptions:
> {code}
> Caused by: org.apache.hadoop.ipc.RemoteException: java.io.IOException: java.lang.NullPointerException
> 	at org.apache.hadoop.io.SequenceFile$Writer.getLength(SequenceFile.java:1099)
> 	at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.getLength(SequenceFileLogWriter.java:314)
> 	at org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1291)
> 	at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1388)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:2192)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1985)
> 	at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3400)
> 	at sun.reflect.GeneratedMethodAccessor17.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 	at java.lang.reflect.Method.invoke(Method.java:597)
> 	at org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:366)
> 	at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1351)
> 	at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:920)
> 	at org.apache.hadoop.hbase.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:152)
> 	at $Proxy1.multi(Unknown Source)
> 	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1691)
> 	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1689)
> 	at org.apache.hadoop.hbase.client.ServerCallable.withoutRetries(ServerCallable.java:214)
> {code}
> and 
> {code}
> 	java.lang.NullPointerException
> 		at org.apache.hadoop.io.SequenceFile$Writer.checkAndWriteSync(SequenceFile.java:1026)
> 		at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1068)
> 		at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1035)
> 		at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.append(SequenceFileLogWriter.java:279)
> 		at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.hlogFlush(HLog.java:1237)
> 		at org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1271)
> 		at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1391)
> 		at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:2192)
> 		at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1985)
> 		at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3400)
> 		at sun.reflect.GeneratedMethodAccessor33.invoke(Unknown Source)
> 		at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 		at java.lang.reflect.Method.invoke(Method.java:597)
> 		at org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:366)
> 		at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1351)
> {code}
> It seems the root cause of the issue is that we open a new log writer and close the old
one at HLog#rollWriter() holding the updateLock, but the other threads doing syncer() calls
> {code} 
> logSyncerThread.hlogFlush(this.writer);
> {code}
> without holding the updateLock. LogSyncer only synchronizes against concurrent appends
and flush(), but not on the passed writer, which can be closed already by rollWriter(). In
this case, since SequenceFile#Writer.close() sets it's out field as null, we get the NPE.


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message