hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jingcheng Du (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HBASE-18381) HBase regionserver crashes when reading MOB file with column qualifier >64MB
Date Mon, 17 Jul 2017 04:26:01 GMT

    [ https://issues.apache.org/jira/browse/HBASE-18381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16089289#comment-16089289
] 

Jingcheng Du edited comment on HBASE-18381 at 7/17/17 4:25 AM:
---------------------------------------------------------------

Thanks [~tedyu@apache.org], this is not related with MOB, this is because of the default protobuf
size limit of 64MB, non-MOB cells have the same limit too.
Your changes in HFile are not necessary in the patch, the test can pass with only the changes
in the configurations which means the test should pass if the configuration is set properly
in branch-2. And I don't think this would be an issue in branch-2 and latter branches.
64MB is too large and we have filed a chapter to explain why we don't recommend it in https://hbase.apache.org/book.html#faq.
Besides, we have a test to address the large cell size in TestMobStoreScanner#testGetMassive.

Hi [~huaxiang],
[~djelinski] ran into this issue in CDH release (on HBase 1.2.0-cdh5.10.0), would you mind
trying in your env? Thanks a lot.


was (Author: jingcheng.du):
Thanks [~tedyu@apache.org], this is not related with MOB, this is because of the default protobuf
size limit of 64MB, non-MOB cells have the same limit too.
Your changes in HFile are not necessary in the patch, the test can pass with only the changes
in the configurations which means the test should pass if the configuration is set properly
in branch-2. And I don't think this would be an issue in branch-2 and latter branches.
64MB is too large and we have filed a chapter to explain why we don't recommend it in https://hbase.apache.org/book.html#faq.
Besides, we have a test to address the large cell size in TestMobStoreScanner#testGetMassive.

Hi [~huaxiang], [~djelinski] ran into this issue in CDH release (on HBase 1.2.0-cdh5.10.0),
would you mind trying in your env? Thanks a lot.

