[
https://issues.apache.org/jira/browse/HBASE-12270?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14272049#comment-14272049
]
Andrew Purtell commented on HBASE-12270:
----------------------------------------
I applied the patch and looped the test 10 times locally. On the 4th execution
the test failed:
{noformat}
-------------------------------------------------------
T E S T S
-------------------------------------------------------
Running org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite
Tests run: 216, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 245.764 sec
<<< FAILURE! - in org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite
testStoreFileCacheOnWrite[105](org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite)
Time elapsed: 1.297 sec <<< FAILURE!
java.lang.AssertionError: shouldBeCached: true
isCached: false
Test description: [cacheOnWrite=INDEX_BLOCKS, compress=GZ,
encoderType=NO_BLOCK_ENCODING, cacheCompressedData=true, blockCache=BucketCache]
block: HFileBlock [ fileOffset=24571 headerSize()=33 blockType=LEAF_INDEX
onDiskSizeWithoutHeader=172 uncompressedSizeWithoutHeader=538
prevBlockOffset=12147 isUseHBaseChecksum()=true checksumType=NULL
bytesPerChecksum=512 onDiskDataSizeWithHeader=201 getOnDiskSizeWithHeader()=205
totalChecksumBytes()=4 isUnpacked()=true buf=[ java.nio.HeapByteBuffer[pos=0
lim=575 cap=608], array().length=608, arrayOffset()=0 ]
dataBeginsWith=\x00\x00\x00\x09\x00\x00\x00\x00\x00\x00\x004\x00\x00\x00l\x00\x00\x00\xA0\x00\x00\x00\xD8\x00\x00\x01\x10\x00\x00\x01G
fileContext=HFileContext [ usesHBaseChecksum=true checksumType=NULL
bytesPerChecksum=512 blocksize=65536 encoding=NONE includesMvcc=true
includesTags=true compressAlgo=NONE compressTags=false cryptoContext=[
cipher=NONE keyHash=NONE ] ] ]
encodingInCache: NONE
blockCacheKey: dc6d2a06579e4da4b49ac8575ea3e90a_24571
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.assertTrue(Assert.java:41)
at
org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite.readStoreFile(TestCacheOnWrite.java:283)
at
org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite.testStoreFileCacheOnWriteInternals(TestCacheOnWrite.java:242)
at
org.apache.hadoop.hbase.io.hfile.TestCacheOnWrite.testStoreFileCacheOnWrite(TestCacheOnWrite.java:237)
Results :
Failed tests:
TestCacheOnWrite.testStoreFileCacheOnWrite:237->testStoreFileCacheOnWriteInternals:242->readStoreFile:283
shouldBeCached: true
isCached: false
Test description: [cacheOnWrite=INDEX_BLOCKS, compress=GZ,
encoderType=NO_BLOCK_ENCODING, cacheCompressedData=true, blockCache=BucketCache]
block: HFileBlock [ fileOffset=24571 headerSize()=33 blockType=LEAF_INDEX
onDiskSizeWithoutHeader=172 uncompressedSizeWithoutHeader=538
prevBlockOffset=12147 isUseHBaseChecksum()=true checksumType=NULL
bytesPerChecksum=512 onDiskDataSizeWithHeader=201 getOnDiskSizeWithHeader()=205
totalChecksumBytes()=4 isUnpacked()=true buf=[ java.nio.HeapByteBuffer[pos=0
lim=575 cap=608], array().length=608, arrayOffset()=0 ]
dataBeginsWith=\x00\x00\x00\x09\x00\x00\x00\x00\x00\x00\x004\x00\x00\x00l\x00\x00\x00\xA0\x00\x00\x00\xD8\x00\x00\x01\x10\x00\x00\x01G
fileContext=HFileContext [ usesHBaseChecksum=true checksumType=NULL
bytesPerChecksum=512 blocksize=65536 encoding=NONE includesMvcc=true
includesTags=true compressAlgo=NONE compressTags=false cryptoContext=[
cipher=NONE keyHash=NONE ] ] ]
encodingInCache: NONE
blockCacheKey: dc6d2a06579e4da4b49ac8575ea3e90a_24571
Tests run: 216, Failures: 1, Errors: 0, Skipped: 0
{noformat}
> A bug in the bucket cache, with cache blocks on write enabled
> -------------------------------------------------------------
>
> Key: HBASE-12270
> URL: https://issues.apache.org/jira/browse/HBASE-12270
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.94.11, 0.98.6.1
> Environment: I can reproduce it on a simple 2 node cluster, one
> running the master and another running a RS. I was testing on ec2.
> I used the following configurations for the cluster.
> hbase-env:HBASE_REGIONSERVER_OPTS=-Xmx2G -XX:MaxDirectMemorySize=5G
> -XX:CMSInitiatingOccupancyFraction=88 -XX:+AggressiveOpts -verbose:gc
> -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -Xlog
> gc:/tmp/hbase-regionserver-gc.log
> hbase-site:
> hbase.bucketcache.ioengine=offheap
> hbase.bucketcache.size=4196
> hbase.rs.cacheblocksonwrite=true
> hfile.block.index.cacheonwrite=true
> hfile.block.bloom.cacheonwrite=true
> Reporter: Khaled Elmeleegy
> Assignee: Liu Shaohui
> Priority: Critical
> Fix For: 1.0.0, 2.0.0, 0.98.10, 1.1.0
>
> Attachments: HBASE-12270-0.98-v1.diff, HBASE-12270-v1.diff,
> HBASE-12270-v2.diff, HBASE-12270-v2.patch, TestHBase.java, TestKey.java
>
>
> In my experiments, I have writers streaming their output to HBase. The reader
> powers a web page and does this scatter/gather, where it reads 1000 keys
> written last and passes them the the front end. With this workload, I get the
> exception below at the region server. Again, I am using HBAse (0.98.6.1). Any
> help is appreciated.
> 2014-10-10 15:06:44,173 ERROR
> [B.DefaultRpcServer.handler=62,queue=2,port=60020] ipc.RpcServer: Unexpected
> throwable object
> java.lang.IllegalArgumentException
> at java.nio.Buffer.position(Buffer.java:236)
> at
> org.apache.hadoop.hbase.util.ByteBufferUtils.skip(ByteBufferUtils.java:434)
> at
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.readKeyValueLen(HFileReaderV2.java:849)
> at
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.next(HFileReaderV2.java:760)
> at
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:248)
> at
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:152)
> at
> org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:317)
> at
> org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:176)
> at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:1780)
> at
> org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:3758)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:1950)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1936)
> at
> org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1913)
> at
> org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3157)
> at
> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:29587)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2027)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:108)
> 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:744)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)