[
https://issues.apache.org/jira/browse/HBASE-11625?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14102583#comment-14102583
]
Paul Fleetwood commented on HBASE-11625:
----------------------------------------
I may have a reproduction of this issue. I've generated a bulk loadable HFile
(which I've attached) using the HFileOutputFormat, and am unable to perform
scans on it.
Here is what I do:
Running 0.98.5 on Mac in single instance mode...
- Use the hbase shell to create a table
- Use the bulkload tool to load the attached file into the table: ./bin/hbase
org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles
- Use the hbase shell to count the rows in the new table - see an exception
- Scan the table completely in the hbase shell
- Attempt to count the table again, see it succeed
Something about the scan "fixes" things.
The exception that I see when running the count is this:
RROR: org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException:
Expected nextCallSeq: 2 But the nextCallSeq got from client: 1;
request=scanner_id: 547 number_of_rows: 10 close_scanner: false next_call_seq: 1
at
org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3110)
at
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:29587)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2026)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:98)
at
org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:114)
at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:94)
at java.lang.Thread.run(Thread.java:695)
But, I've learned that this is a red herring. This issue is the result of a
client retry (causing the sequence numbers to mismatch). The retry itself is
caused by another failure, which looks like this (please note the following
callstacks were captured while running my own application, but I believe they
are the same in the count):
java.io.IOException: Could not iterate StoreFileScanner[HFileScanner for reader
reader=file:/var/folders/bn/qpypwv8s3r7g3ksgxdj0hlw80000gn/T/hbase-paulfleetwood/hbase/data/paul2_lxlv1_prod/events/d2697b9d34be632e481ab33433a28699/common/69823512bede4392be352761adc669e6_SeqId_26_,
compression=none, cacheConf=CacheConfig:enabled [cacheDataOnRead=true]
[cacheDataOnWrite=false] [cacheIndexesOnWrite=false] [cacheBloomsOnWrite=false]
[cacheEvictOnClose=false] [cacheCompressed=false][prefetchOnOpen=false],
firstKey=\x00Sa\xC8Dw\xE3\xE8i\x9C\xD2\xDB\x1C\xC3Mk\xF4\x99!1\xD1\xBF"\x99w/common:entity\x00id\x00\x03/1398917188791/Put,
lastKey=\x0FS\x88\xAD\xE1\xFA\x11:\x85\x88\xDE\x12\x12\xF0\xD8s\x9A\x06X\x1B\x84\x1A\x8B\xA7\xC1/common:txn\x00timeoutPolicy\x00\x03/1401466337205/Put,
avgKeyLen=58, avgValueLen=11, entries=345444, length=27516812,
cur=\x00Sc'M\x5C\xC8\x0A\xD5\xC0P\xA53U\x01,\xDF=\x8D<\x0F\xA6\x00\xCC\xCB/common:entity\x00type\x00\x03/1399007053829/Put/vlen=4/mvcc=0]
This is caused by something like the following:
java.io.IOException: Failed to read compressed block at 65621,
onDiskSizeWithoutHeader=65620, preReadHeaderSize=33, header.length=33, header
bytes:
\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00
Which, is caused by:
java.io.IOException: Invalid HFile block magic: FJ\xA8Yt\x04@$
That last exception is thrown with the following call stack:
BlockType.parse(byte[], int, int) line: 154
BlockType.read(ByteBuffer) line: 165
HFileBlock.<init>(ByteBuffer, boolean) line: 239
HFileBlock$FSReaderV2.readBlockDataInternal(FSDataInputStream, long, long, int,
boolean, boolean) line: 1446
HFileBlock$FSReaderV2.readBlockData(long, long, int, boolean) line: 1312
HFileReaderV2.readBlock(long, long, boolean, boolean, boolean, boolean,
BlockType) line: 387
HFileReaderV2$ScannerV2(HFileReaderV2$AbstractScannerV2).readNextDataBlock()
line: 635
HFileReaderV2$ScannerV2.next() line: 749
StoreFileScanner.next() line: 136
KeyValueHeap.next() line: 108
StoreScanner.next(List<Cell>, int) line: 537
KeyValueHeap.next(List<Cell>, int) line: 140
HRegion$RegionScannerImpl.populateResult(List<Cell>, KeyValueHeap, int, byte[],
int, short) line: 3937
HRegion$RegionScannerImpl.nextInternal(List<Cell>, int) line: 4017
HRegion$RegionScannerImpl.nextRaw(List<Cell>, int) line: 3885
HRegion$RegionScannerImpl.nextRaw(List<Cell>) line: 3876
HRegionServer.scan(RpcController, ClientProtos$ScanRequest) line: 3158
ClientProtos$ClientService$2.callBlockingMethod(Descriptors$MethodDescriptor,
RpcController, Message) line: 29587
RpcServer.call(BlockingService, MethodDescriptor, Message, CellScanner, long,
MonitoredRPCHandler) line: 2026
CallRunner.run() line: 98
BalancedQueueRpcExecutor(RpcExecutor).consumerLoop(BlockingQueue<CallRunner>)
line: 114
RpcExecutor$1.run() line: 94
Thread.run() line: 695
> Reading datablock throws "Invalid HFile block magic" and can not switch to
> hdfs checksum
> -----------------------------------------------------------------------------------------
>
> Key: HBASE-11625
> URL: https://issues.apache.org/jira/browse/HBASE-11625
> Project: HBase
> Issue Type: Bug
> Components: HFile
> Affects Versions: 0.94.21, 0.98.4
> Reporter: qian wang
>
> when using hbase checksum,call readBlockDataInternal() in hfileblock.java, it
> could happen file corruption but it only can switch to hdfs checksum
> inputstream till validateBlockChecksum(). If the datablock's header corrupted
> when b = new HFileBlock(),it throws the exception "Invalid HFile block magic"
> and the rpc call fail
--
This message was sent by Atlassian JIRA
(v6.2#6252)