From cbb334035d87542ed06693bb9c8534f64360672b Mon Sep 17 00:00:00 2001 From: anoopsjohn Date: Wed, 29 Oct 2014 14:39:03 +0530 Subject: [PATCH] HBASE-12297 Support DBB usage in Bloom and HFileIndex area. --- .../HFileBlockDefaultDecodingContext.java | 1 + .../apache/hadoop/hbase/util/ByteBufferUtils.java | 15 +++++++++++++++ .../hadoop/hbase/io/hfile/ChecksumUtil.java | 7 ++++++- .../apache/hadoop/hbase/io/hfile/HFileBlock.java | 1 + .../hadoop/hbase/io/hfile/HFileBlockIndex.java | 15 +++++++-------- .../apache/hadoop/hbase/util/ByteBloomFilter.java | 14 +++++++------- .../hadoop/hbase/util/CompoundBloomFilter.java | 2 +- 7 files changed, 38 insertions(+), 17 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java index 18407d13c8a..78bb0d6af0d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java @@ -85,6 +85,7 @@ public class HFileBlockDefaultDecodingContext implements } Compression.Algorithm compression = fileContext.getCompression(); + assert blockBufferWithoutHeader.hasArray(); if (compression != Compression.Algorithm.NONE) { Compression.decompress(blockBufferWithoutHeader.array(), blockBufferWithoutHeader.arrayOffset(), in, onDiskSizeWithoutHeader, diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 2a7b8c0e938..26899469a8f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -478,6 +478,21 @@ public final class ByteBufferUtils { 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) { if (buf1.hasArray() && buf2.hasArray()) { return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java index 328221326b0..f34b83dc144 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.zip.Checksum; 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.ChecksumType; @@ -132,7 +133,11 @@ public class ChecksumUtil { } // Extract the header and compute checksum for the header. 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 consumed = hdrSize; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index 7f3d80f55fc..6e08bc29481 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -1583,6 +1583,7 @@ public class HFileBlock implements Cacheable { // the header has been read when reading the previous block, copy // to this block's header // headerBuf is HBB + assert headerBuf.hasArray(); System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index 5140181e9f6..d9c10c4d2ee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -26,7 +26,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; 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.encoding.DataBlockEncoding; 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.ClassSize; import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; @@ -345,10 +345,9 @@ public class HFileBlockIndex { int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1)); int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - keyRelOffset; - int keyOffset = b.arrayOffset() + - Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset + - SECONDARY_INDEX_ENTRY_OVERHEAD; - targetMidKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen); + int keyOffset = Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset + + SECONDARY_INDEX_ENTRY_OVERHEAD; + targetMidKey = ByteBufferUtils.toBytes(b, keyOffset, keyLen); } else { // The middle of the root-level index. targetMidKey = blockKeys[rootCount / 2]; @@ -489,9 +488,7 @@ public class HFileBlockIndex { int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) - targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD; - int from = nonRootIndex.arrayOffset() + targetKeyOffset; - int to = from + targetKeyLength; - return Arrays.copyOfRange(nonRootIndex.array(), from, to); + return ByteBufferUtils.toBytes(nonRootIndex, targetKeyOffset, targetKeyLength); } /** @@ -546,6 +543,8 @@ public class HFileBlockIndex { // we have to compare in this order, because the comparator order // 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(), nonRootIndex.arrayOffset() + midKeyOffset, midLength); int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java index 73b9edf65d1..56c37762f66 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java @@ -415,12 +415,11 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); } - return contains(buf, offset, length, theBloom.array(), - theBloom.arrayOffset(), (int) byteSize, hash, hashCount); + return contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, hashCount); } 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 hash1 = hash.hash(buf, offset, length, 0); @@ -433,7 +432,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { for (int i = 0; i < hashCount; i++) { int hashLoc = Math.abs(compositeHash % bloomBitSize); compositeHash += hash2; - if (!get(hashLoc, bloomArray, bloomOffset)) { + if (!get(hashLoc, bloomBuf, bloomOffset)) { return false; } } @@ -441,7 +440,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { // Test mode with "fake lookups" to estimate "ideal false positive rate". for (int i = 0; i < hashCount; i++) { int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); - if (!get(hashLoc, bloomArray, bloomOffset)){ + if (!get(hashLoc, bloomBuf, bloomOffset)){ return false; } } @@ -471,10 +470,11 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { * @param pos index of bit * @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 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]; return (curByte != 0); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java index 2661f3f9c6e..beda805ed1a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java @@ -109,7 +109,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); result = ByteBloomFilter.contains(key, keyOffset, keyLength, - bloomBuf.array(), bloomBuf.arrayOffset() + bloomBlock.headerSize(), + bloomBuf, bloomBlock.headerSize(), bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); }