You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Victor Xu (JIRA)" <ji...@apache.org> on 2016/01/09 15:15:39 UTC

[jira] [Created] (HBASE-15085) IllegalStateException was thrown when scanning on bulkloaded HFiles

Victor Xu created HBASE-15085:
---------------------------------

             Summary: IllegalStateException was thrown when scanning on bulkloaded HFiles
                 Key: HBASE-15085
                 URL: https://issues.apache.org/jira/browse/HBASE-15085
             Project: HBase
          Issue Type: Bug
    Affects Versions: 1.1.2, 0.98.12
         Environment: HBase-0.98.12 & Hadoop-2.6.0 & JDK1.7
HBase-1.1.2 & Hadoop-2.6.0 & JDK1.7
            Reporter: Victor Xu
            Assignee: Victor Xu


IllegalStateException was thrown when we scanned from an HFile which was bulk loaded several minutes ago, as shown below:
{code}
2015-12-16 22:20:54,456 ERROR com.taobao.kart.coprocessor.server.KartCoprocessor: icbu_ae_ws_product,/0055,1450275490479.6a6a700f465ad074287fed720c950f7c. batchNotify exception
java.lang.IllegalStateException: EncodedScanner works only on encoded data blocks
        at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$EncodedScannerV2.updateCurrentBlock(HFileReaderV2.java:1042)
        at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$EncodedScannerV2.seekTo(HFileReaderV2.java:1093)
        at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:244)
        at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:152)
        at org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:329)
        at org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:188)
        at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:1879)
        at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:4068)
        at org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:2029)
        at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2015)
        at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:1992)
{code}
I used 'hbase hfile' command to analyse the meta and block info of the hfile, finding that even through the DATA_BLOCK_ENCODING was 'DIFF' in FileInfo, the actual data blocks was written without any encoding algorithms(BlockType was 'DATA', not 'ENCODED_DATA'):
{code}
Fileinfo:
    BLOOM_FILTER_TYPE = ROW
    BULKLOAD_SOURCE_TASK = attempt_1442077249005_606706_r_000012_0
    BULKLOAD_TIMESTAMP = \x00\x00\x01R\x12$\x13\x12
    DATA_BLOCK_ENCODING = DIFF
...
DataBlock Header:
HFileBlock [ fileOffset=0 headerSize()=33 blockType=DATA onDiskSizeWithoutHeader=65591 uncompressedSizeWithoutHeader=65571 prevBlockOffset=-1 isUseHBaseChecksum()=true checksumType=CRC32 bytesPerChecksum=16384 onDiskDataSizeWithHeader=65604 getOnDiskSizeWithHeader()=65624 totalChecksumBytes()=20 isUnpacked()=true buf=[ java.nio.HeapByteBuffer[pos=0 lim=65624 cap=65657], array().length=65657, arrayOffset()=0 ] dataBeginsWith=\x00\x00\x003\x00\x00\x00\x0A\x00\x10/0008:1000000008\x01dprod fileContext=HFileContext [ usesHBaseChecksum=true checksumType=CRC32 bytesPerChecksum=16384 blocksize=65536 encoding=NONE includesMvcc=true includesTags=false compressAlgo=NONE compressTags=false cryptoContext=[ cipher=NONE keyHash=NONE ] ] ]
{code}
The data block encoding in file info was not consistent with the one in data block, which means there must be something wrong with the bulkload process.
After debugging on each step of bulkload, I found that LoadIncrementalHFiles had a bug when loading hfile into a splitted region. 
{code}
/**
   * Copy half of an HFile into a new HFile.
   */
  private static void copyHFileHalf(
      Configuration conf, Path inFile, Path outFile, Reference reference,
      HColumnDescriptor familyDescriptor)
  throws IOException {
    FileSystem fs = inFile.getFileSystem(conf);
    CacheConfig cacheConf = new CacheConfig(conf);
    HalfStoreFileReader halfReader = null;
    StoreFile.Writer halfWriter = null;
    try {
      halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, conf);
      Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();

      int blocksize = familyDescriptor.getBlocksize();
      Algorithm compression = familyDescriptor.getCompression();
      BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
// use CF's DATA_BLOCK_ENCODING to initialize HFile writer
      HFileContext hFileContext = new HFileContextBuilder()
                                  .withCompression(compression)
                                  .withChecksumType(HStore.getChecksumType(conf))
                                  .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
                                  .withBlockSize(blocksize)
                                  .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding())
                                  .build();
      halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
          fs)
              .withFilePath(outFile)
              .withBloomType(bloomFilterType)
              .withFileContext(hFileContext)
              .build();
      HFileScanner scanner = halfReader.getScanner(false, false, false);
      scanner.seekTo();
      do {
        KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
        halfWriter.append(kv);
      } while (scanner.next());

// force encoding setting with the original HFile's file info
      for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
        if (shouldCopyHFileMetaKey(entry.getKey())) {
          halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
        }
      }
    } finally {
      if (halfWriter != null) halfWriter.close();
      if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
    }
  }
{code}
As shown above, when an HFile which has a DIFF encoding is bulkloaded into a splitted region whose CF's DATA_BLOCK_ENCODING is NONE, the two new HFiles would have inconsistent encodings.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)