hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] Created: (HADOOP-4562) Logs filled with "IOException: Checksum ok was sent and should not be sent again"
Date Fri, 31 Oct 2008 19:03:44 GMT
Logs filled with "IOException: Checksum ok was sent and should not be sent again"
---------------------------------------------------------------------------------

                 Key: HADOOP-4562
                 URL: https://issues.apache.org/jira/browse/HADOOP-4562
             Project: Hadoop Core
          Issue Type: Bug
          Components: dfs
    Affects Versions: 0.19.0
            Reporter: stack
            Priority: Blocker


Updating hbase to use 0.19.0RC0 or latest from branch-0.19, I see reams of this in logs:

{code}
2008-10-31 18:33:41,296 INFO org.apache.hadoop.fs.FSInputChecker: java.io.IOException: Checksum
ok was sent and should not be sent again
        at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1064)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1613)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1663)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1590)
        at java.io.DataInputStream.readByte(DataInputStream.java:248)
        at org.apache.hadoop.io.WritableUtils.readVLong(WritableUtils.java:325)
        at org.apache.hadoop.io.WritableUtils.readVInt(WritableUtils.java:346)
        at org.apache.hadoop.io.Text.readString(Text.java:400)
        at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1471)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1428)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1417)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1412)
        at org.apache.hadoop.io.MapFile$Reader.open(MapFile.java:293)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$HbaseMapFile$HbaseReader.<init>(HStoreFile.java:632)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader.<init>(HStoreFile.java:714)
        at org.apache.hadoop.hbase.regionserver.HStoreFile.getReader(HStoreFile.java:413)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:262)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1729)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:469)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1004)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:976)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:901)
        at java.lang.Thread.run(Thread.java:619)

2008-10-31 18:33:41,272 DEBUG org.apache.hadoop.hbase.regionserver.HStore: loaded /hbasetrunk/-ROOT-/70236052/info/info/1689673398714621203,
isReference=false, sequence id=1
2008-10-31 18:33:41,274 DEBUG org.apache.hadoop.hbase.regionserver.HStore: Loaded 1 file(s)
in hstore 70236052/info, max sequence id 1
2008-10-31 18:33:41,296 INFO org.apache.hadoop.fs.FSInputChecker: java.io.IOException: Checksum
ok was sent and should not be sent again
        at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1064)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1613)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1663)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1590)
        at java.io.DataInputStream.readByte(DataInputStream.java:248)
        at org.apache.hadoop.io.WritableUtils.readVLong(WritableUtils.java:325)
        at org.apache.hadoop.io.WritableUtils.readVInt(WritableUtils.java:346)
        at org.apache.hadoop.io.Text.readString(Text.java:400)
        at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1471)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1428)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1417)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1412)
        at org.apache.hadoop.io.MapFile$Reader.open(MapFile.java:293)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$HbaseMapFile$HbaseReader.<init>(HStoreFile.java:632)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader.<init>(HStoreFile.java:714)
        at org.apache.hadoop.hbase.regionserver.HStoreFile.getReader(HStoreFile.java:413)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:262)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1729)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:469)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1004)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:976)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:901)
        at java.lang.Thread.run(Thread.java:619)

2008-10-31 18:33:41,298 INFO org.apache.hadoop.fs.FSInputChecker: java.io.IOException: Checksum
ok was sent and should not be sent again
        at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1064)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1613)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1663)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at org.apache.hadoop.io.Text.readString(Text.java:402)
        at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1471)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1428)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1417)
        at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1412)
        at org.apache.hadoop.io.MapFile$Reader.open(MapFile.java:293)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$HbaseMapFile$HbaseReader.<init>(HStoreFile.java:632)
        at org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader.<init>(HStoreFile.java:714)
        at org.apache.hadoop.hbase.regionserver.HStoreFile.getReader(HStoreFile.java:413)
        at org.apache.hadoop.hbase.regionserver.HStore.<init>(HStore.java:262)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateHStore(HRegion.java:1729)
        at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:469)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.instantiateRegion(HRegionServer.java:1004)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.openRegion(HRegionServer.java:976)
        at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:901)
        at java.lang.Thread.run(Thread.java:619)
{code}

HBase is just opening a mapfile.

Here is from svn blame and history:

{code}
706798    hairong         if (sentChecksumOk) {
706798    hairong            // this should not happen; log the error for the debugging purpose
706798    hairong            LOG.info(StringUtils.stringifyException(new IOException(
708724    rangadi              "Checksum ok was sent and should not be sent again")));


r708724 | rangadi | 2008-10-28 16:33:40 -0700 (Tue, 28 Oct 2008) | 1 line

HADOOP-4499. DFSClient should invoke checksumOk only once. (Raghu Angadi)
------------------------------------------------------------------------
r706798 | hairong | 2008-10-21 15:19:07 -0700 (Tue, 21 Oct 2008) | 1 line

Merge -r 706795:706796 from trunk to main to move the change log of HADOOP-3914.
{code}

Code comment says this condition should never happen.

Looking at code, IIUC, we get this exception if we reread inside a block.

For now, I've marked it a blocker.  HBase can't use 0.19.0 if this is the carry-on.

I'll dig in some more.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message