|
Mikhail Bautin
2011-12-02, 05:59
Dhruba Borthakur
2011-12-02, 06:32
Stack
2011-12-02, 06:33
Mikhail Bautin
2011-12-02, 06:58
Mikhail Bautin
2011-12-02, 07:12
Dhruba Borthakur
2011-12-02, 07:16
Todd Lipcon
2011-12-02, 07:18
Mikhail Bautin
2011-12-02, 07:20
Mikhail Bautin
2011-12-02, 08:54
Stack
2011-12-02, 14:55
Nicolas Spiegelberg
2011-12-02, 16:39
|
-
SequenceFileLogReader uses a reflection hack resulting in runtime failuresMikhail Bautin 2011-12-02, 05:59
Hello,
The following reflection hack is from SequenceFileLogReader.java. try { Field fIn = FilterInputStream.class.getDeclaredField("in"); fIn.setAccessible(true); Object realIn = fIn.get(this.in); Method getFileLength = realIn.getClass(). getMethod("getFileLength", new Class<?> []{}); getFileLength.setAccessible(true); long realLength = ((Long)getFileLength. invoke(realIn, new Object []{})).longValue(); assert(realLength >= this.length); adjust = realLength - this.length; } catch(Exception e) { SequenceFileLogReader.LOG.warn( "Error while trying to get accurate file length. " + "Truncation / data loss may occur if RegionServers die.", e); } In my testing this resulted in the following error: 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() at java.lang.Class.getMethod(Class.java:1605) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader$WALReader$WALReaderFSDataInputStream.getPos(SequenceFileLogReader.java:108) at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1434) at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1424) at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1419) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader$WALReader.<init>(SequenceFileLogReader.java:57) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.init(SequenceFileLogReader.java:158) at org.apache.hadoop.hbase.regionserver.wal.HLog.getReader(HLog.java:681) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getReader(HLogSplitter.java:821) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getReader(HLogSplitter.java:734) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:382) 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) at java.lang.Thread.run(Thread.java:680) 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? Thanks, --Mikhail
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresDhruba Borthakur 2011-12-02, 06:32
This reflection should occur only once, not at every write to the HLog, so
the performance impact should be minimal, is it not? why are you seeing the exception now? are u using a new unit test or a new hdfs jar? On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin < [EMAIL PROTECTED]> wrote: > Hello, > > The following reflection hack is from SequenceFileLogReader.java. > > try { > Field fIn = FilterInputStream.class.getDeclaredField("in"); > fIn.setAccessible(true); > Object realIn = fIn.get(this.in); > Method getFileLength = realIn.getClass(). > getMethod("getFileLength", new Class<?> []{}); > getFileLength.setAccessible(true); > long realLength = ((Long)getFileLength. > invoke(realIn, new Object []{})).longValue(); > assert(realLength >= this.length); > adjust = realLength - this.length; > } catch(Exception e) { > SequenceFileLogReader.LOG.warn( > "Error while trying to get accurate file length. " + > "Truncation / data loss may occur if RegionServers die.", e); > } > > In my testing this resulted in the following error: > > 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() > at java.lang.Class.getMethod(Class.java:1605) > at > > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader$WALReader$WALReaderFSDataInputStream.getPos(SequenceFileLogReader.java:108) > at > org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1434) > at > org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1424) > at > org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1419) > at > > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader$WALReader.<init>(SequenceFileLogReader.java:57) > at > > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.init(SequenceFileLogReader.java:158) > at > org.apache.hadoop.hbase.regionserver.wal.HLog.getReader(HLog.java:681) > at > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getReader(HLogSplitter.java:821) > at > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getReader(HLogSplitter.java:734) > at > > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:382) > 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) > at java.lang.Thread.run(Thread.java:680) > > 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? > > Thanks, > --Mikhail > -- Subscribe to my posts at http://www.facebook.com/dhruba
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresStack 2011-12-02, 06:33
On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin
<[EMAIL PROTECTED]> 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
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresMikhail Bautin 2011-12-02, 06:58
@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 <[EMAIL PROTECTED]> wrote: > On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin > <[EMAIL PROTECTED]> 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 >
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresMikhail Bautin 2011-12-02, 07:12
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 < [EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]> wrote: > >> On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin >> <[EMAIL PROTECTED]> 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
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresDhruba Borthakur 2011-12-02, 07:16
what hadoop version are you using?
On Thu, Dec 1, 2011 at 11:12 PM, Mikhail Bautin < [EMAIL PROTECTED]> 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 < > [EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]> wrote: > > > >> On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin > >> <[EMAIL PROTECTED]> 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: Subscribe to my posts at http://www.facebook.com/dhruba
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresTodd Lipcon 2011-12-02, 07:18
ChecksumFileSystem doesn't support hflush/sync()/etc -- so I can
imagine if you kill -9 it while writing you'd get a truncated commit log, or even one where the last checksum chunk is incorrect. Maybe best to run this test against a pseudo-distributed HDFS? Or RawLocalFileSystem? -Todd On Thu, Dec 1, 2011 at 10:58 PM, Mikhail Bautin <[EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]> wrote: > >> On Thu, Dec 1, 2011 at 9:59 PM, Mikhail Bautin >> <[EMAIL PROTECTED]> 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 >> -- Todd Lipcon Software Engineer, Cloudera
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresMikhail Bautin 2011-12-02, 07:20
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 <[EMAIL PROTECTED]> wrote: > what hadoop version are you using? > > > On Thu, Dec 1, 2011 at 11:12 PM, Mikhail Bautin < > [EMAIL PROTECTED]> 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 < > > [EMAIL PROTECTED]> 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.
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresMikhail Bautin 2011-12-02, 08:54
OK, I've switched to RawLocalFileSystem and it seemed to fix log splitting
issues. However, I'm still seeing the following when loading random data (not killing the regionserver yet). Any idea about what this could be? java.util.concurrent.ExecutionException: java.io.IOException: java.io.IOException: java.lang.NullPointerException at org.apache.hadoop.io.SequenceFile$Writer.checkAndWriteSync(SequenceFile.java:975) at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1017) at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:984) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.append(SequenceFileLogWriter.java:208) at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.hlogFlush(HLog.java:1223) at org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1253) at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1370) at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:2024) at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1818) at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3180) at sun.reflect.GeneratedMethodAccessor11.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:364) at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1325) at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatchCallback(HConnectionManager.java:1557) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatch(HConnectionManager.java:1409) at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:892) at org.apache.hadoop.hbase.client.HTable.doPut(HTable.java:750) at org.apache.hadoop.hbase.client.HTable.put(HTable.java:725) at org.apache.hadoop.hbase.util.MultiThreadedWriter$HBaseWriter.putIntoTables(MultiThreadedWriter.java:200) at org.apache.hadoop.hbase.util.MultiThreadedWriter$HBaseWriter.bulkInsertKey(MultiThreadedWriter.java:185) at org.apache.hadoop.hbase.util.MultiThreadedWriter$HBaseWriter.run(MultiThreadedWriter.java:132) Caused by: java.io.IOException: java.io.IOException: java.lang.NullPointerException at org.apache.hadoop.io.SequenceFile$Writer.checkAndWriteSync(SequenceFile.java:975) at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1017) at org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:984) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.append(SequenceFileLogWriter.java:208) at org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.hlogFlush(HLog.java:1223) at org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1253) at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1370) at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:2024) at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1818) at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3180) at sun.reflect.GeneratedMethodAccessor11.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:364) at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1325) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) at java.lang.reflect.Constructor.newInstance(Constructor.java:513) at org.apache.hadoop.hbase.RemoteExceptionHandler.decodeRemoteException(RemoteExceptionHandler.java:96) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.translateException(HConnectionManager.java:1652) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.getRegionServerWithoutRetries(HConnectionManager.java:1367) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1383) at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:680) Thanks, On Thu, Dec 1, 2011 at 11:20 PM, Mikhail Bautin < [EMAIL PROTECTED]> wrote:
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresStack 2011-12-02, 14:55
On Thu, Dec 1, 2011 at 10:58 PM, Mikhail Bautin
<[EMAIL PROTECTED]> 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. > This should sink the 0.92.0RC0. I have not seen it in my testing. I was running against the 0.20.205 branch which is some commits beyond 0.20.205 but there is nothing CHANGES.txt that I can see that should have 'fixed' the above. I made HBASE-4935. St.Ack
-
Re: SequenceFileLogReader uses a reflection hack resulting in runtime failuresNicolas Spiegelberg 2011-12-02, 16:39
I think it's good to remove the reflection when we can, more because it's
easier to catch compile-time errors than run-time. The perf is negligible when you cache. As I recall, the problem here is the function was private in older versions. We just need to make sure that we don't support running HBase on top of old versions of Hadoop (0.20.4) before we remove any reflection and change compilation requirements. On 12/2/11 6:55 AM, "Stack" <[EMAIL PROTECTED]> wrote: >On Thu, Dec 1, 2011 at 10:58 PM, Mikhail Bautin ><[EMAIL PROTECTED]> 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. >> > >This should sink the 0.92.0RC0. > >I have not seen it in my testing. I was running against the 0.20.205 >branch which is some commits beyond 0.20.205 but there is nothing >CHANGES.txt that I can see that should have 'fixed' the above. I made >HBASE-4935. > >St.Ack |