001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements. See the NOTICE file distributed with this
004 * work for additional information regarding copyright ownership. The ASF
005 * licenses this file to you under the Apache License, Version 2.0 (the
006 * "License"); you may not use this file except in compliance with the License.
007 * You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
013 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
014 * License for the specific language governing permissions and limitations
015 * under the License.
016 */
017package org.apache.hadoop.hbase.io.encoding;
018
019import java.io.DataInputStream;
020import java.io.DataOutputStream;
021import java.io.IOException;
022import java.nio.ByteBuffer;
023
024import org.apache.hadoop.hbase.Cell;
025import org.apache.hadoop.hbase.CellComparator;
026import org.apache.hadoop.hbase.io.hfile.HFileContext;
027import org.apache.hadoop.hbase.nio.ByteBuff;
028import org.apache.yetus.audience.InterfaceAudience;
029
030/**
031 * Encoding of KeyValue. It aims to be fast and efficient using assumptions:
032 * <ul>
033 * <li>the KeyValues are stored sorted by key</li>
034 * <li>we know the structure of KeyValue</li>
035 * <li>the values are always iterated forward from beginning of block</li>
036 * <li>knowledge of Key Value format</li>
037 * </ul>
038 * It is designed to work fast enough to be feasible as in memory compression.
039 */
040@InterfaceAudience.Private
041public interface DataBlockEncoder {
042// TODO: This Interface should be deprecated and replaced. It presumes hfile and carnal knowledge of
043// Cell internals. It was done for a different time. Remove. Purge.
044  /**
045   * Starts encoding for a block of KeyValues. Call
046   * {@link #endBlockEncoding(HFileBlockEncodingContext, DataOutputStream, byte[])} to finish
047   * encoding of a block.
048   * @param encodingCtx
049   * @param out
050   * @throws IOException
051   */
052  void startBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out)
053      throws IOException;
054
055  /**
056   * Encodes a KeyValue.
057   * @param cell
058   * @param encodingCtx
059   * @param out
060   * @return unencoded kv size written
061   * @throws IOException
062   */
063  int encode(Cell cell, HFileBlockEncodingContext encodingCtx, DataOutputStream out)
064      throws IOException;
065
066  /**
067   * Ends encoding for a block of KeyValues. Gives a chance for the encoder to do the finishing
068   * stuff for the encoded block. It must be called at the end of block encoding.
069   * @param encodingCtx
070   * @param out
071   * @param uncompressedBytesWithHeader
072   * @throws IOException
073   */
074  void endBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out,
075      byte[] uncompressedBytesWithHeader) throws IOException;
076
077  /**
078   * Decode.
079   * @param source Compressed stream of KeyValues.
080   * @param decodingCtx
081   * @return Uncompressed block of KeyValues.
082   * @throws IOException If there is an error in source.
083   */
084  ByteBuffer decodeKeyValues(DataInputStream source, HFileBlockDecodingContext decodingCtx)
085      throws IOException;
086
087  /**
088   * Return first key in block as a cell. Useful for indexing. Typically does not make
089   * a deep copy but returns a buffer wrapping a segment of the actual block's
090   * byte array. This is because the first key in block is usually stored
091   * unencoded.
092   * @param block encoded block we want index, the position will not change
093   * @return First key in block as a cell.
094   */
095  Cell getFirstKeyCellInBlock(ByteBuff block);
096
097  /**
098   * Create a HFileBlock seeker which find KeyValues within a block.
099   * @param comparator what kind of comparison should be used
100   * @param decodingCtx
101   * @return A newly created seeker.
102   */
103  EncodedSeeker createSeeker(CellComparator comparator, HFileBlockDecodingContext decodingCtx);
104
105  /**
106   * Creates a encoder specific encoding context
107   *
108   * @param encoding
109   *          encoding strategy used
110   * @param headerBytes
111   *          header bytes to be written, put a dummy header here if the header
112   *          is unknown
113   * @param meta
114   *          HFile meta data
115   * @return a newly created encoding context
116   */
117  HFileBlockEncodingContext newDataBlockEncodingContext(
118      DataBlockEncoding encoding, byte[] headerBytes, HFileContext meta);
119
120  /**
121   * Creates an encoder specific decoding context, which will prepare the data
122   * before actual decoding
123   *
124   * @param meta
125   *          HFile meta data        
126   * @return a newly created decoding context
127   */
128  HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta);
129
130  /**
131   * An interface which enable to seek while underlying data is encoded.
132   *
133   * It works on one HFileBlock, but it is reusable. See
134   * {@link #setCurrentBuffer(ByteBuff)}.
135   */
136  interface EncodedSeeker {
137    /**
138     * Set on which buffer there will be done seeking.
139     * @param buffer Used for seeking.
140     */
141    void setCurrentBuffer(ByteBuff buffer);
142
143    /**
144     * From the current position creates a cell using the key part
145     * of the current buffer
146     * @return key at current position
147     */
148    Cell getKey();
149
150    /**
151     * Does a shallow copy of the value at the current position. A shallow
152     * copy is possible because the returned buffer refers to the backing array
153     * of the original encoded buffer.
154     * @return value at current position
155     */
156    ByteBuffer getValueShallowCopy();
157
158    /**
159     * @return the Cell at the current position. Includes memstore timestamp.
160     */
161    Cell getCell();
162
163    /** Set position to beginning of given block */
164    void rewind();
165
166    /**
167     * Move to next position
168     * @return true on success, false if there is no more positions.
169     */
170    boolean next();
171
172    /**
173     * Moves the seeker position within the current block to:
174     * <ul>
175     * <li>the last key that that is less than or equal to the given key if
176     * <code>seekBefore</code> is false</li>
177     * <li>the last key that is strictly less than the given key if <code>
178     * seekBefore</code> is true. The caller is responsible for loading the
179     * previous block if the requested key turns out to be the first key of the
180     * current block.</li>
181     * </ul>
182     * @param key - Cell to which the seek should happen
183     * @param seekBefore find the key strictly less than the given key in case
184     *          of an exact match. Does not matter in case of an inexact match.
185     * @return 0 on exact match, 1 on inexact match.
186     */
187    int seekToKeyInBlock(Cell key, boolean seekBefore);
188
189    /**
190     * Compare the given key against the current key
191     * @param comparator
192     * @param key
193     * @return -1 is the passed key is smaller than the current key, 0 if equal and 1 if greater
194     */
195    public int compareKey(CellComparator comparator, Cell key);
196  }
197}