Dhruba: It's 0.20.205.0, the default one for the open-source HBase trunk. I'll try to follow Todd's advice and run the test against a different filesystem.
Thanks, --Mikhail On Thu, Dec 1, 2011 at 11:16 PM, Dhruba Borthakur <dhr...@gmail.com> wrote: > what hadoop version are you using? > > > On Thu, Dec 1, 2011 at 11:12 PM, Mikhail Bautin < > bautin.mailing.li...@gmail.com> wrote: > > > After fixing the getFileLength() method access bug, the error I'm seeing > in > > my local multi-process cluster load test is different. Do we ever expect > to > > see checksum errors on the local filesystem? > > > > 11/12/01 22:52:52 INFO wal.HLogSplitter: Splitting hlog: > > file:/tmp/hbase-mbautin/hbase/.logs/192.168.1.25 > > ,54409,1322808663310-splitting/192.168.1.25 > > %2C54409%2C1322808663310.1322808734995, > > length=25989120 > > 11/12/01 22:52:52 INFO fs.FSInputChecker: Found checksum error: b[3627, > > 3627]= > > org.apache.hadoop.fs.ChecksumException: Checksum error: > > file:/tmp/hbase-mbautin/hbase/.logs/192.168.1.25 > > ,54409,1322808663310-splitting/192.168.1.25 > > %2C54409%2C1322808663310.1322808734995 > > at 25689088 > > at > > > > > org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.readChunk(ChecksumFileSystem.java:219) > > at > > > > > org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237) > > at > > org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:189) > > at > org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158) > > at java.io.DataInputStream.read(DataInputStream.java:132) > > at java.io.DataInputStream.readFully(DataInputStream.java:178) > > at > > > > > org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:63) > > at > > org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:101) > > at > > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1937) > > at > > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1837) > > at > > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1883) > > at > > > > > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:203) > > at > > > > > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:177) > > at > > > > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getNextLogLine(HLogSplitter.java:765) > > at > > > > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:402) > > at > > > > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:351) > > at > > > > > org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:113) > > at > > > > > org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:266) > > at > > > > > org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker.java:197) > > at > > > > > org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:165) > > > > Thanks, > > --Mikhail > > > > On Thu, Dec 1, 2011 at 10:58 PM, Mikhail Bautin < > > bautin.mailing.li...@gmail.com> wrote: > > > > > @Stack: I am using hadoop-0.20.205.0 (the default Hadoop version from > > > pom.xml). There is a private getFileLength() method, but getMethod() > does > > > not allow to retrieve it. We should use getDeclaredMethod() -- this > > appears > > > to work in my testing. I will include that fix in the HBaseClusterTest > > > diff. Not sure why no one saw this bug before. > > > > > > @Dhruba: I am running RestartMetaTest, which I am porting from 0.89-fb. > > > This is a test that starts a local HBase cluster as multiple processes > > (on > > > different ports), loads some data, and does a real kill -9 on the > > > regionserver serving meta. I saw this bug in the data loading part, not > > > because of killing the regionserver. > > > > > > Thanks, > > > --Mikhail > > > > > > On Thu, Dec 1, 2011 at 10:33 PM, Stack <st...@duboce.net> wrote: > > > > > >> On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin > > >> <bautin.mailing.li...@gmail.com> wrote: > > >> > 11/12/01 21:40:07 WARN wal.SequenceFileLogReader: Error while trying > > to > > >> get > > >> > accurate file length. Truncation / data loss may occur if > > RegionServers > > >> > die. > > >> > java.lang.NoSuchMethodException: > > >> > > > >> > > > org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.getFileLength() > > >> > > >> Your hadoop doesn't have this Mikhail? > > >> > > >> > Besides, even when it works, the reflection-based solution is > probably > > >> > _much_ slower than straightforward method access. Should we create a > > >> Hadoop > > >> > patch to expose the appropriate API call and get rid of the > reflection > > >> hack? > > >> > > > >> > > >> It would be the sensible thing to do; much more sensible than the > > >> reflection gymnastics we have going on here. > > >> > > >> St.Ack > > >> > > > > > > > > > > > > -- > Subscribe to my posts at http://www.facebook.com/dhruba >