[ 
https://issues.apache.org/jira/browse/HBASE-20403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Todd Lipcon reassigned HBASE-20403:
-----------------------------------

    Assignee: Todd Lipcon  (was: Umesh Agashe)

Attached patch has a test which reproduces a failure even on non-encrypted 
systems. Changing the prefetch to use pread instead of streaming read seems to 
fix the issue. We should verify that it also fixes it on encrypted filesystems.

> Prefetch sometimes doesn't work with encrypted file system
> ----------------------------------------------------------
>
>                 Key: HBASE-20403
>                 URL: https://issues.apache.org/jira/browse/HBASE-20403
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 2.0.0-beta-2
>            Reporter: Umesh Agashe
>            Assignee: Todd Lipcon
>            Priority: Major
>             Fix For: 3.0.0
>
>         Attachments: hbase-20403.patch
>
>
> Log from long running test has following stack trace a few times:
> {code}
> 2018-04-09 18:33:21,523 WARN 
> org.apache.hadoop.hbase.io.hfile.HFileReaderImpl: Prefetch 
> path=hdfs://ns1/hbase/data/default/IntegrationTestBigLinkedList_20180409172704/35f1a7ef13b9d327665228abdbcdffae/meta/9089d98b2a6b4847b3fcf6aceb124988,
>  offset=36884200, end=231005989
> java.lang.IllegalArgumentException
>   at java.nio.Buffer.limit(Buffer.java:275)
>   at 
> org.apache.hadoop.hdfs.ByteBufferStrategy.readFromBlock(ReaderStrategy.java:183)
>   at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:705)
>   at 
> org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:766)
>   at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:831)
>   at 
> org.apache.hadoop.crypto.CryptoInputStream.read(CryptoInputStream.java:197)
>   at java.io.DataInputStream.read(DataInputStream.java:149)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock.readWithExtra(HFileBlock.java:762)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readAtOffset(HFileBlock.java:1559)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1771)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1594)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock(HFileReaderImpl.java:1488)
>   at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$1.run(HFileReaderImpl.java:278)
>   at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
> {code}
> Size on disk calculations seem to get messed up due to encryption. Possible 
> fixes can be:
> * if file is encrypted with FileStatus#isEncrypted() and do not prefetch.
> * document that hbase.rs.prefetchblocksonopen cannot be true if file is 
> encrypted.



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

Reply via email to