HBASE-13451 - Make the HFileBlockIndex blockKeys to Cells so that it could

be easy to use in the CellComparators (Ram)
This commit is contained in:
ramkrishna 2015-06-09 12:03:01 +05:30
parent c62b396f9f
commit 487e4aa74f
23 changed files with 451 additions and 325 deletions

View File

@ -770,8 +770,10 @@ public final class CellUtil {
sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
sb.append('/');
sb.append(Type.codeToType(cell.getTypeByte()));
sb.append("/vlen=");
sb.append(cell.getValueLength());
if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
sb.append("/vlen=");
sb.append(cell.getValueLength());
}
sb.append("/seqid=");
sb.append(cell.getSequenceId());
return sb.toString();

View File

@ -2074,24 +2074,6 @@ public class Bytes implements Comparable<Bytes> {
return result;
}
/**
* Binary search for keys in indexes using Bytes.BYTES_RAWCOMPARATOR
*
* @param arr array of byte arrays to search for
* @param key the key you want to find
* @param offset the offset in the key you want to find
* @param length the length of the key
* @return zero-based index of the key, if the key is present in the array.
* Otherwise, a value -(i + 1) such that the key is between arr[i -
* 1] and arr[i] non-inclusively, where i is in [0, i], if we define
* arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above
* means that this function can return 2N + 1 different values
* ranging from -(N + 1) to N - 1.
*/
public static int binarySearch(byte[][] arr, byte[] key, int offset, int length) {
return binarySearch(arr, key, offset, length, (CellComparator) null);
}
/**
* Binary search for keys in indexes.
*
@ -2111,17 +2093,16 @@ public class Bytes implements Comparable<Bytes> {
@Deprecated
public static int binarySearch(byte [][]arr, byte []key, int offset,
int length, RawComparator<?> comparator) {
return binarySearch(arr, key, offset, length, (CellComparator)null);
return binarySearch(arr, key, offset, length);
}
/**
* Binary search for keys in indexes.
* Binary search for keys in indexes using Bytes.BYTES_RAWCOMPARATOR.
*
* @param arr array of byte arrays to search for
* @param key the key you want to find
* @param offset the offset in the key you want to find
* @param length the length of the key
* @param comparator a comparator to compare.
* @return zero-based index of the key, if the key is present in the array.
* Otherwise, a value -(i + 1) such that the key is between arr[i -
* 1] and arr[i] non-inclusively, where i is in [0, i], if we define
@ -2129,23 +2110,18 @@ public class Bytes implements Comparable<Bytes> {
* means that this function can return 2N + 1 different values
* ranging from -(N + 1) to N - 1.
*/
public static int binarySearch(byte [][]arr, byte []key, int offset,
int length, CellComparator comparator) {
public static int binarySearch(byte[][] arr, byte[] key, int offset, int length) {
int low = 0;
int high = arr.length - 1;
KeyValue.KeyOnlyKeyValue r = new KeyValue.KeyOnlyKeyValue();
r.setKey(key, offset, length);
while (low <= high) {
int mid = (low+high) >>> 1;
int mid = (low + high) >>> 1;
// we have to compare in this order, because the comparator order
// has special logic when the 'left side' is a special key.
int cmp = 0;
if (comparator != null) {
cmp = comparator.compare(r, arr[mid], 0, arr[mid].length);
} else {
cmp = Bytes.BYTES_RAWCOMPARATOR.compare(key, offset, length, arr[mid], 0, arr[mid].length);
}
int cmp = Bytes.BYTES_RAWCOMPARATOR
.compare(key, offset, length, arr[mid], 0, arr[mid].length);
// key lives above the midpoint
if (cmp > 0)
low = mid + 1;
@ -2156,7 +2132,7 @@ public class Bytes implements Comparable<Bytes> {
else
return mid;
}
return - (low+1);
return -(low + 1);
}
/**
@ -2172,7 +2148,7 @@ public class Bytes implements Comparable<Bytes> {
* means that this function can return 2N + 1 different values
* ranging from -(N + 1) to N - 1.
* @return the index of the block
* @deprecated Use {@link Bytes#binarySearch(byte[][], Cell, Comparator)}
* @deprecated Use {@link Bytes#binarySearch(Cell[], Cell, CellComparator)}
*/
@Deprecated
public static int binarySearch(byte[][] arr, Cell key, RawComparator<Cell> comparator) {
@ -2212,16 +2188,14 @@ public class Bytes implements Comparable<Bytes> {
* ranging from -(N + 1) to N - 1.
* @return the index of the block
*/
public static int binarySearch(byte[][] arr, Cell key, Comparator<Cell> comparator) {
public static int binarySearch(Cell[] arr, Cell key, CellComparator comparator) {
int low = 0;
int high = arr.length - 1;
KeyValue.KeyOnlyKeyValue r = new KeyValue.KeyOnlyKeyValue();
while (low <= high) {
int mid = (low+high) >>> 1;
// we have to compare in this order, because the comparator order
// has special logic when the 'left side' is a special key.
r.setKey(arr[mid], 0, arr[mid].length);
int cmp = comparator.compare(key, r);
int cmp = comparator.compare(key, arr[mid]);
// key lives above the midpoint
if (cmp > 0)
low = mid + 1;

View File

@ -59,7 +59,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
protected final Cell splitCell;
private byte[] firstKey = null;
private Cell firstKey = null;
private boolean firstKeySeeked = false;
@ -262,7 +262,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
@Override
public boolean seekBefore(Cell key) throws IOException {
if (top) {
Cell fk = new KeyValue.KeyOnlyKeyValue(getFirstKey(), 0, getFirstKey().length);
Cell fk = getFirstKey();
if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) {
return false;
}
@ -319,18 +319,18 @@ public class HalfStoreFileReader extends StoreFile.Reader {
}
@Override
public byte[] midkey() throws IOException {
public Cell midkey() throws IOException {
// Returns null to indicate file is not splitable.
return null;
}
@Override
public byte[] getFirstKey() {
public Cell getFirstKey() {
if (!firstKeySeeked) {
HFileScanner scanner = getScanner(true, true, false);
try {
if (scanner.seekTo()) {
this.firstKey = Bytes.toBytes(scanner.getKey());
this.firstKey = new KeyValue.KeyOnlyKeyValue(Bytes.toBytes(scanner.getKey()));
}
firstKeySeeked = true;
} catch (IOException e) {

View File

@ -17,7 +17,7 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
package org.apache.hadoop.hbase.io.hfile;
import java.io.DataInput;
import java.io.IOException;
@ -26,15 +26,15 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Hash;
/**
* A Bloom filter implementation built on top of {@link BloomFilterChunk},
* encapsulating a set of fixed-size Bloom filters written out at the time of
* A Bloom filter implementation built on top of
* {@link org.apache.hadoop.hbase.util.BloomFilterChunk}, encapsulating
* a set of fixed-size Bloom filters written out at the time of
* {@link org.apache.hadoop.hbase.io.hfile.HFile} generation into the data
* block stream, and loaded on demand at query time. This class only provides
* reading capabilities.
@ -84,7 +84,11 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
throw new IllegalArgumentException("Invalid hash type: " + hashType);
}
// We will pass null for ROW block
index = new HFileBlockIndex.BlockIndexReader(comparator, 1);
if(comparator == null) {
index = new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
} else {
index = new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator, 1);
}
index.readRootIndex(meta, numChunks);
}

View File

@ -17,9 +17,10 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.BloomFilterBase;
import org.apache.hadoop.hbase.CellComparator;
@ -47,7 +48,7 @@ public class CompoundBloomFilterBase implements BloomFilterBase {
protected long totalByteSize;
protected long totalMaxKeys;
/** Hash function type to use, as defined in {@link Hash} */
/** Hash function type to use, as defined in {@link org.apache.hadoop.hbase.util.Hash} */
protected int hashType;
/** Comparator used to compare Bloom filter keys */
protected CellComparator comparator;

View File

@ -16,7 +16,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
package org.apache.hadoop.hbase.io.hfile;
import java.io.DataInput;
import java.io.DataOutput;
@ -29,9 +29,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
import org.apache.hadoop.hbase.io.hfile.InlineBlockWriter;
import org.apache.hadoop.hbase.util.BloomFilterChunk;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**

View File

@ -394,13 +394,13 @@ public class HFile {
byte[] getLastKey();
byte[] midkey() throws IOException;
Cell midkey() throws IOException;
long length();
long getEntries();
byte[] getFirstKey();
Cell getFirstKey();
long indexSize();

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
@ -51,10 +52,11 @@ import org.apache.hadoop.util.StringUtils;
/**
* Provides functionality to write ({@link BlockIndexWriter}) and read
* ({@link BlockIndexReader}) single-level and multi-level block indexes.
* ({@link org.apache.hadoop.hbase.io.hfile.BlockIndexReader})
* single-level and multi-level block indexes.
*
* Examples of how to use the block index writer can be found in
* {@link org.apache.hadoop.hbase.util.CompoundBloomFilterWriter} and
* {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter} and
* {@link HFileWriterImpl}. Examples of how to use the reader can be
* found in {@link HFileWriterImpl} and TestHFileBlockIndex.
*/
@ -96,124 +98,179 @@ public class HFileBlockIndex {
2 * Bytes.SIZEOF_INT;
/**
* The reader will always hold the root level index in the memory. Index
* blocks at all other levels will be cached in the LRU cache in practice,
* although this API does not enforce that.
*
* All non-root (leaf and intermediate) index blocks contain what we call a
* "secondary index": an array of offsets to the entries within the block.
* This allows us to do binary search for the entry corresponding to the
* given key without having to deserialize the block.
* An implementation of the BlockIndexReader that deals with block keys which are plain
* byte[] like MetaBlock or the Bloom Block for ROW bloom.
* Does not need a comparator. It can work on Bytes.BYTES_RAWCOMPARATOR
*/
public static class BlockIndexReader implements HeapSize {
/** Needed doing lookup on blocks. */
private final CellComparator comparator;
static class ByteArrayKeyBlockIndexReader extends BlockIndexReader {
// Root-level data.
// TODO : Convert these to Cells (ie) KeyValue.KeyOnlyKV
private byte[][] blockKeys;
private long[] blockOffsets;
private int[] blockDataSizes;
private int rootCount = 0;
// Mid-key metadata.
private long midLeafBlockOffset = -1;
private int midLeafBlockOnDiskSize = -1;
private int midKeyEntry = -1;
/** Pre-computed mid-key */
private AtomicReference<byte[]> midKey = new AtomicReference<byte[]>();
/**
* The number of levels in the block index tree. One if there is only root
* level, two for root and leaf levels, etc.
*/
private int searchTreeLevel;
/** A way to read {@link HFile} blocks at a given offset */
private CachingBlockReader cachingBlockReader;
public BlockIndexReader(final CellComparator c, final int treeLevel,
public ByteArrayKeyBlockIndexReader(final int treeLevel,
final CachingBlockReader cachingBlockReader) {
this(c, treeLevel);
this(treeLevel);
this.cachingBlockReader = cachingBlockReader;
}
public BlockIndexReader(final CellComparator c, final int treeLevel)
{
public ByteArrayKeyBlockIndexReader(final int treeLevel) {
// Can be null for METAINDEX block
comparator = c;
searchTreeLevel = treeLevel;
}
/**
* @return true if the block index is empty.
*/
protected long calculateHeapSizeForBlockKeys(long heapSize) {
// Calculating the size of blockKeys
if (blockKeys != null) {
heapSize += ClassSize.REFERENCE;
// Adding array + references overhead
heapSize += ClassSize.align(ClassSize.ARRAY + blockKeys.length * ClassSize.REFERENCE);
// Adding bytes
for (byte[] key : blockKeys) {
heapSize += ClassSize.align(ClassSize.ARRAY + key.length);
}
}
return heapSize;
}
@Override
public boolean isEmpty() {
return blockKeys.length == 0;
}
/**
* Verifies that the block index is non-empty and throws an
* {@link IllegalStateException} otherwise.
* @param i
* from 0 to {@link #getRootBlockCount() - 1}
*/
public void ensureNonEmpty() {
if (blockKeys.length == 0) {
throw new IllegalStateException("Block index is empty or not loaded");
}
public byte[] getRootBlockKey(int i) {
return blockKeys[i];
}
/**
* Return the data block which contains this key. This function will only
* be called when the HFile version is larger than 1.
*
* @param key the key we are looking for
* @param currentBlock the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding
* @return reader a basic way to load blocks
* @throws IOException
*/
public HFileBlock seekToDataBlock(final Cell key, HFileBlock currentBlock, boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, currentBlock,
cacheBlocks,
pread, isCompaction, expectedDataBlockEncoding);
if (blockWithScanInfo == null) {
return null;
} else {
return blockWithScanInfo.getHFileBlock();
}
}
/**
* Return the BlockWithScanInfo which contains the DataBlock with other scan
* info such as nextIndexedKey. This function will only be called when the
* HFile version is larger than 1.
*
* @param key
* the key we are looking for
* @param currentBlock
* the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding.
* @return the BlockWithScanInfo which contains the DataBlock with other
* scan info such as nextIndexedKey.
* @throws IOException
*/
@Override
public BlockWithScanInfo loadDataBlockWithScanInfo(Cell key, HFileBlock currentBlock,
boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
boolean cacheBlocks, boolean pread, boolean isCompaction,
DataBlockEncoding expectedDataBlockEncoding) throws IOException {
// this would not be needed
return null;
}
@Override
public Cell midkey() throws IOException {
// Not needed here
return null;
}
protected void initialize(int numEntries) {
blockKeys = new byte[numEntries][];
}
protected void add(final byte[] key, final long offset, final int dataSize) {
blockOffsets[rootCount] = offset;
blockKeys[rootCount] = key;
blockDataSizes[rootCount] = dataSize;
rootCount++;
}
@Override
public int rootBlockContainingKey(byte[] key, int offset, int length, CellComparator comp) {
int pos = Bytes.binarySearch(blockKeys, key, offset, length);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
if (pos >= 0) {
// This means this is an exact match with an element of blockKeys.
assert pos < blockKeys.length;
return pos;
}
// Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
// and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
// blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
// key < blockKeys[0], meaning the file does not contain the given key.
int i = -pos - 1;
assert 0 <= i && i <= blockKeys.length;
return i - 1;
}
@Override
public int rootBlockContainingKey(Cell key) {
// Should not be called on this because here it deals only with byte[]
throw new UnsupportedOperationException(
"Cannot search for a key that is of Cell type. Only plain byte array keys " +
"can be searched for");
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("size=" + rootCount).append("\n");
for (int i = 0; i < rootCount; i++) {
sb.append("key=").append(KeyValue.keyToString(blockKeys[i]))
.append("\n offset=").append(blockOffsets[i])
.append(", dataSize=" + blockDataSizes[i]).append("\n");
}
return sb.toString();
}
}
/**
* An implementation of the BlockIndexReader that deals with block keys which are the key
* part of a cell like the Data block index or the ROW_COL bloom blocks
* This needs a comparator to work with the Cells
*/
static class CellBasedKeyBlockIndexReader extends BlockIndexReader {
private Cell[] blockKeys;
/** Pre-computed mid-key */
private AtomicReference<Cell> midKey = new AtomicReference<Cell>();
/** Needed doing lookup on blocks. */
private CellComparator comparator;
public CellBasedKeyBlockIndexReader(final CellComparator c, final int treeLevel,
final CachingBlockReader cachingBlockReader) {
this(c, treeLevel);
this.cachingBlockReader = cachingBlockReader;
}
public CellBasedKeyBlockIndexReader(final CellComparator c, final int treeLevel) {
// Can be null for METAINDEX block
comparator = c;
searchTreeLevel = treeLevel;
}
protected long calculateHeapSizeForBlockKeys(long heapSize) {
if (blockKeys != null) {
heapSize += ClassSize.REFERENCE;
// Adding array + references overhead
heapSize += ClassSize.align(ClassSize.ARRAY + blockKeys.length * ClassSize.REFERENCE);
// Adding blockKeys
for (Cell key : blockKeys) {
heapSize += ClassSize.align(CellUtil.estimatedHeapSizeOf(key));
}
}
// Add comparator and the midkey atomicreference
heapSize += 2 * ClassSize.REFERENCE;
return heapSize;
}
@Override
public boolean isEmpty() {
return blockKeys.length == 0;
}
/**
* @param i
* from 0 to {@link #getRootBlockCount() - 1}
*/
public Cell getRootBlockKey(int i) {
return blockKeys[i];
}
@Override
public BlockWithScanInfo loadDataBlockWithScanInfo(Cell key, HFileBlock currentBlock,
boolean cacheBlocks, boolean pread, boolean isCompaction,
DataBlockEncoding expectedDataBlockEncoding) throws IOException {
int rootLevelIndex = rootBlockContainingKey(key);
if (rootLevelIndex < 0 || rootLevelIndex >= blockOffsets.length) {
return null;
@ -227,7 +284,7 @@ public class HFileBlockIndex {
int currentOnDiskSize = blockDataSizes[rootLevelIndex];
if (rootLevelIndex < blockKeys.length - 1) {
nextIndexedKey = new KeyValue.KeyOnlyKeyValue(blockKeys[rootLevelIndex + 1]);
nextIndexedKey = blockKeys[rootLevelIndex + 1];
} else {
nextIndexedKey = HConstants.NO_NEXT_INDEXED_KEY;
}
@ -314,18 +371,12 @@ public class HFileBlockIndex {
return blockWithScanInfo;
}
/**
* An approximation to the {@link HFile}'s mid-key. Operates on block
* boundaries, and does not go inside blocks. In other words, returns the
* first key of the middle block of the file.
*
* @return the first key of the middle block
*/
public byte[] midkey() throws IOException {
@Override
public Cell midkey() throws IOException {
if (rootCount == 0)
throw new IOException("HFile empty");
byte[] targetMidKey = this.midKey.get();
Cell targetMidKey = this.midKey.get();
if (targetMidKey != null) {
return targetMidKey;
}
@ -348,7 +399,8 @@ public class HFileBlockIndex {
keyRelOffset;
int keyOffset = Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset
+ SECONDARY_INDEX_ENTRY_OVERHEAD;
targetMidKey = ByteBufferUtils.toBytes(b, keyOffset, keyLen);
byte[] bytes = ByteBufferUtils.toBytes(b, keyOffset, keyLen);
targetMidKey = new KeyValue.KeyOnlyKeyValue(bytes, 0, bytes.length);
} else {
// The middle of the root-level index.
targetMidKey = blockKeys[rootCount / 2];
@ -358,13 +410,174 @@ public class HFileBlockIndex {
return targetMidKey;
}
/**
* @param i from 0 to {@link #getRootBlockCount() - 1}
*/
public byte[] getRootBlockKey(int i) {
return blockKeys[i];
protected void initialize(int numEntries) {
blockKeys = new Cell[numEntries];
}
/**
* Adds a new entry in the root block index. Only used when reading.
*
* @param key Last key in the block
* @param offset file offset where the block is stored
* @param dataSize the uncompressed data size
*/
protected void add(final byte[] key, final long offset, final int dataSize) {
blockOffsets[rootCount] = offset;
// Create the blockKeys as Cells once when the reader is opened
blockKeys[rootCount] = new KeyValue.KeyOnlyKeyValue(key, 0, key.length);
blockDataSizes[rootCount] = dataSize;
rootCount++;
}
@Override
public int rootBlockContainingKey(final byte[] key, int offset, int length,
CellComparator comp) {
// This should always be called with Cell not with a byte[] key
throw new UnsupportedOperationException("Cannot find for a key containing plain byte " +
"array. Only cell based keys can be searched for");
}
@Override
public int rootBlockContainingKey(Cell key) {
// Here the comparator should not be null as this happens for the root-level block
int pos = Bytes.binarySearch(blockKeys, key, comparator);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
if (pos >= 0) {
// This means this is an exact match with an element of blockKeys.
assert pos < blockKeys.length;
return pos;
}
// Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
// and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
// blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
// key < blockKeys[0], meaning the file does not contain the given key.
int i = -pos - 1;
assert 0 <= i && i <= blockKeys.length;
return i - 1;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("size=" + rootCount).append("\n");
for (int i = 0; i < rootCount; i++) {
sb.append("key=").append((blockKeys[i]))
.append("\n offset=").append(blockOffsets[i])
.append(", dataSize=" + blockDataSizes[i]).append("\n");
}
return sb.toString();
}
}
/**
* The reader will always hold the root level index in the memory. Index
* blocks at all other levels will be cached in the LRU cache in practice,
* although this API does not enforce that.
*
* All non-root (leaf and intermediate) index blocks contain what we call a
* "secondary index": an array of offsets to the entries within the block.
* This allows us to do binary search for the entry corresponding to the
* given key without having to deserialize the block.
*/
static abstract class BlockIndexReader implements HeapSize {
protected long[] blockOffsets;
protected int[] blockDataSizes;
protected int rootCount = 0;
// Mid-key metadata.
protected long midLeafBlockOffset = -1;
protected int midLeafBlockOnDiskSize = -1;
protected int midKeyEntry = -1;
/**
* The number of levels in the block index tree. One if there is only root
* level, two for root and leaf levels, etc.
*/
protected int searchTreeLevel;
/** A way to read {@link HFile} blocks at a given offset */
protected CachingBlockReader cachingBlockReader;
/**
* @return true if the block index is empty.
*/
public abstract boolean isEmpty();
/**
* Verifies that the block index is non-empty and throws an
* {@link IllegalStateException} otherwise.
*/
public void ensureNonEmpty() {
if (isEmpty()) {
throw new IllegalStateException("Block index is empty or not loaded");
}
}
/**
* Return the data block which contains this key. This function will only
* be called when the HFile version is larger than 1.
*
* @param key the key we are looking for
* @param currentBlock the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding
* @return reader a basic way to load blocks
* @throws IOException
*/
public HFileBlock seekToDataBlock(final Cell key, HFileBlock currentBlock, boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, currentBlock,
cacheBlocks,
pread, isCompaction, expectedDataBlockEncoding);
if (blockWithScanInfo == null) {
return null;
} else {
return blockWithScanInfo.getHFileBlock();
}
}
/**
* Return the BlockWithScanInfo which contains the DataBlock with other scan
* info such as nextIndexedKey. This function will only be called when the
* HFile version is larger than 1.
*
* @param key
* the key we are looking for
* @param currentBlock
* the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding.
* @return the BlockWithScanInfo which contains the DataBlock with other
* scan info such as nextIndexedKey.
* @throws IOException
*/
public abstract BlockWithScanInfo loadDataBlockWithScanInfo(Cell key, HFileBlock currentBlock,
boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException;
/**
* An approximation to the {@link HFile}'s mid-key. Operates on block
* boundaries, and does not go inside blocks. In other words, returns the
* first key of the middle block of the file.
*
* @return the first key of the middle block
*/
public abstract Cell midkey() throws IOException;
/**
* @param i from 0 to {@link #getRootBlockCount() - 1}
*/
@ -402,27 +615,8 @@ public class HFileBlockIndex {
// When we want to find the meta index block or bloom block for ROW bloom
// type Bytes.BYTES_RAWCOMPARATOR would be enough. For the ROW_COL bloom case we need the
// CellComparator.
public int rootBlockContainingKey(final byte[] key, int offset, int length,
CellComparator comp) {
int pos = Bytes.binarySearch(blockKeys, key, offset, length, comp);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
if (pos >= 0) {
// This means this is an exact match with an element of blockKeys.
assert pos < blockKeys.length;
return pos;
}
// Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
// and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
// blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
// key < blockKeys[0], meaning the file does not contain the given key.
int i = -pos - 1;
assert 0 <= i && i <= blockKeys.length;
return i - 1;
}
public abstract int rootBlockContainingKey(final byte[] key, int offset, int length,
CellComparator comp);
/**
* Finds the root-level index block containing the given key.
@ -438,7 +632,7 @@ public class HFileBlockIndex {
// Bytes.BYTES_RAWCOMPARATOR would be enough. For the ROW_COL bloom case we
// need the CellComparator.
public int rootBlockContainingKey(final byte[] key, int offset, int length) {
return rootBlockContainingKey(key, offset, length, comparator);
return rootBlockContainingKey(key, offset, length, null);
}
/**
@ -447,41 +641,7 @@ public class HFileBlockIndex {
* @param key
* Key to find
*/
public int rootBlockContainingKey(final Cell key) {
// Here the comparator should not be null as this happens for the root-level block
int pos = Bytes.binarySearch(blockKeys, key, comparator);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
if (pos >= 0) {
// This means this is an exact match with an element of blockKeys.
assert pos < blockKeys.length;
return pos;
}
// Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
// and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
// blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
// key < blockKeys[0], meaning the file does not contain the given key.
int i = -pos - 1;
assert 0 <= i && i <= blockKeys.length;
return i - 1;
}
/**
* Adds a new entry in the root block index. Only used when reading.
*
* @param key Last key in the block
* @param offset file offset where the block is stored
* @param dataSize the uncompressed data size
*/
private void add(final byte[] key, final long offset, final int dataSize) {
blockOffsets[rootCount] = offset;
blockKeys[rootCount] = key;
blockDataSizes[rootCount] = dataSize;
rootCount++;
}
public abstract int rootBlockContainingKey(final Cell key);
/**
* The indexed key at the ith position in the nonRootIndex. The position starts at 0.
@ -489,7 +649,7 @@ public class HFileBlockIndex {
* @param i the ith position
* @return The indexed key at the ith position in the nonRootIndex.
*/
private byte[] getNonRootIndexedKey(ByteBuffer nonRootIndex, int i) {
protected byte[] getNonRootIndexedKey(ByteBuffer nonRootIndex, int i) {
int numEntries = nonRootIndex.getInt(0);
if (i < 0 || i >= numEntries) {
return null;
@ -653,10 +813,9 @@ public class HFileBlockIndex {
* @param numEntries the number of root-level index entries
* @throws IOException
*/
public void readRootIndex(DataInput in, final int numEntries)
throws IOException {
public void readRootIndex(DataInput in, final int numEntries) throws IOException {
blockOffsets = new long[numEntries];
blockKeys = new byte[numEntries][];
initialize(numEntries);
blockDataSizes = new int[numEntries];
// If index size is zero, no index was written.
@ -670,6 +829,10 @@ public class HFileBlockIndex {
}
}
protected abstract void initialize(int numEntries);
protected abstract void add(final byte[] key, final long offset, final int dataSize);
/**
* Read in the root-level index from the given input stream. Must match
* what was written into the root level by
@ -711,37 +874,16 @@ public class HFileBlockIndex {
midKeyEntry = in.readInt();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("size=" + rootCount).append("\n");
for (int i = 0; i < rootCount; i++) {
sb.append("key=").append(KeyValue.keyToString(blockKeys[i]))
.append("\n offset=").append(blockOffsets[i])
.append(", dataSize=" + blockDataSizes[i]).append("\n");
}
return sb.toString();
}
@Override
public long heapSize() {
long heapSize = ClassSize.align(6 * ClassSize.REFERENCE +
// The BlockIndexReader does not have the blockKey, comparator and the midkey atomic reference
long heapSize = ClassSize.align(3 * ClassSize.REFERENCE +
2 * Bytes.SIZEOF_INT + ClassSize.OBJECT);
// Mid-key metadata.
heapSize += MID_KEY_METADATA_SIZE;
// Calculating the size of blockKeys
if (blockKeys != null) {
// Adding array + references overhead
heapSize += ClassSize.align(ClassSize.ARRAY + blockKeys.length
* ClassSize.REFERENCE);
// Adding bytes
for (byte[] key : blockKeys) {
heapSize += ClassSize.align(ClassSize.ARRAY + key.length);
}
}
heapSize = calculateHeapSizeForBlockKeys(heapSize);
if (blockOffsets != null) {
heapSize += ClassSize.align(ClassSize.ARRAY + blockOffsets.length
@ -756,6 +898,7 @@ public class HFileBlockIndex {
return ClassSize.align(heapSize);
}
protected abstract long calculateHeapSizeForBlockKeys(long heapSize);
}
/**

View File

@ -410,7 +410,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
}
try {
System.out.println("Mid-key: " + Bytes.toStringBinary(reader.midkey()));
System.out.println("Mid-key: " + (CellUtil.getCellKeyAsString(reader.midkey())));
} catch (Exception e) {
System.out.println ("Unable to retrieve the midkey");
}

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NoTagsKeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -73,10 +72,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
private static final Log LOG = LogFactory.getLog(HFileReaderImpl.class);
/** Data block index reader keeping the root data index in memory */
private HFileBlockIndex.BlockIndexReader dataBlockIndexReader;
private HFileBlockIndex.CellBasedKeyBlockIndexReader dataBlockIndexReader;
/** Meta block index reader -- always single level */
private HFileBlockIndex.BlockIndexReader metaBlockIndexReader;
private HFileBlockIndex.ByteArrayKeyBlockIndexReader metaBlockIndexReader;
private final FixedFileTrailer trailer;
@ -189,10 +188,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
// Comparator class name is stored in the trailer in version 2.
comparator = trailer.createComparator();
dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator,
dataBlockIndexReader = new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator,
trailer.getNumDataIndexLevels(), this);
metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader(
null, 1);
metaBlockIndexReader = new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
// Parse load-on-open data.
@ -309,7 +307,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
}
private String toStringFirstKey() {
return KeyValue.keyToString(getFirstKey());
if(getFirstKey() == null)
return null;
return CellUtil.getCellKeyAsString(getFirstKey());
}
private String toStringLastKey() {
@ -341,7 +341,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* first KeyValue.
*/
@Override
public byte [] getFirstKey() {
public Cell getFirstKey() {
if (dataBlockIndexReader == null) {
throw new BlockIndexNotLoadedException();
}
@ -357,8 +357,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
*/
@Override
public byte[] getFirstRowKey() {
byte[] firstKey = getFirstKey();
return firstKey == null? null: KeyValueUtil.createKeyValueFromKey(firstKey).getRow();
Cell firstKey = getFirstKey();
// We have to copy the row part to form the row key alone
return firstKey == null? null: CellUtil.cloneRow(firstKey);
}
/**
@ -1215,7 +1216,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
// Per meta key from any given file, synchronize reads for said block. This
// is OK to do for meta blocks because the meta block index is always
// single-level.
synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
synchronized (metaBlockIndexReader
.getRootBlockKey(block)) {
// Check cache for block. If found return.
long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset);
@ -1387,7 +1389,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @throws IOException
*/
@Override
public byte[] midkey() throws IOException {
public Cell midkey() throws IOException {
return dataBlockIndexReader.midkey();
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -597,12 +598,12 @@ public class HRegionFileSystem {
} else {
//check if smaller than first key
KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
byte[] firstKey = f.createReader().getFirstKey();
Cell firstKey = f.createReader().getFirstKey();
// If firstKey is null means storefile is empty.
if (firstKey == null) {
return null;
}
if (f.getReader().getComparator().compare(splitKey, firstKey, 0, firstKey.length) < 0) {
if (f.getReader().getComparator().compare(splitKey, firstKey) < 0) {
return null;
}
}

View File

@ -1838,9 +1838,8 @@ public class HStore implements Store {
return false;
}
// TODO: Cache these keys rather than make each time?
byte [] fk = r.getFirstKey();
if (fk == null) return false;
KeyValue firstKV = KeyValueUtil.createKeyValueFromKey(fk, 0, fk.length);
Cell firstKV = r.getFirstKey();
if (firstKV == null) return false;
byte [] lk = r.getLastKey();
KeyValue lastKV = KeyValueUtil.createKeyValueFromKey(lk, 0, lk.length);
KeyValue firstOnRow = state.getTargetKey();
@ -1884,9 +1883,9 @@ public class HStore implements Store {
*/
private boolean seekToScanner(final HFileScanner scanner,
final KeyValue firstOnRow,
final KeyValue firstKV)
final Cell firstKV)
throws IOException {
KeyValue kv = firstOnRow;
Cell kv = firstOnRow;
// If firstOnRow < firstKV, set to firstKV
if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
int result = scanner.seekTo(kv);

View File

@ -678,21 +678,20 @@ public class StoreFile {
// Get first, last, and mid keys. Midkey is the key that starts block
// in middle of hfile. Has column and timestamp. Need to return just
// the row we want to split on as midkey.
byte [] midkey = this.reader.midkey();
Cell midkey = this.reader.midkey();
if (midkey != null) {
KeyValue mk = KeyValueUtil.createKeyValueFromKey(midkey, 0, midkey.length);
byte [] fk = this.reader.getFirstKey();
KeyValue firstKey = KeyValueUtil.createKeyValueFromKey(fk, 0, fk.length);
Cell firstKey = this.reader.getFirstKey();
byte [] lk = this.reader.getLastKey();
KeyValue lastKey = KeyValueUtil.createKeyValueFromKey(lk, 0, lk.length);
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
if (comparator.compareRows(mk, firstKey) == 0 || comparator.compareRows(mk, lastKey) == 0) {
if (comparator.compareRows(midkey, firstKey) == 0
|| comparator.compareRows(midkey, lastKey) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("cannot split because midkey is the same as first or last row");
}
return null;
}
return mk.getRow();
return CellUtil.cloneRow(midkey);
}
return null;
}
@ -1371,8 +1370,7 @@ public class StoreFile {
.createLastOnRow(scan.getStartRow()) : KeyValueUtil.createLastOnRow(scan
.getStopRow());
// TODO this is in hot path? Optimize and avoid 2 extra object creations.
KeyValue.KeyOnlyKeyValue firstKeyKV =
new KeyValue.KeyOnlyKeyValue(this.getFirstKey(), 0, this.getFirstKey().length);
Cell firstKeyKV = this.getFirstKey();
KeyValue.KeyOnlyKeyValue lastKeyKV =
new KeyValue.KeyOnlyKeyValue(this.getLastKey(), 0, this.getLastKey().length);
boolean nonOverLapping = ((getComparator().compare(firstKeyKV, largestScanKeyValue)) > 0
@ -1493,7 +1491,7 @@ public class StoreFile {
return reader.getLastRowKey();
}
public byte[] midkey() throws IOException {
public Cell midkey() throws IOException {
return reader.midkey();
}
@ -1513,7 +1511,7 @@ public class StoreFile {
return deleteFamilyCnt;
}
public byte[] getFirstKey() {
public Cell getFirstKey() {
return reader.getFirstKey();
}

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import com.google.common.annotations.VisibleForTesting;
/**
* The basic building block for the {@link CompoundBloomFilter}
* The basic building block for the {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter}
*/
@InterfaceAudience.Private
public class BloomFilterChunk implements BloomFilterBase {

View File

@ -28,6 +28,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.BloomType;

View File

@ -776,8 +776,8 @@ public class HBaseFsck extends Configured implements Closeable {
getConf()), getConf());
if ((reader.getFirstKey() != null)
&& ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
reader.getFirstKey()) > 0))) {
storeFirstKey = reader.getFirstKey();
((KeyValue.KeyOnlyKeyValue) reader.getFirstKey()).getKey()) > 0))) {
storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey()).getKey();
}
if ((reader.getLastKey() != null)
&& ((storeLastKey == null) || (comparator.compare(storeLastKey,
@ -790,7 +790,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey();
currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey();
currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey);
currentRegionBoundariesInformation.storesFirstKey = storeFirstKey;
currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey);
if (currentRegionBoundariesInformation.metaFirstKey.length == 0)
currentRegionBoundariesInformation.metaFirstKey = null;
@ -879,7 +879,7 @@ public class HBaseFsck extends Configured implements Closeable {
CacheConfig cacheConf = new CacheConfig(getConf());
hf = HFile.createReader(fs, hfile.getPath(), cacheConf, getConf());
hf.loadFileInfo();
KeyValue startKv = KeyValueUtil.createKeyValueFromKey(hf.getFirstKey());
Cell startKv = hf.getFirstKey();
start = startKv.getRow();
KeyValue endKv = KeyValueUtil.createKeyValueFromKey(hf.getLastKey());
end = endKv.getRow();

View File

@ -101,9 +101,8 @@ public class TestHalfStoreFileReader {
HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
r.loadFileInfo();
byte [] midkey = r.midkey();
KeyValue midKV = KeyValueUtil.createKeyValueFromKey(midkey);
midkey = midKV.getRow();
Cell midKV = r.midkey();
byte[] midkey = ((KeyValue.KeyOnlyKeyValue)midKV).getRow();
//System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
@ -167,9 +166,8 @@ public class TestHalfStoreFileReader {
HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
r.loadFileInfo();
byte[] midkey = r.midkey();
KeyValue midKV = KeyValueUtil.createKeyValueFromKey(midkey);
midkey = midKV.getRow();
Cell midKV = r.midkey();
byte[] midkey = ((KeyValue.KeyOnlyKeyValue)midKV).getRow();
Reference bottom = new Reference(midkey, Reference.Range.bottom);
Reference top = new Reference(midkey, Reference.Range.top);
@ -217,7 +215,7 @@ public class TestHalfStoreFileReader {
assertNull(foundKeyValue);
}
private Cell doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, KeyValue seekBefore,
private Cell doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, Cell seekBefore,
CacheConfig cacheConfig)
throws IOException {
final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,

View File

@ -205,7 +205,7 @@ public class TestHFileBlockIndex {
BlockReaderWrapper brw = new BlockReaderWrapper(blockReader);
HFileBlockIndex.BlockIndexReader indexReader =
new HFileBlockIndex.BlockIndexReader(
new HFileBlockIndex.CellBasedKeyBlockIndexReader(
CellComparator.COMPARATOR, numLevels, brw);
indexReader.readRootIndex(blockReader.blockRange(rootIndexOffset,
@ -493,16 +493,17 @@ public class TestHFileBlockIndex {
long expected = ClassSize.estimateBase(cl, false);
HFileBlockIndex.BlockIndexReader bi =
new HFileBlockIndex.BlockIndexReader(null, 1);
new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
long actual = bi.heapSize();
// Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets,
// int [] blockDataSizes) are all null they are not going to show up in the
// HeapSize calculation, so need to remove those array costs from expected.
expected -= ClassSize.align(3 * ClassSize.ARRAY);
// Already the block keys are not there in this case
expected -= ClassSize.align(2 * ClassSize.ARRAY);
if (expected != actual) {
ClassSize.estimateBase(cl, true);
expected = ClassSize.estimateBase(cl, true);
assertEquals(expected, actual);
}
}
@ -574,7 +575,7 @@ public class TestHFileBlockIndex {
assertEquals(expectedNumLevels,
reader.getTrailer().getNumDataIndexLevels());
assertTrue(Bytes.equals(keys[0], reader.getFirstKey()));
assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey()).getKey()));
assertTrue(Bytes.equals(keys[NUM_KV - 1], reader.getLastKey()));
LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1]));
@ -631,7 +632,7 @@ public class TestHFileBlockIndex {
// Validate the mid-key.
assertEquals(
Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)),
Bytes.toStringBinary(reader.midkey()));
reader.midkey());
assertEquals(UNCOMPRESSED_INDEX_SIZES[testI],
reader.getTrailer().getUncompressedDataIndexSize());

View File

@ -187,7 +187,7 @@ public class TestHFileSeek extends TestCase {
fs.getFileStatus(path).getLen(), new CacheConfig(conf), conf);
reader.loadFileInfo();
KeySampler kSampler =
new KeySampler(rng, reader.getFirstKey(), reader.getLastKey(),
new KeySampler(rng, ((KeyValue)reader.getFirstKey()).getKey(), reader.getLastKey(),
keyLenGen);
HFileScanner scanner = reader.getScanner(false, USE_PREAD);
BytesWritable key = new BytesWritable();

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
@ -148,11 +149,10 @@ public class TestHFileWriterV2 {
// Comparator class name is stored in the trailer in version 2.
CellComparator comparator = trailer.createComparator();
HFileBlockIndex.BlockIndexReader dataBlockIndexReader =
new HFileBlockIndex.BlockIndexReader(comparator,
new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator,
trailer.getNumDataIndexLevels());
HFileBlockIndex.BlockIndexReader metaBlockIndexReader =
new HFileBlockIndex.BlockIndexReader(
null, 1);
new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
HFileBlock.BlockIterator blockIter = blockReader.blockRange(
trailer.getLoadOnOpenDataOffset(),
@ -164,7 +164,7 @@ public class TestHFileWriterV2 {
trailer.getDataIndexCount());
if (findMidKey) {
byte[] midkey = dataBlockIndexReader.midkey();
Cell midkey = dataBlockIndexReader.midkey();
assertNotNull("Midkey should not be null", midkey);
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
@ -179,11 +180,10 @@ public class TestHFileWriterV3 {
// Comparator class name is stored in the trailer in version 2.
CellComparator comparator = trailer.createComparator();
HFileBlockIndex.BlockIndexReader dataBlockIndexReader =
new HFileBlockIndex.BlockIndexReader(comparator,
new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator,
trailer.getNumDataIndexLevels());
HFileBlockIndex.BlockIndexReader metaBlockIndexReader =
new HFileBlockIndex.BlockIndexReader(
null, 1);
new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
HFileBlock.BlockIterator blockIter = blockReader.blockRange(
trailer.getLoadOnOpenDataOffset(),
@ -194,7 +194,7 @@ public class TestHFileWriterV3 {
blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), trailer.getDataIndexCount());
if (findMidKey) {
byte[] midkey = dataBlockIndexReader.midkey();
Cell midkey = dataBlockIndexReader.midkey();
assertNotNull("Midkey should not be null", midkey);
}

View File

@ -46,15 +46,14 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompoundBloomFilter;
import org.apache.hadoop.hbase.util.CompoundBloomFilterBase;
import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.junit.Before;
import org.junit.Test;

View File

@ -170,7 +170,7 @@ public class TestStoreFile extends HBaseTestCase {
// Split on a row, not in middle of row. Midkey returned by reader
// may be in middle of row. Create new one with empty column and
// timestamp.
KeyValue kv = KeyValueUtil.createKeyValueFromKey(reader.midkey());
Cell kv = reader.midkey();
byte [] midRow = kv.getRow();
kv = KeyValueUtil.createKeyValueFromKey(reader.getLastKey());
byte [] finalRow = kv.getRow();
@ -314,8 +314,8 @@ public class TestStoreFile extends HBaseTestCase {
private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f)
throws IOException {
byte [] midkey = f.createReader().midkey();
KeyValue midKV = KeyValueUtil.createKeyValueFromKey(midkey);
Cell midkey = f.createReader().midkey();
KeyValue midKV = (KeyValue)midkey;
byte [] midRow = midKV.getRow();
// Create top split.
HRegionInfo topHri = new HRegionInfo(regionFs.getRegionInfo().getTable(),
@ -332,7 +332,7 @@ public class TestStoreFile extends HBaseTestCase {
this.fs, bottomPath, conf, cacheConf, BloomType.NONE).createReader();
ByteBuffer previous = null;
LOG.info("Midkey: " + midKV.toString());
ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey);
ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midKV.getKey());
try {
// Now make two HalfMapFiles and assert they can read the full backing
// file, one from the top and the other from the bottom.
@ -348,7 +348,7 @@ public class TestStoreFile extends HBaseTestCase {
if ((topScanner.getReader().getComparator().compare(midKV, key.array(),
key.arrayOffset(), key.limit())) > 0) {
fail("key=" + Bytes.toStringBinary(key) + " < midkey=" +
Bytes.toStringBinary(midkey));
midkey);
}
if (first) {
first = false;