> HBase regionserver crashes when reading MOB file with column qualifier >64MB
> ----------------------------------------------------------------------------
>
>                 Key: HBASE-18381
>                 URL: https://issues.apache.org/jira/browse/HBASE-18381
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 2.0.0-alpha-1
>         Environment:  HBase 1.2.0-cdh5.10.0
>            Reporter: Daniel Jelinski
>            Assignee: Ted Yu
>            Priority: Critical
>             Fix For: 2.0.0-beta-1
>
>         Attachments: 18381.v1.txt, 18381.v2.txt
>
>
> After putting a cell with 64MB column qualifier to a MOB-enabled table, region server
crashes when flushing data. Subsequent WAL replay attempts also result in region server crashes.
> Gist of code used to create the table:
> private String table = "poisonPill";
> private byte[] familyBytes = Bytes.toBytes("cf");
> private void createTable(Connection conn) throws IOException {
>    Admin hbase_admin = conn.getAdmin();
>    HTableDescriptor htable = new HTableDescriptor(TableName.valueOf(table));
>    HColumnDescriptor hfamily = new HColumnDescriptor(familyBytes);
>    hfamily.setMobEnabled(true);
>    htable.setConfiguration("hfile.format.version","3");
>    htable.addFamily(hfamily);
>    hbase_admin.createTable(htable);
> }
> private void killTable(Connection conn) throws IOException {
>    Table tbl = conn.getTable(TableName.valueOf(table));
>    byte[] data = new byte[1<<26];
>    byte[] smalldata = new byte[0];
>    Put put = new Put(Bytes.toBytes("1"));
>    put.addColumn(familyBytes, data, smalldata);
>    tbl.put(put);
> }
> Region server logs (redacted):
> 2017-07-11 09:34:53,747 INFO org.apache.hadoop.hbase.regionserver.HRegion: Flushing 1/1
column families, memstore=64.00 MB; WAL is null, using passed sequenceid=7
> 2017-07-11 09:34:53,757 WARN org.apache.hadoop.hbase.io.hfile.HFileWriterV2: A minimum
HFile version of 3 is required to support cell attributes/tags. Consider setting hfile.format.version
accordingly.
> 2017-07-11 09:34:54,504 INFO org.apache.hadoop.hbase.mob.DefaultMobStoreFlusher: Flushed,
sequenceid=7, memsize=67109096, hasBloomFilter=true, into tmp file hdfs://sandbox/hbase/data/default/poisonPill/f82e20f32302dfdd95c89ecc3be5a211/.tmp/7858d223eddd4199ad220fc77bb612eb
> 2017-07-11 09:34:54,694 ERROR org.apache.hadoop.hbase.regionserver.HStore: Failed to
open store file : hdfs://sandbox/hbase/data/default/poisonPill/f82e20f32302dfdd95c89ecc3be5a211/.tmp/7858d223eddd4199ad220fc77bb612eb,
keeping it in tmp location
> org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading HFile Trailer
from file hdfs://sandbox/hbase/data/default/poisonPill/f82e20f32302dfdd95c89ecc3be5a211/.tmp/7858d223eddd4199ad220fc77bb612eb
> 	at org.apache.hadoop.hbase.io.hfile.HFile.pickReaderVersion(HFile.java:497)
> 	at org.apache.hadoop.hbase.io.hfile.HFile.createReader(HFile.java:525)
> 	at org.apache.hadoop.hbase.regionserver.StoreFile$Reader.<init>(StoreFile.java:1105)
> 	at org.apache.hadoop.hbase.regionserver.StoreFileInfo.open(StoreFileInfo.java:265)
> 	at org.apache.hadoop.hbase.regionserver.StoreFile.open(StoreFile.java:404)
> 	at org.apache.hadoop.hbase.regionserver.StoreFile.createReader(StoreFile.java:509)
> 	at org.apache.hadoop.hbase.regionserver.StoreFile.createReader(StoreFile.java:499)
> 	at org.apache.hadoop.hbase.regionserver.HStore.createStoreFileAndReader(HStore.java:675)
> 	at org.apache.hadoop.hbase.regionserver.HStore.createStoreFileAndReader(HStore.java:667)
> 	at org.apache.hadoop.hbase.regionserver.HStore.validateStoreFile(HStore.java:1746)
> 	at org.apache.hadoop.hbase.regionserver.HStore.flushCache(HStore.java:942)
> 	at org.apache.hadoop.hbase.regionserver.HStore$StoreFlusherImpl.flushCache(HStore.java:2299)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushCacheAndCommit(HRegion.java:2372)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2102)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEdits(HRegion.java:4139)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.replayRecoveredEditsIfAny(HRegion.java:3934)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:828)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:799)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:6480)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:6441)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:6412)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:6368)
> 	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:6319)
> 	at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:362)
> 	at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:129)
> 	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol message was too
large.  May be malicious.  Use CodedInputStream.setSizeLimit() to increase the size limit.
> 	at com.google.protobuf.InvalidProtocolBufferException.sizeLimitExceeded(InvalidProtocolBufferException.java:110)
> 	at com.google.protobuf.CodedInputStream.refillBuffer(CodedInputStream.java:755)
> 	at com.google.protobuf.CodedInputStream.isAtEnd(CodedInputStream.java:701)
> 	at com.google.protobuf.CodedInputStream.readTag(CodedInputStream.java:99)
> 	at org.apache.hadoop.hbase.protobuf.generated.HFileProtos$FileInfoProto.<init>(HFileProtos.java:82)
> 	at org.apache.hadoop.hbase.protobuf.generated.HFileProtos$FileInfoProto.<init>(HFileProtos.java:46)
> 	at org.apache.hadoop.hbase.protobuf.generated.HFileProtos$FileInfoProto$1.parsePartialFrom(HFileProtos.java:135)
> 	at org.apache.hadoop.hbase.protobuf.generated.HFileProtos$FileInfoProto$1.parsePartialFrom(HFileProtos.java:130)
> 	at com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:200)
> 	at com.google.protobuf.AbstractParser.parsePartialDelimitedFrom(AbstractParser.java:241)
> 	at com.google.protobuf.AbstractParser.parseDelimitedFrom(AbstractParser.java:253)
> 	at com.google.protobuf.AbstractParser.parseDelimitedFrom(AbstractParser.java:259)
> 	at com.google.protobuf.AbstractParser.parseDelimitedFrom(AbstractParser.java:49)
> 	at org.apache.hadoop.hbase.protobuf.generated.HFileProtos$FileInfoProto.parseDelimitedFrom(HFileProtos.java:297)
> 	at org.apache.hadoop.hbase.io.hfile.HFile$FileInfo.read(HFile.java:752)
> 	at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.<init>(HFileReaderV2.java:161)
> 	at org.apache.hadoop.hbase.io.hfile.HFileReaderV3.<init>(HFileReaderV3.java:77)
> 	at org.apache.hadoop.hbase.io.hfile.HFile.pickReaderVersion(HFile.java:487)
> 	... 28 more 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message