You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "Rushabh S Shah (JIRA)" <ji...@apache.org> on 2018/12/14 19:38:00 UTC

[jira] [Created] (HBASE-21607) HBase Region server read fails in case of datanode disk error.

Rushabh S Shah created HBASE-21607:
--------------------------------------

             Summary: HBase Region server read fails in case of datanode disk error.
                 Key: HBASE-21607
                 URL: https://issues.apache.org/jira/browse/HBASE-21607
             Project: HBase
          Issue Type: Bug
          Components: regionserver
    Affects Versions: 1.3.2
            Reporter: Rushabh S Shah


Hbase region server reads failed with following error.
{noformat}
2018-11-30 16:49:18,760 WARN [,queue=12,port=60020] hdfs.BlockReaderFactory - BlockReaderFactory(fileName=<fileName>, block=BP-1618467445-<ip>-1516873463430:blk_1090719164_16982933): error creating ShortCircuitReplica.
java.io.IOException: invalid metadata header version 0. Can only handle version 1.
at org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica.<init>(ShortCircuitReplica.java:129)
at org.apache.hadoop.hdfs.BlockReaderFactory.requestFileDescriptors(BlockReaderFactory.java:558)
at org.apache.hadoop.hdfs.BlockReaderFactory.createShortCircuitReplicaInfo(BlockReaderFactory.java:490)
at org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.create(ShortCircuitCache.java:782)
at org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.fetchOrCreate(ShortCircuitCache.java:716)
at org.apache.hadoop.hdfs.BlockReaderFactory.getBlockReaderLocal(BlockReaderFactory.java:422)
at org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:333)
at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1145)
at org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1087)
at org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1444)
at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1407)
at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:89)
at org.apache.hadoop.hbase.io.hfile.HFileBlock.positionalReadWithExtra(HFileBlock.java:834)
at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1530)
at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1781)
at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1624)
at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:455)
at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$EncodedScannerV2.seekTo(HFileReaderV2.java:1263)
at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:297)
at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:189)
at org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:372)
at org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:220)
at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:2164)
at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.initializeScanners(HRegion.java:5916)
at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:5890)
at org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:2739)
at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2719)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7197)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7156)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7149)
at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2250)
at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:35068)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2373)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124)
at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:188)
at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:168)

 

2018-11-30 16:49:18,760 WARN [,queue=12,port=60020] shortcircuit.ShortCircuitCache - ShortCircuitCache(0x16fd768f): failed to load 1090719164_BP-1618467445-<ip>-1516873463430
2018-11-30 16:49:18,761 DEBUG [,queue=12,port=60020] ipc.RpcServer - RpcServer.FifoWFPBQ.default.handler=246,queue=12,port=60020: callId: 46940 service: ClientService methodName: Get size: 443 connection: <ip>:48798 deadline: 1543596678759
2018-11-30 16:49:18,761 DEBUG [,queue=12,port=60020] ipc.RpcServer - RpcServer.FifoWFPBQ.default.handler=246,queue=12,port=60020: callId: 46940 service: ClientService methodName: Get size: 443 connection: <ip>:48798 deadline: 1543596678759
java.io.IOException: Could not seek StoreFileScanner[HFileScanner for reader reader=hdfs://<fileName>, compression=none, cacheConf=bl
ockCache=LruBlockCache{blockCount=696858, currentSize=7625195376, freeSize=5689203856, maxSize=13314399232, heapSize=7625195376, minSize=12648679424, minFactor=0.95, multiSize=6324339712, multiFactor=0.5, singleSize=3162169856, singleFactor=0.25}, cacheDa
taOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false, firstKey=<keyName>
at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:207)
at org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:372)
at org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:220)
at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:2164)
at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.initializeScanners(HRegion.java:5916)
at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:5890)
at org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:2739)
at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2719)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7197)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7156)
at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7149)
at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2250)
at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:35068)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2373)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124)
at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:188)
at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:168)
Caused by: java.io.IOException: Invalid HFile block magic: \x00\x00\x00\x00\x00\x00\x00\x00
at org.apache.hadoop.hbase.io.hfile.BlockType.parse(BlockType.java:155)
at org.apache.hadoop.hbase.io.hfile.BlockType.read(BlockType.java:167)
at org.apache.hadoop.hbase.io.hfile.HFileBlock.<init>(HFileBlock.java:345)
at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1821)
at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1624)
at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:455)
at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$EncodedScannerV2.seekTo(HFileReaderV2.java:1263)
at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:297)
at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:189)
... 16 more


{noformat}
TheĀ "Invalid HFile block magic" exception was thrown all the way back to the client.

The first stack trace indicates that hdfs short circuit read failed and then it fell back to hdfs local rpc read and that succeeded.

The underlying reason is there was a write error in one of the disks. The datanode was receiving and mirroring(to downstream nodes in the pipeline) just fine but was writing garbage data(all \x00\x00\x00\x00\x00\x00\x00\x00) to disk. Datanode hasn't detected this disk as bad yet.

We have hbase checksum turned on and hdfs checksum evaluation turned off. While reading this erroneous block, datanode didn't do any checksum validation and streamed all the data as is to region server and relied on hbase checksum validation to catch it. But we have a special case in hbase checksum validation that if the checksumType byte is 0, then it is NULL and considers that checksum validation is not needed for this block. Refer to [ChecksumUtil|https://github.com/apache/hbase/blob/branch-1.3/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java#L95] and [ChecksumType|https://github.com/apache/hbase/blob/branch-1.3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ChecksumType.java#L33].
 Thanks [~lhofhansl] and [~karanmehta93] for help debugging this.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)