Daniel Jelinski created HBASE-18381:
---------------------------------------

             Summary: HBase regionserver crash when flushing MOB 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: 1.2.0
         Environment:  HBase 1.2.0-cdh5.10.0
            Reporter: Daniel Jelinski
            Priority: Critical


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)

Reply via email to