HBASE-12297 Support DBB usage in Bloom and HFileIndex area.

This commit is contained in:
anoopsjohn 2014-10-29 14:39:03 +05:30
parent 3a9cf5b2cd
commit cbb334035d
7 changed files with 38 additions and 17 deletions

View File

@ -85,6 +85,7 @@ public class HFileBlockDefaultDecodingContext implements
} }
Compression.Algorithm compression = fileContext.getCompression(); Compression.Algorithm compression = fileContext.getCompression();
assert blockBufferWithoutHeader.hasArray();
if (compression != Compression.Algorithm.NONE) { if (compression != Compression.Algorithm.NONE) {
Compression.decompress(blockBufferWithoutHeader.array(), Compression.decompress(blockBufferWithoutHeader.array(),
blockBufferWithoutHeader.arrayOffset(), in, onDiskSizeWithoutHeader, blockBufferWithoutHeader.arrayOffset(), in, onDiskSizeWithoutHeader,

View File

@ -478,6 +478,21 @@ public final class ByteBufferUtils {
return output; return output;
} }
/**
* Copy the given number of bytes from specified offset into a new byte[]
* @param buffer
* @param offset
* @param length
* @return a new byte[] containing the bytes in the specified range
*/
public static byte[] toBytes(ByteBuffer buffer, int offset, int length) {
byte[] output = new byte[length];
for (int i = 0; i < length; i++) {
output[i] = buffer.get(offset + i);
}
return output;
}
public static int compareTo(ByteBuffer buf1, int o1, int len1, ByteBuffer buf2, int o2, int len2) { public static int compareTo(ByteBuffer buf1, int o1, int len1, ByteBuffer buf2, int o2, int len2) {
if (buf1.hasArray() && buf2.hasArray()) { if (buf1.hasArray() && buf2.hasArray()) {
return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(), return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(),

View File

@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
import java.util.zip.Checksum; import java.util.zip.Checksum;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.ChecksumType;
@ -132,7 +133,11 @@ public class ChecksumUtil {
} }
// Extract the header and compute checksum for the header. // Extract the header and compute checksum for the header.
ByteBuffer hdr = block.getBufferWithHeader(); ByteBuffer hdr = block.getBufferWithHeader();
checksumObject.update(hdr.array(), hdr.arrayOffset(), hdrSize); if (hdr.hasArray()) {
checksumObject.update(hdr.array(), hdr.arrayOffset(), hdrSize);
} else {
checksumObject.update(ByteBufferUtils.toBytes(hdr, 0, hdrSize), 0, hdrSize);
}
int off = hdrSize; int off = hdrSize;
int consumed = hdrSize; int consumed = hdrSize;

View File

@ -1583,6 +1583,7 @@ public class HFileBlock implements Cacheable {
// the header has been read when reading the previous block, copy // the header has been read when reading the previous block, copy
// to this block's header // to this block's header
// headerBuf is HBB // headerBuf is HBB
assert headerBuf.hasArray();
System.arraycopy(headerBuf.array(), System.arraycopy(headerBuf.array(),
headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize); headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
} else { } else {

View File

@ -26,7 +26,6 @@ import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader; import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
@ -345,10 +345,9 @@ public class HFileBlockIndex {
int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1)); int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1));
int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) -
keyRelOffset; keyRelOffset;
int keyOffset = b.arrayOffset() + int keyOffset = Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset
Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset + + SECONDARY_INDEX_ENTRY_OVERHEAD;
SECONDARY_INDEX_ENTRY_OVERHEAD; targetMidKey = ByteBufferUtils.toBytes(b, keyOffset, keyLen);
targetMidKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen);
} else { } else {
// The middle of the root-level index. // The middle of the root-level index.
targetMidKey = blockKeys[rootCount / 2]; targetMidKey = blockKeys[rootCount / 2];
@ -489,9 +488,7 @@ public class HFileBlockIndex {
int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) - int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) -
targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD; targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD;
int from = nonRootIndex.arrayOffset() + targetKeyOffset; return ByteBufferUtils.toBytes(nonRootIndex, targetKeyOffset, targetKeyLength);
int to = from + targetKeyLength;
return Arrays.copyOfRange(nonRootIndex.array(), from, to);
} }
/** /**
@ -546,6 +543,8 @@ public class HFileBlockIndex {
// we have to compare in this order, because the comparator order // we have to compare in this order, because the comparator order
// has special logic when the 'left side' is a special key. // has special logic when the 'left side' is a special key.
// TODO make KeyOnlyKeyValue to be Buffer backed and avoid array() call. This has to be
// done after HBASE-12224 & HBASE-12282
nonRootIndexKV.setKey(nonRootIndex.array(), nonRootIndexKV.setKey(nonRootIndex.array(),
nonRootIndex.arrayOffset() + midKeyOffset, midLength); nonRootIndex.arrayOffset() + midKeyOffset, midLength);
int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV); int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV);

View File

@ -415,12 +415,11 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
+ " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
} }
return contains(buf, offset, length, theBloom.array(), return contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, hashCount);
theBloom.arrayOffset(), (int) byteSize, hash, hashCount);
} }
public static boolean contains(byte[] buf, int offset, int length, public static boolean contains(byte[] buf, int offset, int length,
byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash, ByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash,
int hashCount) { int hashCount) {
int hash1 = hash.hash(buf, offset, length, 0); int hash1 = hash.hash(buf, offset, length, 0);
@ -433,7 +432,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
for (int i = 0; i < hashCount; i++) { for (int i = 0; i < hashCount; i++) {
int hashLoc = Math.abs(compositeHash % bloomBitSize); int hashLoc = Math.abs(compositeHash % bloomBitSize);
compositeHash += hash2; compositeHash += hash2;
if (!get(hashLoc, bloomArray, bloomOffset)) { if (!get(hashLoc, bloomBuf, bloomOffset)) {
return false; return false;
} }
} }
@ -441,7 +440,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
// Test mode with "fake lookups" to estimate "ideal false positive rate". // Test mode with "fake lookups" to estimate "ideal false positive rate".
for (int i = 0; i < hashCount; i++) { for (int i = 0; i < hashCount; i++) {
int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
if (!get(hashLoc, bloomArray, bloomOffset)){ if (!get(hashLoc, bloomBuf, bloomOffset)){
return false; return false;
} }
} }
@ -471,10 +470,11 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
* @param pos index of bit * @param pos index of bit
* @return true if bit at specified index is 1, false if 0. * @return true if bit at specified index is 1, false if 0.
*/ */
static boolean get(int pos, byte[] bloomArray, int bloomOffset) { static boolean get(int pos, ByteBuffer bloomBuf, int bloomOffset) {
int bytePos = pos >> 3; //pos / 8 int bytePos = pos >> 3; //pos / 8
int bitPos = pos & 0x7; //pos % 8 int bitPos = pos & 0x7; //pos % 8
byte curByte = bloomArray[bloomOffset + bytePos]; // TODO access this via Util API which can do Unsafe access if possible(?)
byte curByte = bloomBuf.get(bloomOffset + bytePos);
curByte &= bitvals[bitPos]; curByte &= bitvals[bitPos];
return (curByte != 0); return (curByte != 0);
} }

View File

@ -109,7 +109,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly();
result = ByteBloomFilter.contains(key, keyOffset, keyLength, result = ByteBloomFilter.contains(key, keyOffset, keyLength,
bloomBuf.array(), bloomBuf.arrayOffset() + bloomBlock.headerSize(), bloomBuf, bloomBlock.headerSize(),
bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount);
} }