hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stack <st...@duboce.net>
Subject Re: corrupt WAL and Java Heap Space...
Date Fri, 15 Jul 2011 18:59:06 GMT
I'd have expected the log to be recoverable up to the last time you
called sync.  What were you seeing?  Do you have the log still?  (It
should recover to the last edit)

St.Ack

On Fri, Jul 15, 2011 at 11:32 AM, Andy Sautins
<andy.sautins@returnpath.net> wrote:
>
>  Thanks.  I filed JIRA HBASE-4107 ( https://issues.apache.org/jira/browse/HBASE-4107
).
>
>  It does seem like the OOME is causing a write to the WAL to be left in an inconsistent
state.  I haven't had a chance to look yet, but it would seem that the flush isn't atomic,
so possibly the data was synced but the checksum wasn't able to be updated.  If that logic
is right then it would be an issue in the sync to hdfs.
>
>  In either case it is sad that the log looks like it could get left in an unusable state.
 That seems like the last thing we'd really want.  Not sure about keeping a reservoir of
memory around.  It seems you could free just about anything to let the write finish and then
exit potentially ungracefully.  The WAL would need to be recovered, but that's much preferable
to data loss.
>
>  I need to look further but it does feel like the full sync is not atomic and failing
somewhere before the checksum is fully written out can potentially lead to WAL corruption.
 That's a guess.  I need to look at it further.
>
>  Thanks
>
>  Andy
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
> Sent: Friday, July 15, 2011 10:41 AM
> To: user@hbase.apache.org
> Subject: Re: corrupt WAL and Java Heap Space...
>
> Please file an issue.  Sounds like an OOME while writing causes us to
> exit w/o closing the WAL (You think that the case)?  My guess is that
> in this low memory situation, a close might fail anyways (with another
> OOME) unless we did some extra gymnastics releasing the little
> resevoir of memory we keep around to release so cleanup succeeds
> whenever we see OOME.
>
> St.Ack
>
> On Fri, Jul 15, 2011 at 9:32 AM, Andy Sautins
> <andy.sautins@returnpath.net> wrote:
>>
>>   Yesterday we ran into an interesting issue.  We were shutting down our HBase
cluster ( 0.90.1 CDH3u0 ) and in the process one of the nodes encountered a Java heap space
exception.  The bummer is the log file was listed as corrupt from hadoop fsck and was unable
to be read when re-starting the database.  We were able to recover in our situation by removing
the corrupt log and did not appear to lose any data.
>>
>>    Has anyone else seen this issue?  If I'm reading the situation right it looks
like that a Java heap space error during the WAL checksum write could leave the WAL corrupt
which doesn't seem like desired behavior.
>>
>>    I'll looking into it further but any thoughts would be appreciated.
>>
>>
>> 2011-07-14 14:54:53,741 FATAL org.apache.hadoop.hbase.regionserver.wal.HLog: Could
not append. Requesting close of hlog
>> java.io.IOException: Reflection
>>        at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:147)
>>        at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:987)
>>        at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:964)
>> Caused by: java.lang.reflect.InvocationTargetException
>>        at sun.reflect.GeneratedMethodAccessor1336.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:145)
>>        ... 2 more
>> Caused by: java.lang.OutOfMemoryError: Java heap space
>>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$Packet.<init>(DFSClient.java:2375)
>>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:3271)
>>        at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:150)
>>        at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:132)
>>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3354)
>>        at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97)
>>        at org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:944)
>>        ... 6 more
>>
>>
>

Mime
View raw message