Are there any messages about log replay when you restart the region server?
I doubt we would have missed such a serious bug all this time. Reducing the flush time is _not_ the solution. Do you think you can reproduce this with a unit test? Can you file a case? Thanks. -- Lars ________________________________ From: Manuel de Ferran <[email protected]> To: [email protected] Sent: Monday, February 20, 2012 1:58 AM Subject: Re: Flushing to HDFS sooner On Sun, Feb 19, 2012 at 5:00 PM, Ian Varley <[email protected]> wrote: > Manuel, do you have the WAL disabled? > No I did not disable WAL. > > Did you kill *all* the data nodes? Was there anything of note in the logs? > Can you repro this case consistently? Any chance you can try with 0.92 for > comparison? > Yes, I can easily reproduce it on a minimal cluster with a single datanode and a single regionserver. When killing the datanode, I have in regionserver logs : 2012-02-20 08:11:22,342 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: LRU Stats: total=22.6 MB, free=794.94 MB, max=817.54 MB, blocks=331, accesses=6562, hits=6231, hitRatio=94.95%%, cachingAccesses=6562, cachingHits=6231, cachingHitsRatio=94.95%%, evictions=0, evicted=0, evictedPerRun=NaN 2012-02-20 08:12:40,538 WARN org.apache.hadoop.hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block blk_77938759638192339_1445java.io.EOFException at java.io.DataInputStream.readFully(DataInputStream.java:180) at java.io.DataInputStream.readLong(DataInputStream.java:399) at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:122) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2548) 2012-02-20 08:12:40,538 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_77938759638192339_1445 bad datanode[0] 192.168.1.167:57102 2012-02-20 08:12:41,102 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:986) at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:963) Caused by: java.lang.reflect.InvocationTargetException at sun.reflect.GeneratedMethodAccessor2.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 <snip> 2012-02-20 08:12:41,104 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: regionserver60020.logSyncer exiting 2012-02-20 08:12:41,107 INFO org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter: Using syncFs -- HDFS-200 2012-02-20 08:12:41,109 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server serverName=lxc167.machine.com,60020,1329725181594, load=(requests=37, regions=27, usedHeap=52, maxHeap=4087): IOE in log roller It looks like that the regionserver had smth to flush on the datanode. So back to my first thoughts : is there any setting to flush more frequently ? I will try the same with HBase 0.92. > > Ian > > On Feb 19, 2012, at 6:45 AM, "Manuel de Ferran" <[email protected]> > wrote: > > > Greetings, > > > > on a testing platform (running HBase-0.90.3 on top of > Hadoop-0.20-append), > > we did the following : > > - create a dummy table > > - put a single row > > - get this row from the shell > > - wait a few minutes > > - kill -9 the datanodes > > > > Because regionservers could not connect to datanodes, they shutdown. > > > > On restart, the row has vanished. But if we do the same and "flush > 'dummy'" > > from the Shell before killing the datanodes, the row is still there. > > > > Is it related to WAL ? MemStores ? What happened ? > > > > What are the recommended settings so rows are auto-flushed or at least > > flushed more frequently ? > > > > > > Regards >
