@Override public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx, DataOutputStream out) throws IOException { if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) { throw new IOException( this.getClass().getName() + " only accepts " + HFileBlockDefaultEncodingContext.class.getName() + " as the " + "encoding context."); } HFileBlockDefaultEncodingContext encodingCtx = (HFileBlockDefaultEncodingContext) blkEncodingCtx; encodingCtx.prepareEncoding(out); if (encodingCtx.getHFileContext().isIncludesTags() && encodingCtx.getHFileContext().isCompressTags()) { if (encodingCtx.getTagCompressionContext() != null) { // It will be overhead to create the TagCompressionContext again and again for every block // encoding. encodingCtx.getTagCompressionContext().clear(); } else { try { TagCompressionContext tagCompressionContext = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE); encodingCtx.setTagCompressionContext(tagCompressionContext); } catch (Exception e) { throw new IOException("Failed to initialize TagCompressionContext", e); } } } StreamUtils.writeInt(out, 0); // DUMMY length. This will be updated in endBlockEncoding() blkEncodingCtx.setEncodingState(new BufferedDataBlockEncodingState()); }
/** * @param cell * @param out * @param encodingCtx * @return unencoded size added * @throws IOException */ protected final int afterEncodingKeyValue( Cell cell, DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws IOException { int size = 0; if (encodingCtx.getHFileContext().isIncludesTags()) { int tagsLength = cell.getTagsLength(); ByteBufferUtils.putCompressedInt(out, tagsLength); // There are some tags to be written if (tagsLength > 0) { TagCompressionContext tagCompressionContext = encodingCtx.getTagCompressionContext(); // When tag compression is enabled, tagCompressionContext will have a not null value. Write // the tags using Dictionary compression in such a case if (tagCompressionContext != null) { tagCompressionContext.compressTags( out, cell.getTagsArray(), cell.getTagsOffset(), tagsLength); } else { out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength); } } size += tagsLength + KeyValue.TAGS_LENGTH_SIZE; } if (encodingCtx.getHFileContext().isIncludesMvcc()) { // Copy memstore timestamp from the byte buffer to the output stream. long memstoreTS = cell.getSequenceId(); WritableUtils.writeVLong(out, memstoreTS); // TODO use a writeVLong which returns the #bytes written so that 2 time parsing can be // avoided. size += WritableUtils.getVIntSize(memstoreTS); } return size; }