diff --git a/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 722aa21936a..7910c4ada94 100644 --- a/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -27,7 +27,6 @@ import java.util.HashMap; import java.util.Map; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -54,18 +53,10 @@ public class HColumnDescriptor implements WritableComparable // Version 6 adds metadata as a map where keys and values are byte[]. // Version 7 -- add new compression and hfile blocksize to HColumnDescriptor (HBASE-1217) // Version 8 -- reintroduction of bloom filters, changed from boolean to enum - // Version 9 -- add data block encoding - private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 9; + private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)8; - // These constants are used as FileInfo keys public static final String COMPRESSION = "COMPRESSION"; public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT"; - public static final String DATA_BLOCK_ENCODING_ON_DISK = - "DATA_BLOCK_ENCODING_ON_DISK"; - public static final String DATA_BLOCK_ENCODING_IN_CACHE = - "DATA_BLOCK_ENCODING_IN_CACHE"; - public static final String ENCODED_DATA_BLOCK_SEEK = - "ENCODED_DATA_BLOCK_SEEK"; public static final String BLOCKCACHE = "BLOCKCACHE"; /** @@ -88,23 +79,6 @@ public class HColumnDescriptor implements WritableComparable */ public static final String DEFAULT_COMPRESSION = Compression.Algorithm.NONE.getName(); - - /** - * Default data block encoding algorithm on disk. - */ - public static final String DEFAULT_DATA_BLOCK_ENCODING_ON_DISK = - DataBlockEncodings.Algorithm.NONE.toString(); - - /** - * Default data block encoding algorithm in cache. - */ - public static final String DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE = - DataBlockEncodings.Algorithm.NONE.toString(); - - /** - * Do not use encoded seek by default. - */ - public static final boolean DEFAULT_ENCODED_DATA_BLOCK_SEEK = false; /** * Default number of versions of a record to keep. @@ -169,12 +143,6 @@ public class HColumnDescriptor implements WritableComparable DEFAULT_VALUES.put(HConstants.IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY)); DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE)); DEFAULT_VALUES.put(KEEP_DELETED_CELLS, String.valueOf(DEFAULT_KEEP_DELETED)); - DEFAULT_VALUES.put(DATA_BLOCK_ENCODING_ON_DISK, - String.valueOf(DEFAULT_DATA_BLOCK_ENCODING_ON_DISK)); - DEFAULT_VALUES.put(DATA_BLOCK_ENCODING_IN_CACHE, - String.valueOf(DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE)); - DEFAULT_VALUES.put(ENCODED_DATA_BLOCK_SEEK, - String.valueOf(DEFAULT_ENCODED_DATA_BLOCK_SEEK)); } // Column family name @@ -290,9 +258,8 @@ public class HColumnDescriptor implements WritableComparable final boolean blockCacheEnabled, final int blocksize, final int timeToLive, final String bloomFilter, final int scope) { this(familyName, DEFAULT_MIN_VERSIONS, maxVersions, DEFAULT_KEEP_DELETED, - compression, DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, DEFAULT_ENCODED_DATA_BLOCK_SEEK, - inMemory, blockCacheEnabled, blocksize, timeToLive, bloomFilter, scope); + compression, inMemory, blockCacheEnabled, blocksize, timeToLive, + bloomFilter, scope); } /** @@ -304,9 +271,6 @@ public class HColumnDescriptor implements WritableComparable * @param keepDeletedCells Whether to retain deleted cells until they expire * up to maxVersions versions. * @param compression Compression type - * @param dataBlockEncodingOnDisk data block encoding used on disk - * @param dataBlockEncodingInCache data block encoding used in block cache - * @param encodedDataBlockSeek whether to use data block encoding while seeking * @param inMemory If true, column data should be kept in an HRegionServer's * cache * @param blockCacheEnabled If true, MapFile blocks should be cached @@ -323,12 +287,9 @@ public class HColumnDescriptor implements WritableComparable * a : * @throws IllegalArgumentException if the number of versions is <= 0 */ - public HColumnDescriptor(final byte [] familyName, final int minVersions, + public HColumnDescriptor(final byte[] familyName, final int minVersions, final int maxVersions, final boolean keepDeletedCells, - final String compression, - final String dataBlockEncodingOnDisk, - final String dataBlockEncodingInCache, - final boolean encodedDataBlockSeek, final boolean inMemory, + final String compression, final boolean inMemory, final boolean blockCacheEnabled, final int blocksize, final int timeToLive, final String bloomFilter, final int scope) { isLegalFamilyName(familyName); @@ -358,11 +319,6 @@ public class HColumnDescriptor implements WritableComparable setTimeToLive(timeToLive); setCompressionType(Compression.Algorithm. valueOf(compression.toUpperCase())); - setDataBlockEncodingOnDisk(DataBlockEncodings.Algorithm. - valueOf(dataBlockEncodingOnDisk.toUpperCase())); - setDataBlockEncodingInCache(DataBlockEncodings.Algorithm. - valueOf(dataBlockEncodingInCache.toUpperCase())); - setEncodedDataBlockSeek(encodedDataBlockSeek); setBloomFilterType(StoreFile.BloomType. valueOf(bloomFilter.toUpperCase())); setBlocksize(blocksize); @@ -540,71 +496,6 @@ public class HColumnDescriptor implements WritableComparable setValue(COMPRESSION, compressionType); } - /** @return data block encoding algorithm used on disk */ - public DataBlockEncodings.Algorithm getDataBlockEncodingOnDisk() { - String type = getValue(DATA_BLOCK_ENCODING_ON_DISK); - if (type == null) { - type = DEFAULT_DATA_BLOCK_ENCODING_ON_DISK; - } - return DataBlockEncodings.Algorithm.valueOf(type); - } - - /** - * Set data block encoding algorithm. - * @param type What kind of data block encoding will be used. - */ - public void setDataBlockEncodingOnDisk( - DataBlockEncodings.Algorithm type) { - String name; - if (type != null) { - name = type.toString(); - } else { - name = DataBlockEncodings.Algorithm.NONE.toString(); - } - setValue(DATA_BLOCK_ENCODING_ON_DISK, name); - } - - /** @return data block encoding in block cache algorithm */ - public DataBlockEncodings.Algorithm getDataBlockEncodingInCache() { - String type = getValue(DATA_BLOCK_ENCODING_IN_CACHE); - if (type == null) { - type = DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE; - } - return DataBlockEncodings.Algorithm.valueOf(type); - } - - /** - * Set data block encoding algorithm used in block cache. - * @param type What kind of data block encoding will be used. - */ - public void setDataBlockEncodingInCache( - DataBlockEncodings.Algorithm type) { - String name; - if (type != null) { - name = type.toString(); - } else { - name = DataBlockEncodings.Algorithm.NONE.toString(); - } - setValue(DATA_BLOCK_ENCODING_IN_CACHE, name); - } - - /** @return whether we are doing seek operations over encoded data blocks */ - public boolean useEncodedDataBlockSeek() { - String value = getValue(ENCODED_DATA_BLOCK_SEEK); - if (value != null) { - return Boolean.valueOf(value); - } - return DEFAULT_ENCODED_DATA_BLOCK_SEEK; - } - - /** - * Set whether we should seek over encoded data blocks (true) or decode - * blocks first and use normal seek operations (false). - */ - public void setEncodedDataBlockSeek(boolean useEncodedSeek) { - setValue(ENCODED_DATA_BLOCK_SEEK, Boolean.toString(useEncodedSeek)); - } - /** * @return Compression type setting. */ diff --git a/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/src/main/java/org/apache/hadoop/hbase/KeyValue.java index 8a032ab92a1..be7e2d89455 100644 --- a/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -130,27 +130,16 @@ public class KeyValue implements Writable, HeapSize { return COMPARATOR.getRawComparator(); } - /** Size of the key length field in bytes*/ - public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT; - - /** Size of the key type field in bytes */ - public static final int TYPE_SIZE = Bytes.SIZEOF_BYTE; - - /** Size of the row length field in bytes */ - public static final int ROW_LENGTH_SIZE = Bytes.SIZEOF_SHORT; - - /** Size of the family length field in bytes */ - public static final int FAMILY_LENGTH_SIZE = Bytes.SIZEOF_BYTE; - - /** Size of the timestamp field in bytes */ - public static final int TIMESTAMP_SIZE = Bytes.SIZEOF_LONG; - // Size of the timestamp and type byte on end of a key -- a long + a byte. - public static final int TIMESTAMP_TYPE_SIZE = TIMESTAMP_SIZE + TYPE_SIZE; + public static final int TIMESTAMP_TYPE_SIZE = + Bytes.SIZEOF_LONG /* timestamp */ + + Bytes.SIZEOF_BYTE /*keytype*/; // Size of the length shorts and bytes in key. - public static final int KEY_INFRASTRUCTURE_SIZE = ROW_LENGTH_SIZE - + FAMILY_LENGTH_SIZE + TIMESTAMP_TYPE_SIZE; + public static final int KEY_INFRASTRUCTURE_SIZE = + Bytes.SIZEOF_SHORT /*rowlength*/ + + Bytes.SIZEOF_BYTE /*columnfamilylength*/ + + TIMESTAMP_TYPE_SIZE; // How far into the key the row starts at. First thing to read is the short // that says how long the row is. @@ -712,10 +701,10 @@ public class KeyValue implements Writable, HeapSize { */ /** * Produces a string map for this key/value pair. Useful for programmatic use - * and manipulation of the data stored in an HLogKey, for example, printing - * as JSON. Values are left out due to their tendency to be large. If needed, + * and manipulation of the data stored in an HLogKey, for example, printing + * as JSON. Values are left out due to their tendency to be large. If needed, * they can be added manually. - * + * * @return the Map containing data from this key */ public Map toStringMap() { @@ -1333,7 +1322,7 @@ public class KeyValue implements Writable, HeapSize { // Rebuild as: <0:4> int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0; byte [] newBuffer = new byte[getKeyLength() + (2 * Bytes.SIZEOF_INT) + dataLen]; - System.arraycopy(this.bytes, this.offset, newBuffer, 0, + System.arraycopy(this.bytes, this.offset, newBuffer, 0, Math.min(newBuffer.length,this.length)); Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen); if (lenAsVal) { @@ -1404,7 +1393,7 @@ public class KeyValue implements Writable, HeapSize { } /** - * This function is only used in Meta key comparisons so its error message + * This function is only used in Meta key comparisons so its error message * is specific for meta key errors. */ static int getRequiredDelimiterInReverse(final byte [] b, @@ -1572,7 +1561,7 @@ public class KeyValue implements Writable, HeapSize { return getRawComparator().compareRows(left, loffset, llength, right, roffset, rlength); } - + public int compareColumns(final KeyValue left, final byte [] right, final int roffset, final int rlength, final int rfamilyoffset) { int offset = left.getFamilyOffset(); @@ -1606,7 +1595,7 @@ public class KeyValue implements Writable, HeapSize { short lrowlength = left.getRowLength(); short rrowlength = right.getRowLength(); // TsOffset = end of column data. just comparing Row+CF length of each - return ((left.getTimestampOffset() - left.getOffset()) == + return ((left.getTimestampOffset() - left.getOffset()) == (right.getTimestampOffset() - right.getOffset())) && matchingRows(left, lrowlength, right, rrowlength) && compareColumns(left, lrowlength, right, rrowlength) == 0; @@ -2014,24 +2003,10 @@ public class KeyValue implements Writable, HeapSize { } } - /** - * Avoids redundant comparisons for better performance. - */ - public static interface SamePrefixComparator { - /** - * Compare two keys assuming that the first n bytes are the same. - * @param commonPrefix How many bytes are the same. - */ - public int compareIgnoringPrefix(int commonPrefix, - T left, int loffset, int llength, - T right, int roffset, int rlength); - } - /** * Compare key portion of a {@link KeyValue}. */ - public static class KeyComparator - implements RawComparator, SamePrefixComparator { + public static class KeyComparator implements RawComparator { volatile boolean ignoreTimestamp = false; volatile boolean ignoreType = false; @@ -2041,123 +2016,45 @@ public class KeyValue implements Writable, HeapSize { short lrowlength = Bytes.toShort(left, loffset); short rrowlength = Bytes.toShort(right, roffset); int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT, - lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength); + lrowlength, + right, roffset + Bytes.SIZEOF_SHORT, rrowlength); if (compare != 0) { return compare; } - // Compare the rest of the two KVs without making any assumptions about - // the common prefix. This function will not compare rows anyway, so we - // don't need to tell it that the common prefix includes the row. - return compareWithoutRow(0, left, loffset, llength, right, roffset, - rlength, rrowlength); - } - - /** - * Compare the two key-values, ignoring the prefix of the given length - * that is known to be the same between the two. - * @param commonPrefix the prefix length to ignore - */ - @Override - public int compareIgnoringPrefix(int commonPrefix, byte[] left, - int loffset, int llength, byte[] right, int roffset, int rlength) { - // Compare row - short lrowlength = Bytes.toShort(left, loffset); - short rrowlength; - - int comparisonResult = 0; - if (commonPrefix < ROW_LENGTH_SIZE) { - // almost nothing in common - rrowlength = Bytes.toShort(right, roffset); - comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE, - lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength); - } else { // the row length is the same - rrowlength = lrowlength; - if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) { - // The rows are not the same. Exclude the common prefix and compare - // the rest of the two rows. - int common = commonPrefix - ROW_LENGTH_SIZE; - comparisonResult = compareRows( - left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common, - right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common); - } - } - if (comparisonResult != 0) { - return comparisonResult; - } - - assert lrowlength == rrowlength; - - return compareWithoutRow(commonPrefix, left, loffset, llength, right, - roffset, rlength, lrowlength); - } - - /** - * Compare column, timestamp, and key type (everything except the row). - * This method is used both in the normal comparator and the "same-prefix" - * comparator. Note that we are assuming that row portions of both KVs have - * already been parsed and found identical, and we don't validate that - * assumption here. - * @param commonPrefix the length of the common prefix of the two - * key-values being compared, including row length and row - */ - private int compareWithoutRow(int commonPrefix, byte[] left, int loffset, - int llength, byte[] right, int roffset, int rlength, short rowlength) { - // Compare column family. Start comparing past row and family length. - int lcolumnoffset = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + - rowlength + loffset; - int rcolumnoffset = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + - rowlength + roffset; + // Compare column family. Start compare past row and family length. + int lcolumnoffset = Bytes.SIZEOF_SHORT + lrowlength + 1 + loffset; + int rcolumnoffset = Bytes.SIZEOF_SHORT + rrowlength + 1 + roffset; int lcolumnlength = llength - TIMESTAMP_TYPE_SIZE - - (lcolumnoffset - loffset); + (lcolumnoffset - loffset); int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE - - (rcolumnoffset - roffset); + (rcolumnoffset - roffset); - // If row matches, and no column in the 'left' AND put type is 'minimum', + // if row matches, and no column in the 'left' AND put type is 'minimum', // then return that left is larger than right. - // This supports 'last key on a row' - the magic is if there is no column - // in the left operand, and the left operand has a type of '0' - magical - // value, then we say the left is bigger. This will let us seek to the - // last key in a row. + // This supports 'last key on a row' - the magic is if there is no column in the + // left operand, and the left operand has a type of '0' - magical value, + // then we say the left is bigger. This will let us seek to the last key in + // a row. byte ltype = left[loffset + (llength - 1)]; byte rtype = right[roffset + (rlength - 1)]; - // If the column is not specified, the "minimum" key type appears the - // latest in the sorted order, regardless of the timestamp. This is used - // for specifying the last key/value in a given row, because there is no - // "lexicographically last column" (it would be infinitely long). The - // "maximum" key type does not need this behavior. if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) { - // left is "bigger", i.e. it appears later in the sorted order - return 1; + return 1; // left is bigger. } if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) { return -1; } - int common = 0; - if (commonPrefix > 0) { - common = Math.max(0, commonPrefix - - rowlength - ROW_LENGTH_SIZE - FAMILY_LENGTH_SIZE); - common = Math.min(common, Math.min(lcolumnlength, rcolumnlength)); + // TODO the family and qualifier should be compared separately + compare = Bytes.compareTo(left, lcolumnoffset, lcolumnlength, right, + rcolumnoffset, rcolumnlength); + if (compare != 0) { + return compare; } - final int comparisonResult = Bytes.compareTo( - left, lcolumnoffset + common, lcolumnlength - common, - right, rcolumnoffset + common, rcolumnlength - common); - if (comparisonResult != 0) { - return comparisonResult; - } - - return compareTimestampAndType(left, loffset, llength, right, roffset, - rlength, ltype, rtype); - } - - private int compareTimestampAndType(byte[] left, int loffset, int llength, - byte[] right, int roffset, int rlength, byte ltype, byte rtype) { - int compare; if (!this.ignoreTimestamp) { // Get timestamps. long ltimestamp = Bytes.toLong(left, @@ -2172,9 +2069,7 @@ public class KeyValue implements Writable, HeapSize { if (!this.ignoreType) { // Compare types. Let the delete types sort ahead of puts; i.e. types - // of higher numbers sort before those of lesser numbers. Maximum (255) - // appears ahead of everything, and minimum (0) appears after - // everything. + // of higher numbers sort before those of lesser numbers return (0xff & rtype) - (0xff & ltype); } return 0; diff --git a/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java index 6ae416e2ab6..23bbc2b33ce 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java +++ b/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java @@ -28,9 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; -import org.apache.hadoop.hbase.io.hfile.BlockCache; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.Bytes; @@ -63,9 +60,9 @@ public class HalfStoreFileReader extends StoreFile.Reader { * @throws IOException */ public HalfStoreFileReader(final FileSystem fs, final Path p, - final CacheConfig cacheConf, final Reference r, - HFileDataBlockEncoder dataBlockEncoder) throws IOException { - super(fs, p, cacheConf, dataBlockEncoder); + final CacheConfig cacheConf, final Reference r) + throws IOException { + super(fs, p, cacheConf); // This is not actual midkey for this half-file; its just border // around which we split top and bottom. Have to look in files to find // actual last and first keys for bottom and top halves. Half-files don't diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/BitsetKeyDeltaEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/BitsetKeyDeltaEncoder.java deleted file mode 100644 index 09638e47b20..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/BitsetKeyDeltaEncoder.java +++ /dev/null @@ -1,450 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; - -/** - * Compress using bit fields to avoid repetitions of certain fields. - * Also compress the value and key size. - * - * Format: - *
    - *
  • 1 byte: flag
  • - *
  • 1-4 bytes: key length
  • - *
  • 1-4 bytes: value length
  • - *
  • Key parts which are new (number of bytes varies)
  • - *
  • Value (number of bytes varies)
  • - *
- * - * In worst case compressed KeyValue will be one byte longer than original. - */ -public class BitsetKeyDeltaEncoder extends BufferedDataBlockEncoder { - - /* Constants used in flag byte */ - static final int SAME_ROW_FLAG = 1; - static final int SAME_FAMILY_FLAG = 1 << 1; - static final int SAME_QUALIFIER_FLAG = 1 << 2; - static final int SAME_TYPE_FLAG = 1 << 3; - static final int VALUE_SIZE_MASK = (1 << 4) | (1 << 5); - static final int VALUE_SIZE_SHIFT = 4; - static final int KEY_SIZE_MASK = (1 << 6) | (1 << 7); - static final int KEY_SIZE_SHIFT = 6; - - @Override - public void compressKeyValues(DataOutputStream out, - ByteBuffer in, boolean includesMemstoreTS) - throws IOException { - in.rewind(); - ByteBufferUtils.putInt(out, in.limit()); - CompressionState state = new CompressionState(); - while (in.hasRemaining()) { - compressSingleKeyValue(state, out, in); - afterEncodingKeyValue(in, out, includesMemstoreTS); - } - } - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream in, - int allocHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException { - int decompressedSize = in.readInt(); - ByteBuffer buffer = ByteBuffer.allocate(decompressedSize + - allocHeaderLength); - buffer.position(allocHeaderLength); - CompressionState state = new CompressionState(); - while (in.available() > skipLastBytes) { - uncompressSingleKeyValue(in, buffer, state); - afterDecodingKeyValue(in, buffer, includesMemstoreTS); - } - - if (in.available() != skipLastBytes) { - throw new IllegalStateException("Read too much bytes."); - } - - return buffer; - } - - private void uncompressSingleKeyValue(DataInputStream in, - ByteBuffer buffer, - CompressionState state) - throws IOException, EncoderBufferTooSmallException { - byte flag = in.readByte(); - - // Read key length - int keyLengthFitInBytes = 1 + ((flag & KEY_SIZE_MASK) >>> KEY_SIZE_SHIFT); - int keyLength = ByteBufferUtils.readCompressedInt(in, keyLengthFitInBytes); - - // Read value length - int valueLengthFitInBytes = 1 + - ((flag & VALUE_SIZE_MASK) >>> VALUE_SIZE_SHIFT); - int valueLength = - ByteBufferUtils.readCompressedInt(in, valueLengthFitInBytes); - - // Create buffer blob and put length and size there. - ByteBufferUtils.ensureSpace(buffer, keyLength + valueLength - + KeyValue.ROW_OFFSET); - buffer.putInt(keyLength); - buffer.putInt(valueLength); - int prevElementOffset = state.prevOffset + KeyValue.ROW_OFFSET; - int prevRowOffset = prevElementOffset; - prevElementOffset += state.rowLength + KeyValue.ROW_LENGTH_SIZE; - - // Read row - if (state.isFirst() || (flag & SAME_ROW_FLAG) == 0) { - state.rowLength = in.readShort(); - buffer.putShort(state.rowLength); - ByteBufferUtils.copyFromStream(in, buffer, state.rowLength); - } else { - ByteBufferUtils.copyFromBuffer(buffer, buffer, prevRowOffset, - state.rowLength + KeyValue.ROW_LENGTH_SIZE); - } - - - // Read family - int prevFamilyOffset = prevElementOffset; - prevElementOffset += state.familyLength + KeyValue.FAMILY_LENGTH_SIZE; - - if (state.isFirst() || (flag & SAME_FAMILY_FLAG) == 0) { - state.familyLength = in.readByte(); - buffer.put(state.familyLength); - ByteBufferUtils.copyFromStream(in, buffer, state.familyLength); - } else { - ByteBufferUtils.copyFromBuffer(buffer, buffer, prevFamilyOffset, - state.familyLength + KeyValue.FAMILY_LENGTH_SIZE); - } - - // Read qualifier - if (state.isFirst() || (flag & SAME_QUALIFIER_FLAG) == 0) { - state.qualifierLength = keyLength - state.rowLength - state.familyLength - - KeyValue.KEY_INFRASTRUCTURE_SIZE; - ByteBufferUtils.copyFromStream(in, buffer, state.qualifierLength); - } else { - ByteBufferUtils.copyFromBuffer(buffer, buffer, - prevElementOffset, state.qualifierLength); - } - - // Read timestamp - ByteBufferUtils.copyFromStream(in, buffer, KeyValue.TIMESTAMP_SIZE); - - // Read type - if (state.isFirst() || (flag & SAME_TYPE_FLAG) == 0) { - state.type = in.readByte(); - } - buffer.put(state.type); - - // Read value - state.prevOffset = buffer.position() - keyLength - KeyValue.ROW_OFFSET; - ByteBufferUtils.copyFromStream(in, buffer, valueLength); - } - - private void compressSingleKeyValue(CompressionState state, - OutputStream out, ByteBuffer in) throws IOException { - int kvPos = in.position(); - int keyLength = in.getInt(); - int valueLength = in.getInt(); - - byte flags = 0; - - // Key length - int keyLengthFitsInBytes = ByteBufferUtils.intFitsIn(keyLength); - flags |= (keyLengthFitsInBytes - 1) << KEY_SIZE_SHIFT; - - // Value length - int valueLengthFitsInBytes = ByteBufferUtils.intFitsIn(valueLength); - flags |= (valueLengthFitsInBytes - 1) << VALUE_SIZE_SHIFT; - - if (state.isFirst()) { - ByteBufferUtils.copyToStream(out, flags); - - writeKeyValueCompressedLengths(out, in, - keyLengthFitsInBytes, valueLengthFitsInBytes); - - state.readKey(in, keyLength, valueLength); - ByteBufferUtils.copyToStream(out, in, keyLength); - } else { - in.mark(); // beginning of the key - int prevElementOffset = state.prevOffset + KeyValue.ROW_OFFSET + - KeyValue.ROW_LENGTH_SIZE; - - // Row same - short rowLength = in.getShort(); - int prevRowOffset = prevElementOffset; - prevElementOffset += state.rowLength + KeyValue.FAMILY_LENGTH_SIZE; - - if (ByteBufferUtils.arePartsEqual(in, in.position(), rowLength, - prevRowOffset, state.rowLength)) { - flags |= SAME_ROW_FLAG; - } else { - state.rowLength = rowLength; - } - ByteBufferUtils.skip(in, rowLength); - - // Family same - byte familyLength = in.get(); - int prevFamilyOffset = prevElementOffset; - prevElementOffset += state.familyLength; - - if (ByteBufferUtils.arePartsEqual(in, - in.position(), familyLength, - prevFamilyOffset, state.familyLength)) { - flags |= SAME_FAMILY_FLAG; - } else { - state.familyLength = familyLength; - } - ByteBufferUtils.skip(in, familyLength); - - // Qualifier same - int qualifierLength = keyLength - rowLength - familyLength - - KeyValue.KEY_INFRASTRUCTURE_SIZE; - int prevQualifierOffset = prevElementOffset; - if (ByteBufferUtils.arePartsEqual(in, in.position(), qualifierLength, - prevQualifierOffset, state.qualifierLength)) { - flags |= SAME_QUALIFIER_FLAG; - } else { - state.qualifierLength = qualifierLength; - } - ByteBufferUtils.skip(in, qualifierLength + KeyValue.TIMESTAMP_SIZE); - - // Type same - byte type = in.get(); - if (type == state.type) { - flags |= SAME_TYPE_FLAG; - } else { - state.type = type; - } - - // write it - ByteBufferUtils.copyToStream(out, flags); - in.reset(); // return to beginning of the key - - writeKeyValueCompressedLengths(out, in, - keyLengthFitsInBytes, valueLengthFitsInBytes); - - if ((flags & SAME_ROW_FLAG) == 0) { - ByteBufferUtils.copyToStream(out, in, rowLength - + KeyValue.ROW_LENGTH_SIZE); - } else { - ByteBufferUtils.skip(in, rowLength + KeyValue.ROW_LENGTH_SIZE); - } - - if ((flags & SAME_FAMILY_FLAG) == 0) { - ByteBufferUtils.copyToStream(out, in, familyLength - + KeyValue.FAMILY_LENGTH_SIZE); - } else { - ByteBufferUtils.skip(in, familyLength + KeyValue.FAMILY_LENGTH_SIZE); - } - - if ((flags & SAME_QUALIFIER_FLAG) == 0) { - ByteBufferUtils.copyToStream(out, in, qualifierLength); - } else { - ByteBufferUtils.skip(in, qualifierLength); - } - - // Timestamp is always different - ByteBufferUtils.copyToStream(out, in, KeyValue.TIMESTAMP_SIZE); - - if ((flags & SAME_TYPE_FLAG) == 0) { - ByteBufferUtils.copyToStream(out, type); - } - ByteBufferUtils.skip(in, KeyValue.TYPE_SIZE); - } - - // Copy value - state.prevOffset = kvPos; - ByteBufferUtils.copyToStream(out, in, valueLength); - } - - private void writeKeyValueCompressedLengths(OutputStream out, - ByteBuffer in, int keyLengthFitsInBytes, - int valueLengthFitsInBytes) throws IOException { - int off = in.position() - KeyValue.ROW_OFFSET; - ByteBufferUtils.copyToStream(out, in, off + (4 - keyLengthFitsInBytes), - keyLengthFitsInBytes); - off += KeyValue.KEY_LENGTH_SIZE; - ByteBufferUtils.copyToStream(out, in, off + (4 - valueLengthFitsInBytes), - valueLengthFitsInBytes); - } - - @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { - block.mark(); - block.position(Bytes.SIZEOF_INT); - byte flag = block.get(); - int keyLength = ByteBufferUtils.readCompressedInt( - block, 1 + ((flag & KEY_SIZE_MASK) >>> KEY_SIZE_SHIFT)); - - // valueLength - ByteBufferUtils.readCompressedInt( - block, 1 + ((flag & VALUE_SIZE_MASK) >>> VALUE_SIZE_SHIFT)); - int pos = block.position(); - block.reset(); - return ByteBuffer.wrap(block.array(), pos, keyLength).slice(); - } - - @Override - public String toString() { - return BitsetKeyDeltaEncoder.class.getSimpleName(); - } - - @Override - public EncodedSeeker createSeeker(RawComparator comparator, - final boolean includesMemstoreTS) { - return new BufferedEncodedSeeker(comparator) { - private int familyLengthWithSize; - private int rowLengthWithSize; - - private void decode() { - byte type = 0; - - // Read key and value length - byte flag = currentBuffer.get(); - int oldKeyLength = current.keyLength; - int keyLengthFitInBytes = 1 + ((flag & KEY_SIZE_MASK) >>> KEY_SIZE_SHIFT); - current.keyLength = - ByteBufferUtils.readCompressedInt(currentBuffer, - keyLengthFitInBytes); - - // Read value length - int valueLengthFitInBytes = 1 + - ((flag & VALUE_SIZE_MASK) >>> VALUE_SIZE_SHIFT); - current.valueLength = - ByteBufferUtils.readCompressedInt(currentBuffer, - valueLengthFitInBytes); - - if (oldKeyLength != current.keyLength && (flag & SAME_TYPE_FLAG) != 0) { - type = current.keyBuffer[oldKeyLength -1]; - } - - current.lastCommonPrefix = 0; - switch (flag & - (SAME_ROW_FLAG | SAME_FAMILY_FLAG | SAME_QUALIFIER_FLAG)) { - case SAME_ROW_FLAG | SAME_FAMILY_FLAG | SAME_QUALIFIER_FLAG: - current.lastCommonPrefix = current.keyLength - - familyLengthWithSize - rowLengthWithSize - // will be added - KeyValue.TIMESTAMP_TYPE_SIZE; - //$FALL-THROUGH$ - case SAME_ROW_FLAG | SAME_FAMILY_FLAG: - current.lastCommonPrefix += - familyLengthWithSize + rowLengthWithSize; - //$FALL-THROUGH$ - case 0: // fall through - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.keyLength - current.lastCommonPrefix - - Bytes.SIZEOF_BYTE); - break; - - case SAME_FAMILY_FLAG: - //$FALL-THROUGH$ - case SAME_FAMILY_FLAG | SAME_QUALIFIER_FLAG: - - // find size of new row - currentBuffer.get(current.keyBuffer, 0, Bytes.SIZEOF_SHORT); - int oldRowLengthWithSize = rowLengthWithSize; - rowLengthWithSize = Bytes.toShort(current.keyBuffer) + - Bytes.SIZEOF_SHORT; - - // move the column family and qualifier - int moveLength; - if ((flag & SAME_QUALIFIER_FLAG) == 0) { - moveLength = familyLengthWithSize; - } else { - moveLength = current.keyLength - rowLengthWithSize - - KeyValue.TIMESTAMP_TYPE_SIZE; - } - System.arraycopy(current.keyBuffer, oldRowLengthWithSize, - current.keyBuffer, rowLengthWithSize, moveLength); - - // copy row - currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT, - rowLengthWithSize - Bytes.SIZEOF_SHORT); - - // copy qualifier and timestamp - if ((flag & SAME_QUALIFIER_FLAG) == 0) { - currentBuffer.get(current.keyBuffer, - rowLengthWithSize + familyLengthWithSize, - current.keyLength - rowLengthWithSize - - familyLengthWithSize - Bytes.SIZEOF_BYTE); - } else { - currentBuffer.get(current.keyBuffer, - current.keyLength - KeyValue.TIMESTAMP_TYPE_SIZE, - Bytes.SIZEOF_LONG); - } - break; - - case SAME_QUALIFIER_FLAG: - //$FALL-THROUGH$ - case SAME_QUALIFIER_FLAG | SAME_ROW_FLAG: - //$FALL-THROUGH$ - case SAME_ROW_FLAG: - //$FALL-THROUGH$ - default: - throw new RuntimeException("Unexpected flag!"); - } - - // we need to save length for the first key - if ((flag & SAME_ROW_FLAG) == 0) { - rowLengthWithSize = Bytes.toShort(current.keyBuffer) + - Bytes.SIZEOF_SHORT; - familyLengthWithSize = current.keyBuffer[rowLengthWithSize] + - Bytes.SIZEOF_BYTE; - } else if ((flag & SAME_FAMILY_FLAG) != 0) { - familyLengthWithSize = current.keyBuffer[rowLengthWithSize] + - Bytes.SIZEOF_BYTE; - } - - // type - if ((flag & SAME_TYPE_FLAG) == 0) { - currentBuffer.get(current.keyBuffer, - current.keyLength - Bytes.SIZEOF_BYTE, Bytes.SIZEOF_BYTE); - } else if (oldKeyLength != current.keyLength) { - current.keyBuffer[current.keyLength - Bytes.SIZEOF_BYTE] = type; - } - - // value - current.valueOffset = currentBuffer.position(); - ByteBufferUtils.skip(currentBuffer, current.valueLength); - - if (includesMemstoreTS) { - current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer); - } else { - current.memstoreTS = 0; - } - } - - @Override - protected void decodeNext() { - decode(); - } - - @Override - protected void decodeFirst() { - ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT); - decode(); - } - }; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java deleted file mode 100644 index 94b64a3b364..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ /dev/null @@ -1,289 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInput; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; -import org.apache.hadoop.io.WritableUtils; - -/** - * Base class for all data block encoders that use a buffer. - */ -abstract class BufferedDataBlockEncoder implements DataBlockEncoder { - - private static int INITIAL_KEY_BUFFER_SIZE = 512; - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream source, - boolean includesMemstoreTS) throws IOException { - return uncompressKeyValues(source, 0, 0, includesMemstoreTS); - } - - protected static class SeekerState { - protected int valueOffset = -1; - protected int keyLength; - protected int valueLength; - protected int lastCommonPrefix; - - /** We need to store a copy of the key. */ - protected byte[] keyBuffer = new byte[INITIAL_KEY_BUFFER_SIZE]; - - protected long memstoreTS; - protected int nextKvOffset; - - protected boolean isValid() { - return valueOffset != -1; - } - - protected void invalidate() { - valueOffset = -1; - } - - protected void ensureSpaceForKey() { - if (keyLength > keyBuffer.length) { - // rare case, but we need to handle arbitrary length of key - int newKeyBufferLength = Math.max(keyBuffer.length, 1) * 2; - while (keyLength > newKeyBufferLength) { - newKeyBufferLength *= 2; - } - byte[] newKeyBuffer = new byte[newKeyBufferLength]; - System.arraycopy(keyBuffer, 0, newKeyBuffer, 0, keyBuffer.length); - keyBuffer = newKeyBuffer; - } - } - } - - protected abstract static class BufferedEncodedSeeker - implements EncodedSeeker { - - protected final RawComparator comparator; - protected final SamePrefixComparator samePrefixComparator; - protected ByteBuffer currentBuffer; - protected SeekerState current = new SeekerState(); // always valid - protected SeekerState previous = new SeekerState(); // may not be valid - - @SuppressWarnings("unchecked") - public BufferedEncodedSeeker(RawComparator comparator) { - this.comparator = comparator; - if (comparator instanceof SamePrefixComparator) { - this.samePrefixComparator = (SamePrefixComparator) comparator; - } else { - this.samePrefixComparator = null; - } - } - - @Override - public void setCurrentBuffer(ByteBuffer buffer) { - currentBuffer = buffer; - decodeFirst(); - previous.invalidate(); - } - - @Override - public ByteBuffer getKey() { - ByteBuffer keyBuffer = ByteBuffer.allocate(current.keyLength); - keyBuffer.put(current.keyBuffer, 0, current.keyLength); - return keyBuffer; - } - - @Override - public ByteBuffer getValue() { - return ByteBuffer.wrap(currentBuffer.array(), - currentBuffer.arrayOffset() + current.valueOffset, - current.valueLength); - } - - @Override - public ByteBuffer getKeyValue() { - ByteBuffer kvBuffer = ByteBuffer.allocate( - 2 * Bytes.SIZEOF_INT + current.keyLength + current.valueLength); - kvBuffer.putInt(current.keyLength); - kvBuffer.putInt(current.valueLength); - kvBuffer.put(current.keyBuffer, 0, current.keyLength); - kvBuffer.put(currentBuffer.array(), - currentBuffer.arrayOffset() + current.valueOffset, - current.valueLength); - return kvBuffer; - } - - @Override - public KeyValue getKeyValueObject() { - ByteBuffer kvBuf = getKeyValue(); - KeyValue kv = new KeyValue(kvBuf.array(), kvBuf.arrayOffset()); - kv.setMemstoreTS(current.memstoreTS); - return kv; - } - - @Override - public void rewind() { - currentBuffer.rewind(); - decodeFirst(); - previous.invalidate(); - } - - @Override - public boolean next() { - if (!currentBuffer.hasRemaining()) { - return false; - } - decodeNext(); - previous.invalidate(); - return true; - } - - @Override - public int blockSeekTo(byte[] key, int offset, int length, - boolean seekBefore) { - int commonPrefix = 0; - previous.invalidate(); - do { - int comp; - if (samePrefixComparator != null) { - commonPrefix = Math.min(commonPrefix, current.lastCommonPrefix); - - // extend commonPrefix - commonPrefix += ByteBufferUtils.findCommonPrefix( - key, offset + commonPrefix, length - commonPrefix, - current.keyBuffer, commonPrefix, - current.keyLength - commonPrefix); - - comp = samePrefixComparator.compareIgnoringPrefix( - commonPrefix, - key, offset, length, current.keyBuffer, 0, current.keyLength); - } else { - comp = comparator.compare(key, offset, length, - current.keyBuffer, 0, current.keyLength); - } - - if (comp == 0) { // exact match - if (seekBefore) { - moveToPrevious(); - return 1; - } - return 0; - } - - if (comp < 0) { // already too large, check previous - if (previous.isValid()) { - moveToPrevious(); - } - return 1; - } - - // move to next, if more data is available - if (currentBuffer.hasRemaining()) { - savePrevious(); - decodeNext(); - } else { - break; - } - } while (true); - - // we hit end of file, not exact match - return 1; - } - - private void moveToPrevious() { - if (!previous.isValid()) { - throw new IllegalStateException( - "Can move back only once and not in first key in the block."); - } - - SeekerState tmp = previous; - previous = current; - current = tmp; - - // move after last key value - currentBuffer.position(current.nextKvOffset); - - previous.invalidate(); - } - - private void savePrevious() { - previous.valueOffset = current.valueOffset; - previous.keyLength = current.keyLength; - previous.valueLength = current.valueLength; - previous.lastCommonPrefix = current.lastCommonPrefix; - previous.nextKvOffset = current.nextKvOffset; - if (previous.keyBuffer.length != current.keyBuffer.length) { - previous.keyBuffer = current.keyBuffer.clone(); - } else if (!previous.isValid()) { - System.arraycopy(current.keyBuffer, 0, previous.keyBuffer, 0, - current.keyLength); - } else { - // don't copy the common prefix between this key and the previous one - System.arraycopy(current.keyBuffer, current.lastCommonPrefix, - previous.keyBuffer, current.lastCommonPrefix, current.keyLength - - current.lastCommonPrefix); - } - } - - abstract protected void decodeFirst(); - abstract protected void decodeNext(); - } - - protected final void afterEncodingKeyValue(ByteBuffer in, - DataOutputStream out, boolean includesMemstoreTS) { - if (includesMemstoreTS) { - // Copy memstore timestamp from the byte buffer to the output stream. - long memstoreTS = -1; - try { - memstoreTS = ByteBufferUtils.readVLong(in); - WritableUtils.writeVLong(out, memstoreTS); - } catch (IOException ex) { - throw new RuntimeException("Unable to copy memstore timestamp " + - memstoreTS + " after encoding a key/value"); - } - } - } - - static DataInput asDataInputStream(final ByteBuffer in) { - return new DataInputStream(new InputStream() { - @Override - public int read() throws IOException { - return in.get() & 0xff; - } - }); - } - - protected final void afterDecodingKeyValue(DataInputStream source, - ByteBuffer dest, boolean includesMemstoreTS) { - if (includesMemstoreTS) { - long memstoreTS = -1; - try { - // Copy memstore timestamp from the data input stream to the byte - // buffer. - memstoreTS = WritableUtils.readVLong(source); - ByteBufferUtils.writeVLong(dest, memstoreTS); - } catch (IOException ex) { - throw new RuntimeException("Unable to copy memstore timestamp " + - memstoreTS + " after decoding a key/value"); - } - } - } - -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/CompressionState.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/CompressionState.java deleted file mode 100644 index 47d6d4967db..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/CompressionState.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; - -/** - * Stores the state of data block encoder at the beginning of new key. - */ -class CompressionState { - int keyLength; - int valueLength; - - short rowLength; - int prevOffset = FIRST_KEY; - byte familyLength; - int qualifierLength; - byte type; - - final static int FIRST_KEY = -1; - - boolean isFirst() { - return prevOffset == FIRST_KEY; - } - - /** - * Analyze the key and fill the state. - * Uses mark() and reset() in ByteBuffer. - * @param in Buffer at the position where key starts - * @param keyLength Length of key in bytes - * @param valueLength Length of values in bytes - */ - void readKey(ByteBuffer in, int keyLength, int valueLength) { - readKey(in, keyLength, valueLength, 0, null); - } - - /** - * Analyze the key and fill the state assuming we know previous state. - * Uses mark() and reset() in ByteBuffer. - * @param in Buffer at the position where key starts - * @param keyLength Length of key in bytes - * @param valueLength Length of values in bytes - * @param commonPrefix how many first bytes are common with previous KeyValue - * @param previousState State from previous KeyValue - */ - void readKey(ByteBuffer in, int keyLength, int valueLength, - int commonPrefix, CompressionState previousState) { - this.keyLength = keyLength; - this.valueLength = valueLength; - - // fill the state - in.mark(); // mark beginning of key - - if (commonPrefix < KeyValue.ROW_LENGTH_SIZE) { - rowLength = in.getShort(); - ByteBufferUtils.skip(in, rowLength); - - familyLength = in.get(); - - qualifierLength = keyLength - rowLength - familyLength - - KeyValue.KEY_INFRASTRUCTURE_SIZE; - ByteBufferUtils.skip(in, familyLength + qualifierLength); - } else { - rowLength = previousState.rowLength; - familyLength = previousState.familyLength; - qualifierLength = previousState.qualifierLength + - keyLength - previousState.keyLength; - ByteBufferUtils.skip(in, (KeyValue.ROW_LENGTH_SIZE + - KeyValue.FAMILY_LENGTH_SIZE) + - rowLength + familyLength + qualifierLength); - } - - readTimestamp(in); - - type = in.get(); - - in.reset(); - } - - protected void readTimestamp(ByteBuffer in) { - // used in subclasses to add timestamp to state - ByteBufferUtils.skip(in, KeyValue.TIMESTAMP_SIZE); - } - - void copyFrom(CompressionState state) { - keyLength = state.keyLength; - valueLength = state.valueLength; - - rowLength = state.rowLength; - prevOffset = state.prevOffset; - familyLength = state.familyLength; - qualifierLength = state.qualifierLength; - type = state.type; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java deleted file mode 100644 index 7b8a408667e..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; - -/** - * Just copy data, do not do any kind of compression. Use for comparison and - * benchmarking. - */ -public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder { - @Override - public void compressKeyValues(DataOutputStream out, - ByteBuffer in, boolean includesMemstoreTS) throws IOException { - in.rewind(); - ByteBufferUtils.putInt(out, in.limit()); - ByteBufferUtils.copyToStream(out, in, in.limit()); - } - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream source, - int preserveHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException { - int decompressedSize = source.readInt(); - ByteBuffer buffer = ByteBuffer.allocate(decompressedSize + - preserveHeaderLength); - buffer.position(preserveHeaderLength); - ByteBufferUtils.copyFromStream(source, buffer, decompressedSize); - - return buffer; - } - - @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { - int keyLength = block.getInt(Bytes.SIZEOF_INT); - return ByteBuffer.wrap(block.array(), - block.arrayOffset() + 3 * Bytes.SIZEOF_INT, keyLength).slice(); - } - - - @Override - public String toString() { - return CopyKeyDataBlockEncoder.class.getSimpleName(); - } - - @Override - public EncodedSeeker createSeeker(RawComparator comparator, - final boolean includesMemstoreTS) { - return new BufferedEncodedSeeker(comparator) { - @Override - protected void decodeNext() { - current.keyLength = currentBuffer.getInt(); - current.valueLength = currentBuffer.getInt(); - current.ensureSpaceForKey(); - currentBuffer.get(current.keyBuffer, 0, current.keyLength); - current.valueOffset = currentBuffer.position(); - ByteBufferUtils.skip(currentBuffer, current.valueLength); - if (includesMemstoreTS) { - current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer); - } else { - current.memstoreTS = 0; - } - current.nextKvOffset = currentBuffer.position(); - } - - @Override - protected void decodeFirst() { - ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT); - current.lastCommonPrefix = 0; - decodeNext(); - } - }; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java deleted file mode 100644 index 347d2088b89..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.io.RawComparator; - -/** - * Encoding of KeyValue. It aims to be fast and efficient using assumptions: - *
    - *
  • the KeyValue are stored sorted by key
  • - *
  • we know the structure of KeyValue
  • - *
  • the values are iterated always forward from beginning of block
  • - *
  • knowledge of Key Value format
  • - *
- * It is designed to work fast enough to be feasible as in memory compression. - */ -public interface DataBlockEncoder { - /** - * Compress KeyValues and write them to output buffer. - * @param out Where to write compressed data. - * @param in Source of KeyValue for compression. - * @param includesMemstoreTS true if including memstore timestamp after every - * key-value pair - * @throws IOException If there is an error writing to output stream. - */ - public void compressKeyValues(DataOutputStream out, - ByteBuffer in, boolean includesMemstoreTS) throws IOException; - - /** - * Uncompress. - * @param source Compressed stream of KeyValues. - * @param includesMemstoreTS true if including memstore timestamp after every - * key-value pair - * @return Uncompressed block of KeyValues. - * @throws IOException If there is an error in source. - */ - public ByteBuffer uncompressKeyValues(DataInputStream source, - boolean includesMemstoreTS) throws IOException; - - /** - * Uncompress. - * @param source Compressed stream of KeyValues. - * @param allocateHeaderLength allocate this many bytes for the header. - * @param skipLastBytes Do not copy n last bytes. - * @param includesMemstoreTS true if including memstore timestamp after every - * key-value pair - * @return Uncompressed block of KeyValues. - * @throws IOException If there is an error in source. - */ - public ByteBuffer uncompressKeyValues(DataInputStream source, - int allocateHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException; - - /** - * Return first key in block. Useful for indexing. - * @param block encoded block we want index, the position will not change - * @return First key in block. - */ - public ByteBuffer getFirstKeyInBlock(ByteBuffer block); - - /** - * Create a HFileBlock seeker which find KeyValues within a block. - * @param comparator what kind of comparison should be used - * @param includesMemstoreTS true if including memstore timestamp after every - * key-value pair - * @return A newly created seeker. - */ - public EncodedSeeker createSeeker(RawComparator comparator, - boolean includesMemstoreTS); - - /** - * An interface which enable to seek while underlying data is encoded. - * - * It works on one HFileBlock, but it is reusable. See - * {@link #setCurrentBuffer(ByteBuffer)}. - */ - public static interface EncodedSeeker { - /** - * Set on which buffer there will be done seeking. - * @param buffer Used for seeking. - */ - public void setCurrentBuffer(ByteBuffer buffer); - - /** @return key at current position */ - public ByteBuffer getKey(); - - /** @return value at current position */ - public ByteBuffer getValue(); - - /** @return key value at current position. */ - public ByteBuffer getKeyValue(); - - /** - * @return the KeyValue object at the current position. Includes memstore - * timestamp. - */ - public KeyValue getKeyValueObject(); - - /** Set position to beginning of given block */ - public void rewind(); - - /** - * Move to next position - * @return true on success, false if there is no more positions. - */ - public boolean next(); - - /** - * Move position to the same key (or one before it). - * @param key Array where is the key. - * @param offset Key position in array. - * @param length Key length in array. - * @param seekBefore find the key before in case of exact match. Does not - * matter in case of an inexact match. - * @return 0 on exact match, 1 on inexact match. - */ - public int blockSeekTo(byte[] key, int offset, int length, - boolean seekBefore); - } -} \ No newline at end of file diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncodings.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncodings.java deleted file mode 100644 index 40c2c5a6b77..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncodings.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.IOException; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hbase.io.hfile.HFileBlock; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Provide access to all data block encoding algorithms. - */ -public class DataBlockEncodings { - - /** Constructor. This class cannot be instantiated. */ - private DataBlockEncodings() { - } - - /** - * Algorithm type. All of the algorithms are required to have unique id which - * should _NEVER_ be changed. If you want to add a new algorithm/version, - * assign it a new id. Announce the new id in the HBase mailing list to - * prevent collisions. - */ - public static enum Algorithm { - /** - * Disable data block encoding. - */ - NONE(0, null), - BITSET(1, new BitsetKeyDeltaEncoder()), - PREFIX(2, new PrefixKeyDeltaEncoder()), - DIFF(3, new DiffKeyDeltaEncoder()), - FAST_DIFF(4, new FastDiffDeltaEncoder()); - - private final short id; - private final byte[] idInBytes; - private final DataBlockEncoder encoder; - - private Algorithm(int id, DataBlockEncoder encoder) { - if (id < Short.MIN_VALUE || id > Short.MAX_VALUE) { - throw new AssertionError( - "Data block encoding algorithm id is out of range: " + id); - } - this.id = (short) id; - this.idInBytes = Bytes.toBytes(this.id); - if (idInBytes.length != HFileBlock.DATA_BLOCK_ENCODER_ID_SIZE) { - // White this may seem redundant, if we accidentally serialize - // the id as e.g. an int instead of a short, all encoders will break. - throw new RuntimeException("Unexpected length of encoder ID byte " + - "representation: " + Bytes.toStringBinary(idInBytes)); - } - this.encoder = encoder; - } - - /** - * @return Name converted to bytes. - */ - public byte[] getNameInBytes() { - return Bytes.toBytes(toString()); - } - - /** - * @return The id of a data block encoder. - */ - public short getId() { - return id; - } - - /** - * Writes id in bytes. - * @param stream Where it should be written. - */ - public void writeIdInBytes(OutputStream stream) throws IOException { - stream.write(idInBytes); - } - - /** - * Return new data block encoder for given algorithm type. - * @return data block encoder if algorithm is specified, null if none is - * selected. - */ - public DataBlockEncoder getEncoder() { - return encoder; - } - } - - /** - * Maps encoding algorithm ids to algorithm instances for all algorithms in - * the {@link Algorithm} enum. - */ - private static final Map idToAlgorithm = - new HashMap(); - - /** Size of a delta encoding algorithm id */ - public static final int ID_SIZE = Bytes.SIZEOF_SHORT; - - static { - for (Algorithm algo : Algorithm.values()) { - if (idToAlgorithm.containsKey(algo.getId())) { - throw new RuntimeException(String.format( - "Two data block encoder algorithms '%s' and '%s' has same id '%d", - idToAlgorithm.get(algo.getId()).toString(), algo.toString(), - (int) algo.getId())); - } - idToAlgorithm.put(algo.getId(), algo); - } - } - - /** - * Provide access to all data block encoders, even those which are not - * exposed in the enum. Useful for testing and benchmarking. - * @return list of all data block encoders. - */ - public static List getAllEncoders() { - ArrayList encoders = new ArrayList(); - for (Algorithm algo : Algorithm.values()) { - DataBlockEncoder encoder = algo.getEncoder(); - if (encoder != null) { - encoders.add(encoder); - } - } - - // Add encoders that are only used in testing. - encoders.add(new CopyKeyDataBlockEncoder()); - return encoders; - } - - /** - * Find and create data block encoder for given id; - * @param encoderId id of data block encoder. - * @return Newly created data block encoder. - */ - public static DataBlockEncoder getDataBlockEncoderFromId(short encoderId) { - if (!idToAlgorithm.containsKey(encoderId)) { - throw new IllegalArgumentException(String.format( - "There is no data block encoder for given id '%d'", - (int) encoderId)); - } - - return idToAlgorithm.get(encoderId).getEncoder(); - } - - /** - * Find and return name of data block encoder for given id. - * @param encoderId id of data block encoder - * @return name, same as used in options in column family - */ - public static String getNameFromId(short encoderId) { - return idToAlgorithm.get(encoderId).toString(); - } - - /** - * Check if given encoder has this id. - * @param encoder encoder which id will be checked - * @param encoderId id which we except - * @return true if id is right for given encoder, false otherwise - * @exception IllegalArgumentException - * thrown when there is no matching data block encoder - */ - public static boolean isCorrectEncoder(DataBlockEncoder encoder, - short encoderId) { - if (!idToAlgorithm.containsKey(encoderId)) { - throw new IllegalArgumentException(String.format( - "There is no data block encoder for given id '%d'", - (int) encoderId)); - } - - Algorithm algorithm = idToAlgorithm.get(encoderId); - return algorithm.getClass().equals(encoder.getClass()); - } - -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java deleted file mode 100644 index fadcfb6913a..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java +++ /dev/null @@ -1,533 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; - -/** - * Compress using: - * - store size of common prefix - * - save column family once, it is same within HFile - * - use integer compression for key, value and prefix (7-bit encoding) - * - use bits to avoid duplication key length, value length - * and type if it same as previous - * - store in 3 bits length of timestamp field - * - allow diff in timestamp instead of actual value - * - * Format: - * - 1 byte: flag - * - 1-5 bytes: key length (only if FLAG_SAME_KEY_LENGTH is not set in flag) - * - 1-5 bytes: value length (only if FLAG_SAME_VALUE_LENGTH is not set in flag) - * - 1-5 bytes: prefix length - * - ... bytes: rest of the row (if prefix length is small enough) - * - ... bytes: qualifier (or suffix depending on prefix length) - * - 1-8 bytes: timestamp or diff - * - 1 byte: type (only if FLAG_SAME_TYPE is not set in the flag) - * - ... bytes: value - */ -public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { - static final int FLAG_SAME_KEY_LENGTH = 1; - static final int FLAG_SAME_VALUE_LENGTH = 1 << 1; - static final int FLAG_SAME_TYPE = 1 << 2; - static final int FLAG_TIMESTAMP_IS_DIFF = 1 << 3; - static final int MASK_TIMESTAMP_LENGTH = (1 << 4) | (1 << 5) | (1 << 6); - static final int SHIFT_TIMESTAMP_LENGTH = 4; - static final int FLAG_TIMESTAMP_SIGN = 1 << 7; - - protected static class DiffCompressionState extends CompressionState { - long timestamp; - byte[] familyNameWithSize; - - @Override - protected void readTimestamp(ByteBuffer in) { - timestamp = in.getLong(); - } - - @Override - void copyFrom(CompressionState state) { - super.copyFrom(state); - DiffCompressionState state2 = (DiffCompressionState) state; - timestamp = state2.timestamp; - } - } - - private void compressSingleKeyValue(DiffCompressionState previousState, - DiffCompressionState currentState, DataOutputStream out, - ByteBuffer in) throws IOException { - byte flag = 0; - int kvPos = in.position(); - int keyLength = in.getInt(); - int valueLength = in.getInt(); - - long timestamp; - long diffTimestamp = 0; - int diffTimestampFitsInBytes = 0; - - int commonPrefix; - - int timestampFitsInBytes; - - if (previousState.isFirst()) { - currentState.readKey(in, keyLength, valueLength); - currentState.prevOffset = kvPos; - timestamp = currentState.timestamp; - if (timestamp < 0) { - flag |= FLAG_TIMESTAMP_SIGN; - timestamp = -timestamp; - } - timestampFitsInBytes = ByteBufferUtils.longFitsIn(timestamp); - - flag |= (timestampFitsInBytes - 1) << SHIFT_TIMESTAMP_LENGTH; - commonPrefix = 0; - - // put column family - in.mark(); - ByteBufferUtils.skip(in, currentState.rowLength - + KeyValue.ROW_LENGTH_SIZE); - ByteBufferUtils.copyToStream(out, in, currentState.familyLength - + KeyValue.FAMILY_LENGTH_SIZE); - in.reset(); - } else { - // find a common prefix and skip it - commonPrefix = - ByteBufferUtils.findCommonPrefix(in, in.position(), - previousState.prevOffset + KeyValue.ROW_OFFSET, keyLength - - KeyValue.TIMESTAMP_TYPE_SIZE); - // don't compress timestamp and type using prefix - - currentState.readKey(in, keyLength, valueLength, - commonPrefix, previousState); - currentState.prevOffset = kvPos; - timestamp = currentState.timestamp; - boolean minusTimestamp = timestamp < 0; - if (minusTimestamp) { - timestamp = -timestamp; - } - timestampFitsInBytes = ByteBufferUtils.longFitsIn(timestamp); - - if (keyLength == previousState.keyLength) { - flag |= FLAG_SAME_KEY_LENGTH; - } - if (valueLength == previousState.valueLength) { - flag |= FLAG_SAME_VALUE_LENGTH; - } - if (currentState.type == previousState.type) { - flag |= FLAG_SAME_TYPE; - } - - // encode timestamp - diffTimestamp = previousState.timestamp - currentState.timestamp; - boolean minusDiffTimestamp = diffTimestamp < 0; - if (minusDiffTimestamp) { - diffTimestamp = -diffTimestamp; - } - diffTimestampFitsInBytes = ByteBufferUtils.longFitsIn(diffTimestamp); - if (diffTimestampFitsInBytes < timestampFitsInBytes) { - flag |= (diffTimestampFitsInBytes - 1) << SHIFT_TIMESTAMP_LENGTH; - flag |= FLAG_TIMESTAMP_IS_DIFF; - if (minusDiffTimestamp) { - flag |= FLAG_TIMESTAMP_SIGN; - } - } else { - flag |= (timestampFitsInBytes - 1) << SHIFT_TIMESTAMP_LENGTH; - if (minusTimestamp) { - flag |= FLAG_TIMESTAMP_SIGN; - } - } - } - - ByteBufferUtils.copyToStream(out, flag); - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - ByteBufferUtils.putCompressedInt(out, keyLength); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - ByteBufferUtils.putCompressedInt(out, valueLength); - } - - ByteBufferUtils.putCompressedInt(out, commonPrefix); - ByteBufferUtils.skip(in, commonPrefix); - - if (previousState.isFirst() || - commonPrefix < currentState.rowLength + KeyValue.ROW_LENGTH_SIZE) { - int restRowLength = - currentState.rowLength + KeyValue.ROW_LENGTH_SIZE - commonPrefix; - ByteBufferUtils.copyToStream(out, in, restRowLength); - ByteBufferUtils.skip(in, currentState.familyLength + - KeyValue.FAMILY_LENGTH_SIZE); - ByteBufferUtils.copyToStream(out, in, currentState.qualifierLength); - } else { - ByteBufferUtils.copyToStream(out, in, - keyLength - commonPrefix - KeyValue.TIMESTAMP_TYPE_SIZE); - } - - if ((flag & FLAG_TIMESTAMP_IS_DIFF) == 0) { - ByteBufferUtils.putLong(out, timestamp, timestampFitsInBytes); - } else { - ByteBufferUtils.putLong(out, diffTimestamp, diffTimestampFitsInBytes); - } - - if ((flag & FLAG_SAME_TYPE) == 0) { - ByteBufferUtils.copyToStream(out, currentState.type); - } - ByteBufferUtils.skip(in, KeyValue.TIMESTAMP_TYPE_SIZE); - - ByteBufferUtils.copyToStream(out, in, valueLength); - } - - private void uncompressSingleKeyValue(DataInputStream source, - ByteBuffer buffer, - DiffCompressionState state) - throws IOException, EncoderBufferTooSmallException { - // read the column family at the beginning - if (state.isFirst()) { - state.familyLength = source.readByte(); - state.familyNameWithSize = - new byte[(state.familyLength & 0xff) + KeyValue.FAMILY_LENGTH_SIZE]; - state.familyNameWithSize[0] = state.familyLength; - source.read(state.familyNameWithSize, KeyValue.FAMILY_LENGTH_SIZE, - state.familyLength); - } - - // read flag - byte flag = source.readByte(); - - // read key/value/common lengths - int keyLength; - int valueLength; - if ((flag & FLAG_SAME_KEY_LENGTH) != 0) { - keyLength = state.keyLength; - } else { - keyLength = ByteBufferUtils.readCompressedInt(source); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) != 0) { - valueLength = state.valueLength; - } else { - valueLength = ByteBufferUtils.readCompressedInt(source); - } - int commonPrefix = ByteBufferUtils.readCompressedInt(source); - - // create KeyValue buffer and fill it prefix - int keyOffset = buffer.position(); - ByteBufferUtils.ensureSpace(buffer, keyLength + valueLength - + KeyValue.ROW_OFFSET); - buffer.putInt(keyLength); - buffer.putInt(valueLength); - - // copy common from previous key - if (commonPrefix > 0) { - ByteBufferUtils.copyFromBuffer(buffer, buffer, state.prevOffset - + KeyValue.ROW_OFFSET, commonPrefix); - } - - // copy the rest of the key from the buffer - int keyRestLength; - if (state.isFirst() || commonPrefix < - state.rowLength + KeyValue.ROW_LENGTH_SIZE) { - // omit the family part of the key, it is always the same - short rowLength; - int rowRestLength; - - // check length of row - if (commonPrefix < KeyValue.ROW_LENGTH_SIZE) { - // not yet copied, do it now - ByteBufferUtils.copyFromStream(source, buffer, - KeyValue.ROW_LENGTH_SIZE - commonPrefix); - ByteBufferUtils.skip(buffer, -KeyValue.ROW_LENGTH_SIZE); - rowLength = buffer.getShort(); - rowRestLength = rowLength; - } else { - // already in buffer, just read it - rowLength = buffer.getShort(keyOffset + KeyValue.ROW_OFFSET); - rowRestLength = rowLength + KeyValue.ROW_LENGTH_SIZE - commonPrefix; - } - - // copy the rest of row - ByteBufferUtils.copyFromStream(source, buffer, rowRestLength); - state.rowLength = rowLength; - - // copy the column family - buffer.put(state.familyNameWithSize); - - keyRestLength = keyLength - rowLength - - state.familyNameWithSize.length - - (KeyValue.ROW_LENGTH_SIZE + KeyValue.TIMESTAMP_TYPE_SIZE); - } else { - // prevRowWithSizeLength is the same as on previous row - keyRestLength = keyLength - commonPrefix - KeyValue.TIMESTAMP_TYPE_SIZE; - } - // copy the rest of the key, after column family -> column qualifier - ByteBufferUtils.copyFromStream(source, buffer, keyRestLength); - - // handle timestamp - int timestampFitsInBytes = - ((flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH) + 1; - long timestamp = ByteBufferUtils.readLong(source, timestampFitsInBytes); - if ((flag & FLAG_TIMESTAMP_SIGN) != 0) { - timestamp = -timestamp; - } - if ((flag & FLAG_TIMESTAMP_IS_DIFF) != 0) { - timestamp = state.timestamp - timestamp; - } - buffer.putLong(timestamp); - - // copy the type field - byte type; - if ((flag & FLAG_SAME_TYPE) != 0) { - type = state.type; - } else { - type = source.readByte(); - } - buffer.put(type); - - // copy value part - ByteBufferUtils.copyFromStream(source, buffer, valueLength); - - state.keyLength = keyLength; - state.valueLength = valueLength; - state.prevOffset = keyOffset; - state.timestamp = timestamp; - state.type = type; - // state.qualifier is unused - } - - @Override - public void compressKeyValues(DataOutputStream out, - ByteBuffer in, boolean includesMemstoreTS) throws IOException { - in.rewind(); - ByteBufferUtils.putInt(out, in.limit()); - DiffCompressionState previousState = new DiffCompressionState(); - DiffCompressionState currentState = new DiffCompressionState(); - while (in.hasRemaining()) { - compressSingleKeyValue(previousState, currentState, - out, in); - afterEncodingKeyValue(in, out, includesMemstoreTS); - - // swap previousState <-> currentState - DiffCompressionState tmp = previousState; - previousState = currentState; - currentState = tmp; - } - } - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream source, - int allocHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException { - int decompressedSize = source.readInt(); - ByteBuffer buffer = ByteBuffer.allocate(decompressedSize + - allocHeaderLength); - buffer.position(allocHeaderLength); - DiffCompressionState state = new DiffCompressionState(); - while (source.available() > skipLastBytes) { - uncompressSingleKeyValue(source, buffer, state); - afterDecodingKeyValue(source, buffer, includesMemstoreTS); - } - - if (source.available() != skipLastBytes) { - throw new IllegalStateException("Read too much bytes."); - } - - return buffer; - } - - @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { - block.mark(); - block.position(Bytes.SIZEOF_INT); - byte familyLength = block.get(); - ByteBufferUtils.skip(block, familyLength); - byte flag = block.get(); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); // valueLength - ByteBufferUtils.readCompressedInt(block); // commonLength - ByteBuffer result = ByteBuffer.allocate(keyLength); - - // copy row - int pos = result.arrayOffset(); - block.get(result.array(), pos, Bytes.SIZEOF_SHORT); - pos += Bytes.SIZEOF_SHORT; - short rowLength = result.getShort(); - block.get(result.array(), pos, rowLength); - pos += rowLength; - - // copy family - int savePosition = block.position(); - block.position(Bytes.SIZEOF_INT); - block.get(result.array(), pos, familyLength + Bytes.SIZEOF_BYTE); - pos += familyLength + Bytes.SIZEOF_BYTE; - - // copy qualifier - block.position(savePosition); - int qualifierLength = - keyLength - pos + result.arrayOffset() - KeyValue.TIMESTAMP_TYPE_SIZE; - block.get(result.array(), pos, qualifierLength); - pos += qualifierLength; - - // copy the timestamp and type - int timestampFitInBytes = - ((flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH) + 1; - long timestamp = ByteBufferUtils.readLong(block, timestampFitInBytes); - if ((flag & FLAG_TIMESTAMP_SIGN) != 0) { - timestamp = -timestamp; - } - result.putLong(pos, timestamp); - pos += Bytes.SIZEOF_LONG; - block.get(result.array(), pos, Bytes.SIZEOF_BYTE); - - block.reset(); - return result; - } - - @Override - public String toString() { - return DiffKeyDeltaEncoder.class.getSimpleName(); - } - - @Override - public EncodedSeeker createSeeker(RawComparator comparator, - final boolean includesMemstoreTS) { - return new BufferedEncodedSeeker(comparator) { - private static final int TIMESTAMP_WITH_TYPE_LENGTH = - Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE; - private byte[] familyNameWithSize; - private int rowLengthWithSize; - private long timestamp; - - private void decode(boolean isFirst) { - byte flag = currentBuffer.get(); - byte type = 0; - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - if (!isFirst) { - type = current.keyBuffer[current.keyLength - Bytes.SIZEOF_BYTE]; - } - current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - current.valueLength = - ByteBufferUtils.readCompressedInt(currentBuffer); - } - current.lastCommonPrefix = - ByteBufferUtils.readCompressedInt(currentBuffer); - - current.ensureSpaceForKey(); - - if (current.lastCommonPrefix < Bytes.SIZEOF_SHORT) { - // length of row is different, copy everything except family - - // copy the row size - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - Bytes.SIZEOF_SHORT - current.lastCommonPrefix); - rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - - // copy the rest of row - currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT, - rowLengthWithSize - Bytes.SIZEOF_SHORT); - - // copy the column family - System.arraycopy(familyNameWithSize, 0, - current.keyBuffer, rowLengthWithSize, familyNameWithSize.length); - - // copy the qualifier - currentBuffer.get(current.keyBuffer, - rowLengthWithSize + familyNameWithSize.length, - current.keyLength - rowLengthWithSize - - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); - } else if (current.lastCommonPrefix < rowLengthWithSize) { - // we have to copy part of row and qualifier, - // but column family is in right place - - // before column family (rest of row) - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - rowLengthWithSize - current.lastCommonPrefix); - - // after column family (qualifier) - currentBuffer.get(current.keyBuffer, - rowLengthWithSize + familyNameWithSize.length, - current.keyLength - rowLengthWithSize - - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); - } else { - // copy just the ending - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.keyLength - TIMESTAMP_WITH_TYPE_LENGTH - - current.lastCommonPrefix); - } - - - // timestamp - int pos = current.keyLength - TIMESTAMP_WITH_TYPE_LENGTH; - int timestampFitInBytes = 1 + - ((flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH); - long timestampOrDiff = - ByteBufferUtils.readLong(currentBuffer, timestampFitInBytes); - if ((flag & FLAG_TIMESTAMP_SIGN) != 0) { - timestampOrDiff = -timestampOrDiff; - } - if ((flag & FLAG_TIMESTAMP_IS_DIFF) == 0) { // it is timestamp - timestamp = timestampOrDiff; - } else { // it is diff - timestamp = timestamp - timestampOrDiff; - } - Bytes.putLong(current.keyBuffer, pos, timestamp); - pos += Bytes.SIZEOF_LONG; - - // type - if ((flag & FLAG_SAME_TYPE) == 0) { - currentBuffer.get(current.keyBuffer, pos, Bytes.SIZEOF_BYTE); - } else if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - current.keyBuffer[pos] = type; - } - - current.valueOffset = currentBuffer.position(); - ByteBufferUtils.skip(currentBuffer, current.valueLength); - - if (includesMemstoreTS) { - current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer); - } else { - current.memstoreTS = 0; - } - current.nextKvOffset = currentBuffer.position(); - } - - @Override - protected void decodeFirst() { - ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT); - - // read column family - byte familyNameLength = currentBuffer.get(); - familyNameWithSize = new byte[familyNameLength + Bytes.SIZEOF_BYTE]; - familyNameWithSize[0] = familyNameLength; - currentBuffer.get(familyNameWithSize, Bytes.SIZEOF_BYTE, - familyNameLength); - decode(true); - } - - @Override - protected void decodeNext() { - decode(false); - } - }; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java deleted file mode 100644 index 279a393315c..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java +++ /dev/null @@ -1,221 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; - -import org.apache.commons.lang.NotImplementedException; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.io.compress.Compressor; - -/** - * Encapsulates a data block compressed using a particular encoding algorithm. - * Useful for testing and benchmarking. - */ -public class EncodedDataBlock { - private static final int BUFFER_SIZE = 4 * 1024; - protected DataBlockEncoder dataBlockEncoder; - ByteArrayOutputStream uncompressedOutputStream; - ByteBuffer uncompressedBuffer; - private byte[] cacheCompressData; - private ByteArrayOutputStream compressedStream = new ByteArrayOutputStream(); - private boolean includesMemstoreTS; - - /** - * Create a buffer which will be encoded using dataBlockEncoder. - * @param dataBlockEncoder Algorithm used for compression. - */ - public EncodedDataBlock(DataBlockEncoder dataBlockEncoder, - boolean includesMemstoreTS) { - this.dataBlockEncoder = dataBlockEncoder; - uncompressedOutputStream = new ByteArrayOutputStream(BUFFER_SIZE); - } - - /** - * Add KeyValue and compress it. - * @param kv Item to be added and compressed. - */ - public void addKv(KeyValue kv) { - cacheCompressData = null; - uncompressedOutputStream.write( - kv.getBuffer(), kv.getOffset(), kv.getLength()); - } - - /** - * Provides access to compressed value. - * @return Forwards sequential iterator. - */ - public Iterator getIterator() { - final int uncompressedSize = uncompressedOutputStream.size(); - final ByteArrayInputStream bais = new ByteArrayInputStream( - getCompressedData()); - final DataInputStream dis = new DataInputStream(bais); - - - return new Iterator() { - private ByteBuffer decompressedData = null; - - @Override - public boolean hasNext() { - if (decompressedData == null) { - return uncompressedSize > 0; - } - return decompressedData.hasRemaining(); - } - - @Override - public KeyValue next() { - if (decompressedData == null) { - try { - decompressedData = dataBlockEncoder.uncompressKeyValues( - dis, includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException("Problem with data block encoder, " + - "most likely it requested more bytes than are available.", e); - } - decompressedData.rewind(); - } - - int offset = decompressedData.position(); - KeyValue kv = new KeyValue(decompressedData.array(), offset); - decompressedData.position(offset + kv.getLength()); - - return kv; - } - - @Override - public void remove() { - throw new NotImplementedException("remove() is not supported!"); - } - - @Override - public String toString() { - return "Iterator of: " + dataBlockEncoder.getClass().getName(); - } - - }; - } - - /** - * Find the size of minimal buffer that could store compressed data. - * @return Size in bytes of compressed data. - */ - public int getSize() { - return getCompressedData().length; - } - - /** - * Find the size of compressed data assuming that buffer will be compressed - * using given algorithm. - * @param compressor Algorithm used for compression. - * @param buffer Array to be compressed. - * @param offset Offset to beginning of the data. - * @param length Length to be compressed. - * @return Size of compressed data in bytes. - */ - public static int checkCompressedSize(Compressor compressor, byte[] buffer, - int offset, int length) { - byte[] compressedBuffer = new byte[buffer.length]; - // in fact the buffer could be of any positive size - compressor.setInput(buffer, offset, length); - compressor.finish(); - int currentPos = 0; - while (!compressor.finished()) { - try { - // we don't care about compressed data, - // we just want to callculate number of bytes - currentPos += compressor.compress(compressedBuffer, 0, - compressedBuffer.length); - } catch (IOException e) { - throw new RuntimeException( - "For some reason compressor couldn't read data. " + - "It is likely a problem with " + - compressor.getClass().getName(), e); - } - } - return currentPos; - } - - /** - * Estimate size after second stage of compression (e.g. LZO). - * @param compressor Algorithm which will be used for compressions. - * @return Size after second stage of compression. - */ - public int checkCompressedSize(Compressor compressor) { - // compress - byte[] compressedBytes = getCompressedData(); - return checkCompressedSize(compressor, compressedBytes, 0, - compressedBytes.length); - } - - private byte[] getCompressedData() { - // is cached - if (cacheCompressData != null) { - return cacheCompressData; - } - cacheCompressData = doCompressData(); - - return cacheCompressData; - } - - private ByteBuffer getUncompressedBuffer() { - if (uncompressedBuffer == null || - uncompressedBuffer.limit() < uncompressedOutputStream.size()) { - uncompressedBuffer = ByteBuffer.wrap( - uncompressedOutputStream.toByteArray()); - } - return uncompressedBuffer; - } - - /** - * Do the compression. - * @return Compressed byte buffer. - */ - public byte[] doCompressData() { - compressedStream.reset(); - DataOutputStream dataOut = new DataOutputStream(compressedStream); - try { - this.dataBlockEncoder.compressKeyValues( - dataOut, getUncompressedBuffer(), includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException(String.format( - "Bug in decoding part of algorithm %s. " + - "Probably it requested more bytes than are available.", - toString()), e); - } - return compressedStream.toByteArray(); - } - - @Override - public String toString() { - return dataBlockEncoder.toString(); - } - - /** - * Get uncompressed buffer. - * @return The buffer. - */ - public byte[] getRawKeyValues() { - return uncompressedOutputStream.toByteArray(); - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/EncoderBufferTooSmallException.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/EncoderBufferTooSmallException.java deleted file mode 100644 index 55195c18914..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/EncoderBufferTooSmallException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -/** - * Internal error which indicates a bug in a data block encoding algorithm. - */ -public class EncoderBufferTooSmallException extends RuntimeException { - private static final long serialVersionUID = 4767495176134878737L; - - public EncoderBufferTooSmallException(String message) { - super(message); - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java deleted file mode 100644 index ae5229f0d29..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java +++ /dev/null @@ -1,503 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; - -/** - * Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster. - * - * Compress using: - * - store size of common prefix - * - save column family once in the first KeyValue - * - use integer compression for key, value and prefix (128-bit encoding) - * - use bits to avoid duplication key length, value length - * and type if it same as previous - * - store in 3 bits length of prefix timestamp - * with previous KeyValue's timestamp - * - one bit which allow to omit value if it is the same - * - * Format: - * - 1 byte: flag - * - 1-5 bytes: key length (only if FLAG_SAME_KEY_LENGTH is not set in flag) - * - 1-5 bytes: value length (only if FLAG_SAME_VALUE_LENGTH is not set in flag) - * - 1-5 bytes: prefix length - * - ... bytes: rest of the row (if prefix length is small enough) - * - ... bytes: qualifier (or suffix depending on prefix length) - * - 1-8 bytes: timestamp suffix - * - 1 byte: type (only if FLAG_SAME_TYPE is not set in the flag) - * - ... bytes: value (only if FLAG_SAME_VALUE is not set in the flag) - * - */ -public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { - final int MASK_TIMESTAMP_LENGTH = (1 << 0) | (1 << 1) | (1 << 2); - final int SHIFT_TIMESTAMP_LENGTH = 0; - final int FLAG_SAME_KEY_LENGTH = 1 << 3; - final int FLAG_SAME_VALUE_LENGTH = 1 << 4; - final int FLAG_SAME_TYPE = 1 << 5; - final int FLAG_SAME_VALUE = 1 << 6; - - private static class FastDiffCompressionState extends CompressionState { - byte[] timestamp = new byte[KeyValue.TIMESTAMP_SIZE]; - int prevTimestampOffset; - - @Override - protected void readTimestamp(ByteBuffer in) { - in.get(timestamp); - } - - @Override - void copyFrom(CompressionState state) { - super.copyFrom(state); - FastDiffCompressionState state2 = (FastDiffCompressionState) state; - System.arraycopy(state2.timestamp, 0, timestamp, 0, - KeyValue.TIMESTAMP_SIZE); - prevTimestampOffset = state2.prevTimestampOffset; - } - } - - private void compressSingleKeyValue( - FastDiffCompressionState previousState, - FastDiffCompressionState currentState, - OutputStream out, ByteBuffer in) throws IOException { - currentState.prevOffset = in.position(); - int keyLength = in.getInt(); - int valueOffset = currentState.prevOffset + keyLength + KeyValue.ROW_OFFSET; - int valueLength = in.getInt(); - byte flag = 0; - - if (previousState.isFirst()) { - // copy the key, there is no common prefix with none - ByteBufferUtils.copyToStream(out, flag); - ByteBufferUtils.putCompressedInt(out, keyLength); - ByteBufferUtils.putCompressedInt(out, valueLength); - ByteBufferUtils.putCompressedInt(out, 0); - - currentState.readKey(in, keyLength, valueLength); - - ByteBufferUtils.copyToStream(out, in, keyLength + valueLength); - } else { - // find a common prefix and skip it - int commonPrefix = ByteBufferUtils.findCommonPrefix(in, in.position(), - previousState.prevOffset + KeyValue.ROW_OFFSET, - keyLength - KeyValue.TIMESTAMP_TYPE_SIZE); - - currentState.readKey(in, keyLength, valueLength, - commonPrefix, previousState); - - if (keyLength == previousState.keyLength) { - flag |= FLAG_SAME_KEY_LENGTH; - } - if (valueLength == previousState.valueLength) { - flag |= FLAG_SAME_VALUE_LENGTH; - } - if (currentState.type == previousState.type) { - flag |= FLAG_SAME_TYPE; - } - - int prefixTimestamp = findCommonTimestampPrefix( - currentState, previousState); - flag |= (prefixTimestamp) << SHIFT_TIMESTAMP_LENGTH; - - if (ByteBufferUtils.arePartsEqual(in, - previousState.prevOffset + previousState.keyLength + KeyValue.ROW_OFFSET, - previousState.valueLength, valueOffset, valueLength)) { - flag |= FLAG_SAME_VALUE; - } - - ByteBufferUtils.copyToStream(out, flag); - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - ByteBufferUtils.putCompressedInt(out, keyLength); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - ByteBufferUtils.putCompressedInt(out, valueLength); - } - ByteBufferUtils.putCompressedInt(out, commonPrefix); - - ByteBufferUtils.skip(in, commonPrefix); - if (commonPrefix < currentState.rowLength + KeyValue.ROW_LENGTH_SIZE) { - ByteBufferUtils.copyToStream(out, in, - currentState.rowLength + KeyValue.ROW_LENGTH_SIZE - commonPrefix); - ByteBufferUtils.skip(in, currentState.familyLength + - KeyValue.FAMILY_LENGTH_SIZE); - ByteBufferUtils.copyToStream(out, in, currentState.qualifierLength); - } else { - int restKeyLength = keyLength - commonPrefix - - KeyValue.TIMESTAMP_TYPE_SIZE; - ByteBufferUtils.copyToStream(out, in, restKeyLength); - } - ByteBufferUtils.skip(in, prefixTimestamp); - ByteBufferUtils.copyToStream(out, in, - KeyValue.TIMESTAMP_SIZE - prefixTimestamp); - - if ((flag & FLAG_SAME_TYPE) == 0) { - valueOffset -= KeyValue.TYPE_SIZE; - valueLength += KeyValue.TYPE_SIZE; - } - - ByteBufferUtils.skip(in, KeyValue.TYPE_SIZE + currentState.valueLength); - - if ((flag & FLAG_SAME_VALUE) == 0 ) { - ByteBufferUtils.copyToStream(out, in, valueOffset, valueLength); - } else { - if ((flag & FLAG_SAME_TYPE) == 0) { - ByteBufferUtils.copyToStream(out, currentState.type); - } - } - } - } - - private int findCommonTimestampPrefix(FastDiffCompressionState left, - FastDiffCompressionState right) { - int prefixTimestamp = 0; - while (prefixTimestamp < (KeyValue.TIMESTAMP_SIZE - 1) && - left.timestamp[prefixTimestamp] - == right.timestamp[prefixTimestamp]) { - prefixTimestamp++; - } - return prefixTimestamp; // has to be at most 7 bytes - } - - private void uncompressSingleKeyValue(DataInputStream source, - ByteBuffer buffer, FastDiffCompressionState state) - throws IOException, EncoderBufferTooSmallException { - byte flag = source.readByte(); - int prevKeyLength = state.keyLength; - - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - state.keyLength = ByteBufferUtils.readCompressedInt(source); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - state.valueLength = ByteBufferUtils.readCompressedInt(source); - } - int commonLength = ByteBufferUtils.readCompressedInt(source); - - ByteBufferUtils.ensureSpace(buffer, state.keyLength + state.valueLength + - KeyValue.ROW_OFFSET); - - int kvPos = buffer.position(); - - if (!state.isFirst()) { - // copy the prefix - int common; - int prevOffset; - - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - buffer.putInt(state.keyLength); - buffer.putInt(state.valueLength); - prevOffset = state.prevOffset + KeyValue.ROW_OFFSET; - common = commonLength; - } else { - if ((flag & FLAG_SAME_KEY_LENGTH) != 0) { - prevOffset = state.prevOffset; - common = commonLength + KeyValue.ROW_OFFSET; - } else { - buffer.putInt(state.keyLength); - prevOffset = state.prevOffset + KeyValue.KEY_LENGTH_SIZE; - common = commonLength + KeyValue.KEY_LENGTH_SIZE; - } - } - - ByteBufferUtils.copyFromBuffer(buffer, buffer, prevOffset, common); - - // copy the rest of the key from the buffer - int keyRestLength; - if (commonLength < state.rowLength + KeyValue.ROW_LENGTH_SIZE) { - // omit the family part of the key, it is always the same - int rowWithSizeLength; - int rowRestLength; - - // check length of row - if (commonLength < KeyValue.ROW_LENGTH_SIZE) { - // not yet copied, do it now - ByteBufferUtils.copyFromStream(source, buffer, - KeyValue.ROW_LENGTH_SIZE - commonLength); - - rowWithSizeLength = buffer.getShort(buffer.position() - - KeyValue.ROW_LENGTH_SIZE) + KeyValue.ROW_LENGTH_SIZE; - rowRestLength = rowWithSizeLength - KeyValue.ROW_LENGTH_SIZE; - } else { - // already in kvBuffer, just read it - rowWithSizeLength = buffer.getShort(kvPos + KeyValue.ROW_OFFSET) + - KeyValue.ROW_LENGTH_SIZE; - rowRestLength = rowWithSizeLength - commonLength; - } - - // copy the rest of row - ByteBufferUtils.copyFromStream(source, buffer, rowRestLength); - - // copy the column family - ByteBufferUtils.copyFromBuffer(buffer, buffer, - state.prevOffset + KeyValue.ROW_OFFSET + KeyValue.ROW_LENGTH_SIZE - + state.rowLength, state.familyLength - + KeyValue.FAMILY_LENGTH_SIZE); - state.rowLength = (short) (rowWithSizeLength - - KeyValue.ROW_LENGTH_SIZE); - - keyRestLength = state.keyLength - rowWithSizeLength - - state.familyLength - - (KeyValue.FAMILY_LENGTH_SIZE + KeyValue.TIMESTAMP_TYPE_SIZE); - } else { - // prevRowWithSizeLength is the same as on previous row - keyRestLength = state.keyLength - commonLength - - KeyValue.TIMESTAMP_TYPE_SIZE; - } - // copy the rest of the key, after column family == column qualifier - ByteBufferUtils.copyFromStream(source, buffer, keyRestLength); - - // copy timestamp - int prefixTimestamp = - (flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH; - ByteBufferUtils.copyFromBuffer(buffer, buffer, state.prevTimestampOffset, - prefixTimestamp); - state.prevTimestampOffset = buffer.position() - prefixTimestamp; - ByteBufferUtils.copyFromStream(source, buffer, KeyValue.TIMESTAMP_SIZE - - prefixTimestamp); - - // copy the type and value - if ((flag & FLAG_SAME_TYPE) != 0) { - buffer.put(state.type); - if ((flag & FLAG_SAME_VALUE) != 0) { - ByteBufferUtils.copyFromBuffer(buffer, buffer, state.prevOffset + - KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength); - } else { - ByteBufferUtils.copyFromStream(source, buffer, state.valueLength); - } - } else { - if ((flag & FLAG_SAME_VALUE) != 0) { - ByteBufferUtils.copyFromStream(source, buffer, KeyValue.TYPE_SIZE); - ByteBufferUtils.copyFromBuffer(buffer, buffer, state.prevOffset + - KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength); - } else { - ByteBufferUtils.copyFromStream(source, buffer, - state.valueLength + KeyValue.TYPE_SIZE); - } - state.type = buffer.get(state.prevTimestampOffset + - KeyValue.TIMESTAMP_SIZE); - } - } else { // is first element - buffer.putInt(state.keyLength); - buffer.putInt(state.valueLength); - - state.prevTimestampOffset = buffer.position() + state.keyLength - - KeyValue.TIMESTAMP_TYPE_SIZE; - ByteBufferUtils.copyFromStream(source, buffer, state.keyLength - + state.valueLength); - state.rowLength = buffer.getShort(kvPos + KeyValue.ROW_OFFSET); - state.familyLength = buffer.get(kvPos + KeyValue.ROW_OFFSET + - KeyValue.ROW_LENGTH_SIZE + state.rowLength); - state.type = buffer.get(state.prevTimestampOffset + - KeyValue.TIMESTAMP_SIZE); - } - - state.prevOffset = kvPos; - } - - @Override - public void compressKeyValues(DataOutputStream out, - ByteBuffer in, boolean includesMemstoreTS) throws IOException { - in.rewind(); - ByteBufferUtils.putInt(out, in.limit()); - FastDiffCompressionState previousState = new FastDiffCompressionState(); - FastDiffCompressionState currentState = new FastDiffCompressionState(); - while (in.hasRemaining()) { - compressSingleKeyValue(previousState, currentState, - out, in); - afterEncodingKeyValue(in, out, includesMemstoreTS); - - // swap previousState <-> currentState - FastDiffCompressionState tmp = previousState; - previousState = currentState; - currentState = tmp; - } - } - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream source, - int allocHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException { - int decompressedSize = source.readInt(); - ByteBuffer buffer = ByteBuffer.allocate(decompressedSize + - allocHeaderLength); - buffer.position(allocHeaderLength); - FastDiffCompressionState state = new FastDiffCompressionState(); - while (source.available() > skipLastBytes) { - uncompressSingleKeyValue(source, buffer, state); - afterDecodingKeyValue(source, buffer, includesMemstoreTS); - } - - if (source.available() != skipLastBytes) { - throw new IllegalStateException("Read too much bytes."); - } - - return buffer; - } - - @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { - block.mark(); - block.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); // valueLength - ByteBufferUtils.readCompressedInt(block); // commonLength - int pos = block.position(); - block.reset(); - return ByteBuffer.wrap(block.array(), pos, keyLength).slice(); - } - - @Override - public String toString() { - return FastDiffDeltaEncoder.class.getSimpleName(); - } - - @Override - public EncodedSeeker createSeeker(RawComparator comparator, - final boolean includesMemstoreTS) { - return new BufferedEncodedSeeker(comparator) { - private byte[] prevTimestampAndType = new byte[ - Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE]; - private int rowLengthWithSize; - private int columnFamilyLengthWithSize; - - private void decode(boolean isFirst) { - byte flag = currentBuffer.get(); - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - if (!isFirst) { - System.arraycopy(current.keyBuffer, - current.keyLength - prevTimestampAndType.length, - prevTimestampAndType, 0, - prevTimestampAndType.length); - } - current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); - } - if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { - current.valueLength = - ByteBufferUtils.readCompressedInt(currentBuffer); - } - current.lastCommonPrefix = - ByteBufferUtils.readCompressedInt(currentBuffer); - - current.ensureSpaceForKey(); - - if (isFirst) { - // copy everything - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.keyLength - prevTimestampAndType.length); - rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - columnFamilyLengthWithSize = current.keyBuffer[rowLengthWithSize] + - Bytes.SIZEOF_BYTE; - } else if (current.lastCommonPrefix < Bytes.SIZEOF_SHORT) { - // length of row is different, copy everything except family - - // copy the row size - int oldRowLengthWithSize = rowLengthWithSize; - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - Bytes.SIZEOF_SHORT - current.lastCommonPrefix); - rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - - // move the column family - System.arraycopy(current.keyBuffer, oldRowLengthWithSize, - current.keyBuffer, rowLengthWithSize, - columnFamilyLengthWithSize); - - // copy the rest of row - currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT, - rowLengthWithSize - Bytes.SIZEOF_SHORT); - - // copy the qualifier - currentBuffer.get(current.keyBuffer, - rowLengthWithSize + columnFamilyLengthWithSize, - current.keyLength - rowLengthWithSize - - columnFamilyLengthWithSize - prevTimestampAndType.length); - } else if (current.lastCommonPrefix < rowLengthWithSize) { - // we have to copy part of row and qualifier, - // but column family is in right place - - // before column family (rest of row) - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - rowLengthWithSize - current.lastCommonPrefix); - - // after column family (qualifier) - currentBuffer.get(current.keyBuffer, - rowLengthWithSize + columnFamilyLengthWithSize, - current.keyLength - rowLengthWithSize - - columnFamilyLengthWithSize - prevTimestampAndType.length); - } else { - // copy just the ending - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.keyLength - prevTimestampAndType.length - - current.lastCommonPrefix); - } - - // timestamp - int pos = current.keyLength - prevTimestampAndType.length; - int commonTimestampPrefix = (flag & MASK_TIMESTAMP_LENGTH) >>> - SHIFT_TIMESTAMP_LENGTH; - if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - System.arraycopy(prevTimestampAndType, 0, current.keyBuffer, - pos, commonTimestampPrefix); - } - pos += commonTimestampPrefix; - currentBuffer.get(current.keyBuffer, pos, - Bytes.SIZEOF_LONG - commonTimestampPrefix); - pos += Bytes.SIZEOF_LONG - commonTimestampPrefix; - - // type - if ((flag & FLAG_SAME_TYPE) == 0) { - currentBuffer.get(current.keyBuffer, pos, Bytes.SIZEOF_BYTE); - } else if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - current.keyBuffer[pos] = prevTimestampAndType[Bytes.SIZEOF_LONG]; - } - - // handle value - if ((flag & FLAG_SAME_VALUE) == 0) { - current.valueOffset = currentBuffer.position(); - ByteBufferUtils.skip(currentBuffer, current.valueLength); - } - - if (includesMemstoreTS) { - current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer); - } else { - current.memstoreTS = 0; - } - current.nextKvOffset = currentBuffer.position(); - } - - @Override - protected void decodeFirst() { - ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT); - decode(true); - } - - @Override - protected void decodeNext() { - decode(false); - } - }; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java deleted file mode 100644 index 710476cf2d1..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.RawComparator; - -/** - * Compress key by storing size of common prefix with previous KeyValue - * and storing raw size of rest. - * - * Format: - * 1-5 bytes: compressed key length minus prefix (7-bit encoding) - * 1-5 bytes: compressed value length (7-bit encoding) - * 1-3 bytes: compressed length of common key prefix - * ... bytes: rest of key (including timestamp) - * ... bytes: value - * - * In a worst case compressed KeyValue will be three bytes longer than original. - * - */ -public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { - - private int addKv(int offset, DataOutputStream out, - ByteBuffer in, int prevKeyLength) throws IOException { - int keyLength = in.getInt(); - int valueLength = in.getInt(); - - if (offset == -1) { - // copy the key, there is no common prefix with none - ByteBufferUtils.putCompressedInt(out, keyLength); - ByteBufferUtils.putCompressedInt(out, valueLength); - ByteBufferUtils.putCompressedInt(out, 0); - ByteBufferUtils.copyToStream(out, in, keyLength + valueLength); - } else { - // find a common prefix and skip it - int common = ByteBufferUtils.findCommonPrefix( - in, offset + KeyValue.ROW_OFFSET, - in.position(), - Math.min(prevKeyLength, keyLength)); - - ByteBufferUtils.putCompressedInt(out, keyLength - common); - ByteBufferUtils.putCompressedInt(out, valueLength); - ByteBufferUtils.putCompressedInt(out, common); - - ByteBufferUtils.skip(in, common); - ByteBufferUtils.copyToStream(out, in, keyLength - common + valueLength); - } - - return keyLength; - } - - @Override - public void compressKeyValues(DataOutputStream writeHere, - ByteBuffer in, boolean includesMemstoreTS) throws IOException { - in.rewind(); - ByteBufferUtils.putInt(writeHere, in.limit()); - int prevOffset = -1; - int offset = 0; - int keyLength = 0; - while (in.hasRemaining()) { - offset = in.position(); - keyLength = addKv(prevOffset, writeHere, in, keyLength); - afterEncodingKeyValue(in, writeHere, includesMemstoreTS); - prevOffset = offset; - } - } - - @Override - public ByteBuffer uncompressKeyValues(DataInputStream source, - int allocHeaderLength, int skipLastBytes, boolean includesMemstoreTS) - throws IOException { - int decompressedSize = source.readInt(); - ByteBuffer buffer = ByteBuffer.allocate(decompressedSize + - allocHeaderLength); - buffer.position(allocHeaderLength); - int prevKeyOffset = 0; - - while (source.available() > skipLastBytes) { - prevKeyOffset = uncompressKeyValue(source, buffer, prevKeyOffset); - afterDecodingKeyValue(source, buffer, includesMemstoreTS); - } - - if (source.available() != skipLastBytes) { - throw new IllegalStateException("Read too many bytes."); - } - - buffer.limit(buffer.position()); - return buffer; - } - - private int uncompressKeyValue(DataInputStream source, ByteBuffer buffer, - int prevKeyOffset) - throws IOException, EncoderBufferTooSmallException { - int keyLength = ByteBufferUtils.readCompressedInt(source); - int valueLength = ByteBufferUtils.readCompressedInt(source); - int commonLength = ByteBufferUtils.readCompressedInt(source); - int keyOffset; - keyLength += commonLength; - - ByteBufferUtils.ensureSpace(buffer, keyLength + valueLength - + KeyValue.ROW_OFFSET); - - buffer.putInt(keyLength); - buffer.putInt(valueLength); - - // copy the prefix - if (commonLength > 0) { - keyOffset = buffer.position(); - ByteBufferUtils.copyFromBuffer(buffer, buffer, prevKeyOffset, - commonLength); - } else { - keyOffset = buffer.position(); - } - - // copy rest of the key and value - int len = keyLength - commonLength + valueLength; - ByteBufferUtils.copyFromStream(source, buffer, len); - return keyOffset; - } - - @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { - block.mark(); - block.position(Bytes.SIZEOF_INT); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); - int commonLength = ByteBufferUtils.readCompressedInt(block); - if (commonLength != 0) { - throw new AssertionError("Nonzero common length in the first key in " - + "block: " + commonLength); - } - int pos = block.position(); - block.reset(); - return ByteBuffer.wrap(block.array(), pos, keyLength).slice(); - } - - @Override - public String toString() { - return PrefixKeyDeltaEncoder.class.getSimpleName(); - } - - @Override - public EncodedSeeker createSeeker(RawComparator comparator, - final boolean includesMemstoreTS) { - return new BufferedEncodedSeeker(comparator) { - @Override - protected void decodeNext() { - current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); - current.valueLength = ByteBufferUtils.readCompressedInt(currentBuffer); - current.lastCommonPrefix = - ByteBufferUtils.readCompressedInt(currentBuffer); - current.keyLength += current.lastCommonPrefix; - current.ensureSpaceForKey(); - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.keyLength - current.lastCommonPrefix); - current.valueOffset = currentBuffer.position(); - ByteBufferUtils.skip(currentBuffer, current.valueLength); - if (includesMemstoreTS) { - current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer); - } else { - current.memstoreTS = 0; - } - current.nextKvOffset = currentBuffer.position(); - } - - @Override - protected void decodeFirst() { - ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT); - decodeNext(); - } - }; - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java index f9321c6f12c..9123e70be08 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java @@ -1,4 +1,4 @@ - /* +/* * Copyright 2011 The Apache Software Foundation * * Licensed to the Apache Software Foundation (ASF) under one @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; import org.apache.hadoop.io.RawComparator; @@ -59,12 +60,6 @@ public abstract class AbstractHFileReader extends SchemaConfigured /** Filled when we read in the trailer. */ protected final Compression.Algorithm compressAlgo; - /** - * What kind of data block encoding should be used while reading, writing and - * handling cache. - */ - protected final HFileDataBlockEncoder blockEncoder; - /** Last key in the file. Filled in when we read in the file info */ protected byte [] lastKey = null; @@ -98,8 +93,7 @@ public abstract class AbstractHFileReader extends SchemaConfigured protected AbstractHFileReader(Path path, FixedFileTrailer trailer, final FSDataInputStream fsdis, final long fileSize, final boolean closeIStream, - final CacheConfig cacheConf, - final HFileDataBlockEncoder dataBlockEncoder) { + final CacheConfig cacheConf) { super(null, path); this.trailer = trailer; this.compressAlgo = trailer.getCompressionCodec(); @@ -107,8 +101,6 @@ public abstract class AbstractHFileReader extends SchemaConfigured this.fileSize = fileSize; this.istream = fsdis; this.closeIStream = closeIStream; - this.blockEncoder = dataBlockEncoder != null - ? dataBlockEncoder : new NoOpDataBlockEncoder(); this.path = path; this.name = path.getName(); } @@ -283,11 +275,8 @@ public abstract class AbstractHFileReader extends SchemaConfigured protected int blockFetches; - protected final HFile.Reader reader; - - public Scanner(final HFile.Reader reader, final boolean cacheBlocks, + public Scanner(final boolean cacheBlocks, final boolean pread, final boolean isCompaction) { - this.reader = reader; this.cacheBlocks = cacheBlocks; this.pread = pread; this.isCompaction = isCompaction; @@ -307,26 +296,6 @@ public abstract class AbstractHFileReader extends SchemaConfigured if (!isSeeked()) throw new NotSeekedException(); } - - @Override - public int seekTo(byte[] key) throws IOException { - return seekTo(key, 0, key.length); - } - - @Override - public boolean seekBefore(byte[] key) throws IOException { - return seekBefore(key, 0, key.length); - } - - @Override - public int reseekTo(byte[] key) throws IOException { - return reseekTo(key, 0, key.length); - } - - @Override - public HFile.Reader getReader() { - return reader; - } } /** For testing */ @@ -337,4 +306,5 @@ public abstract class AbstractHFileReader extends SchemaConfigured public Path getPath() { return path; } + } diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java index 84523c149db..4c74738be3a 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java @@ -83,9 +83,6 @@ public abstract class AbstractHFileWriter extends SchemaConfigured /** The compression algorithm used. NONE if no compression. */ protected final Compression.Algorithm compressAlgo; - - /** The data block encoding which will be used. NONE if there is no encoding */ - protected final HFileDataBlockEncoder blockEncoder; /** First key in a block. */ protected byte[] firstKeyInBlock = null; @@ -105,9 +102,7 @@ public abstract class AbstractHFileWriter extends SchemaConfigured public AbstractHFileWriter(CacheConfig cacheConf, FSDataOutputStream outputStream, Path path, int blockSize, - Compression.Algorithm compressAlgo, - HFileDataBlockEncoder dataBlockEncoder, - KeyComparator comparator) { + Compression.Algorithm compressAlgo, KeyComparator comparator) { super(null, path); this.outputStream = outputStream; this.path = path; @@ -115,8 +110,6 @@ public abstract class AbstractHFileWriter extends SchemaConfigured this.blockSize = blockSize; this.compressAlgo = compressAlgo == null ? HFile.DEFAULT_COMPRESSION_ALGORITHM : compressAlgo; - this.blockEncoder = dataBlockEncoder != null - ? dataBlockEncoder : new NoOpDataBlockEncoder(); this.comparator = comparator != null ? comparator : Bytes.BYTES_RAWCOMPARATOR; diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java index 4d574707f67..995e28a54c9 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java @@ -39,14 +39,6 @@ public enum BlockType { /** Data block, both versions */ DATA("DATABLK*", BlockCategory.DATA), - /** An encoded data block (e.g. with prefix compression), version 2 */ - ENCODED_DATA("DATABLKE", BlockCategory.DATA) { - @Override - public int getId() { - return DATA.ordinal(); - } - }, - /** Version 2 leaf index block. Appears in the data block section */ LEAF_INDEX("IDXLEAF2", BlockCategory.INDEX), @@ -110,15 +102,6 @@ public enum BlockType { this.metricCat = metricCat; assert magic.length == MAGIC_LENGTH; } - - /** - * Use it instead of oridinal(). It works exactly the same, - * except DATA and ENCODED_DATA got same id. - * @return id between 0 and N - */ - public int getId() { - return ordinal(); - } public void writeToStream(OutputStream out) throws IOException { out.write(magic); diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 48c673f5855..ace20ba4e5d 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -243,7 +243,6 @@ public class HFile { public abstract Writer createWriter(FileSystem fs, Path path, int blockSize, Compression.Algorithm compress, - HFileDataBlockEncoder dataBlockEncoder, final KeyComparator comparator) throws IOException; public abstract Writer createWriter(FileSystem fs, Path path, @@ -372,47 +371,32 @@ public class HFile { } private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis, - long size, boolean closeIStream, CacheConfig cacheConf, - HFileDataBlockEncoder dataBlockEncoder) + long size, boolean closeIStream, CacheConfig cacheConf) throws IOException { FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis, size); switch (trailer.getVersion()) { case 1: return new HFileReaderV1(path, trailer, fsdis, size, closeIStream, - cacheConf, dataBlockEncoder); + cacheConf); case 2: return new HFileReaderV2(path, trailer, fsdis, size, closeIStream, - cacheConf, dataBlockEncoder); + cacheConf); default: throw new IOException("Cannot instantiate reader for HFile version " + trailer.getVersion()); } } - public static Reader createReader( - FileSystem fs, Path path, CacheConfig cacheConf) throws IOException { - return createReader(fs, path, cacheConf, - new NoOpDataBlockEncoder()); - } - - public static Reader createReader(Path path, FSDataInputStream fsdis, - long size, CacheConfig cacheConf) throws IOException { - return createReader(path, fsdis, size, cacheConf, - new NoOpDataBlockEncoder()); - } - public static Reader createReader(FileSystem fs, Path path, - CacheConfig cacheConf, HFileDataBlockEncoder dataBlockEncoder) - throws IOException { + CacheConfig cacheConf) throws IOException { return pickReaderVersion(path, fs.open(path), - fs.getFileStatus(path).getLen(), true, cacheConf, dataBlockEncoder); + fs.getFileStatus(path).getLen(), true, cacheConf); } public static Reader createReader(Path path, FSDataInputStream fsdis, - long size, CacheConfig cacheConf, - HFileDataBlockEncoder dataBlockEncoder) throws IOException { - return pickReaderVersion(path, fsdis, size, false, cacheConf, - dataBlockEncoder); + long size, CacheConfig cacheConf) + throws IOException { + return pickReaderVersion(path, fsdis, size, false, cacheConf); } /* diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index 8d99f9fe9f7..2a04e9b10e8 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -17,9 +17,6 @@ */ package org.apache.hadoop.hbase.io.hfile; -import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH; -import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.NONE; - import java.io.BufferedInputStream; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -34,12 +31,11 @@ import java.nio.ByteBuffer; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.hbase.io.DoubleOutputStream; import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; -import org.apache.hadoop.hbase.regionserver.MemStore; import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.CompoundBloomFilter; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.IOUtils; @@ -49,6 +45,9 @@ import org.apache.hadoop.io.compress.Decompressor; import com.google.common.base.Preconditions; +import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH; +import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.NONE; + /** * Reading {@link HFile} version 1 and 2 blocks, and writing version 2 blocks. *
    @@ -76,26 +75,10 @@ import com.google.common.base.Preconditions; */ public class HFileBlock extends SchemaConfigured implements Cacheable { - public static final boolean FILL_HEADER = true; - public static final boolean DONT_FILL_HEADER = false; - /** The size of a version 2 {@link HFile} block header */ public static final int HEADER_SIZE = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG; - /** - * We store a two-byte encoder ID at the beginning of every encoded data - * block payload (immediately after the block header). - */ - public static final int DATA_BLOCK_ENCODER_ID_SIZE = Bytes.SIZEOF_SHORT; - - /** - * The size of block header when blockType is {@link BlockType#ENCODED_DATA}. - * This extends normal header by adding the id of encoder. - */ - public static final int ENCODED_HEADER_SIZE = HEADER_SIZE - + DATA_BLOCK_ENCODER_ID_SIZE; - /** Just an array of bytes of the right size. */ public static final byte[] DUMMY_HEADER = new byte[HEADER_SIZE]; @@ -124,11 +107,10 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { }; private BlockType blockType; - private int onDiskSizeWithoutHeader; + private final int onDiskSizeWithoutHeader; private final int uncompressedSizeWithoutHeader; private final long prevBlockOffset; private ByteBuffer buf; - private boolean includesMemstoreTS; /** * The offset of this block in the file. Populated by the reader for @@ -164,7 +146,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuffer buf, - boolean fillHeader, long offset, boolean includesMemstoreTS) { + boolean fillHeader, long offset) { this.blockType = blockType; this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader; this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader; @@ -173,7 +155,6 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { if (fillHeader) overwriteHeader(); this.offset = offset; - this.includesMemstoreTS = includesMemstoreTS; } /** @@ -196,15 +177,6 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { return blockType; } - /** @return get data block encoding id that was used to encode this block */ - public short getDataBlockEncodingId() { - if (blockType != BlockType.ENCODED_DATA) { - throw new IllegalArgumentException("Querying encoder ID of a block " + - "of type other than " + BlockType.ENCODED_DATA + ": " + blockType); - } - return buf.getShort(HEADER_SIZE); - } - /** * @return the on-disk size of the block with header size included */ @@ -537,30 +509,29 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { /** Compression algorithm for all blocks this instance writes. */ private final Compression.Algorithm compressAlgo; - /** Data block encoder used for data blocks */ - private final HFileDataBlockEncoder dataBlockEncoder; + /** + * The stream we use to accumulate data in the on-disk format for each + * block (i.e. compressed data, or uncompressed if using no compression). + * We reset this stream at the end of each block and reuse it. The header + * is written as the first {@link #HEADER_SIZE} bytes into this stream. + */ + private ByteArrayOutputStream baosOnDisk; /** - * The stream we use to accumulate data in uncompressed format for each - * block. We reset this stream at the end of each block and reuse it. The - * header is written as the first {@link #HEADER_SIZE} bytes into this - * stream. + * The stream we use to accumulate uncompressed block data for + * cache-on-write. Null when cache-on-write is turned off. */ private ByteArrayOutputStream baosInMemory; /** Compressor, which is also reused between consecutive blocks. */ private Compressor compressor; - /** - * Current block type. Set in {@link #startWriting(BlockType)}. Could be - * changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA} - * to {@link BlockType#ENCODED_DATA}. - */ + /** Current block type. Set in {@link #startWriting(BlockType)}. */ private BlockType blockType; /** * A stream that we write uncompressed bytes to, which compresses them and - * writes them to {@link #baosInMemory}. + * writes them to {@link #baosOnDisk}. */ private DataOutputStream userDataStream; @@ -571,8 +542,14 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { private byte[] onDiskBytesWithHeader; /** - * Valid in the READY state. Contains the header and the uncompressed (but - * potentially encoded, if this is a data block) bytes, so the length is + * The total number of uncompressed bytes written into the current block, + * with header size not included. Valid in the READY state. + */ + private int uncompressedSizeWithoutHeader; + + /** + * Only used when we are using cache-on-write. Valid in the READY state. + * Contains the header and the uncompressed bytes, so the length is * {@link #uncompressedSizeWithoutHeader} + {@link HFileBlock#HEADER_SIZE}. */ private byte[] uncompressedBytesWithHeader; @@ -589,36 +566,30 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ private long[] prevOffsetByType; + /** + * Whether we are accumulating uncompressed bytes for the purpose of + * caching on write. + */ + private boolean cacheOnWrite; + /** The offset of the previous block of the same type */ private long prevOffset; - /** Whether we are including memstore timestamp after every key/value */ - private boolean includesMemstoreTS; - /** - * Unencoded data block for caching on write. Populated before encoding. + * @param compressionAlgorithm + * compression algorithm to use */ - private HFileBlock unencodedDataBlockForCaching; + public Writer(Compression.Algorithm compressionAlgorithm) { + compressAlgo = compressionAlgorithm == null ? NONE + : compressionAlgorithm; - /** - * @param compressionAlgorithm compression algorithm to use - * @param dataBlockEncoderAlgo data block encoding algorithm to use - */ - public Writer(Compression.Algorithm compressionAlgorithm, - HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS) { - compressAlgo = compressionAlgorithm == null ? NONE : compressionAlgorithm; - this.dataBlockEncoder = dataBlockEncoder != null - ? dataBlockEncoder : new NoOpDataBlockEncoder(); - - baosInMemory = new ByteArrayOutputStream(); + baosOnDisk = new ByteArrayOutputStream(); if (compressAlgo != NONE) compressor = compressionAlgorithm.getCompressor(); prevOffsetByType = new long[BlockType.values().length]; for (int i = 0; i < prevOffsetByType.length; ++i) prevOffsetByType[i] = -1; - - this.includesMemstoreTS = includesMemstoreTS; } /** @@ -627,26 +598,44 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { * @return the stream the user can write their data into * @throws IOException */ - public DataOutputStream startWriting(BlockType newBlockType) - throws IOException { + public DataOutputStream startWriting(BlockType newBlockType, + boolean cacheOnWrite) throws IOException { if (state == State.BLOCK_READY && startOffset != -1) { // We had a previous block that was written to a stream at a specific // offset. Save that offset as the last offset of a block of that type. - prevOffsetByType[blockType.getId()] = startOffset; + prevOffsetByType[blockType.ordinal()] = startOffset; } + this.cacheOnWrite = cacheOnWrite; + startOffset = -1; blockType = newBlockType; - baosInMemory.reset(); - baosInMemory.write(DUMMY_HEADER); + baosOnDisk.reset(); + baosOnDisk.write(DUMMY_HEADER); state = State.WRITING; + if (compressAlgo == NONE) { + // We do not need a compression stream or a second uncompressed stream + // for cache-on-write. + userDataStream = new DataOutputStream(baosOnDisk); + } else { + OutputStream compressingOutputStream = + compressAlgo.createCompressionStream(baosOnDisk, compressor, 0); - unencodedDataBlockForCaching = null; + if (cacheOnWrite) { + // We save uncompressed data in a cache-on-write mode. + if (baosInMemory == null) + baosInMemory = new ByteArrayOutputStream(); + baosInMemory.reset(); + baosInMemory.write(DUMMY_HEADER); + userDataStream = new DataOutputStream(new DoubleOutputStream( + compressingOutputStream, baosInMemory)); + } else { + userDataStream = new DataOutputStream(compressingOutputStream); + } + } - // We will compress it later in finishBlock() - userDataStream = new DataOutputStream(baosInMemory); return userDataStream; } @@ -673,125 +662,45 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { if (state == State.BLOCK_READY) return; - // This will set state to BLOCK_READY. finishBlock(); + state = State.BLOCK_READY; } /** * An internal method that flushes the compressing stream (if using * compression), serializes the header, and takes care of the separate - * uncompressed stream for caching on write, if applicable. Sets block - * write state to "block ready". + * uncompressed stream for caching on write, if applicable. Block writer + * state transitions must be managed by the caller. */ private void finishBlock() throws IOException { userDataStream.flush(); + uncompressedSizeWithoutHeader = userDataStream.size(); - // This does an array copy, so it is safe to cache this byte array. - uncompressedBytesWithHeader = baosInMemory.toByteArray(); - prevOffset = prevOffsetByType[blockType.getId()]; + onDiskBytesWithHeader = baosOnDisk.toByteArray(); + prevOffset = prevOffsetByType[blockType.ordinal()]; + putHeader(onDiskBytesWithHeader, 0); - // We need to set state before we can package the block up for - // cache-on-write. In a way, the block is ready, but not yet encoded or - // compressed. - state = State.BLOCK_READY; - encodeDataBlockForDisk(); + if (cacheOnWrite && compressAlgo != NONE) { + uncompressedBytesWithHeader = baosInMemory.toByteArray(); - doCompression(); - putHeader(uncompressedBytesWithHeader, 0, onDiskBytesWithHeader.length, - uncompressedBytesWithHeader.length); - - if (unencodedDataBlockForCaching != null) { - // We now know the final on-disk size, save it for caching. - unencodedDataBlockForCaching.onDiskSizeWithoutHeader = - getOnDiskSizeWithoutHeader(); - unencodedDataBlockForCaching.overwriteHeader(); - } - } - - /** - * Do compression if it is enabled, or re-use the uncompressed buffer if - * it is not. Fills in the compressed block's header if doing compression. - */ - private void doCompression() throws IOException { - // do the compression - if (compressAlgo != NONE) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - baos.write(DUMMY_HEADER); - - // compress the data - OutputStream compressingOutputStream = - compressAlgo.createCompressionStream(baos, compressor, 0); - compressingOutputStream.write(uncompressedBytesWithHeader, HEADER_SIZE, - uncompressedBytesWithHeader.length - HEADER_SIZE); - - // finish compression stream - compressingOutputStream.flush(); - - onDiskBytesWithHeader = baos.toByteArray(); - putHeader(onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length, - uncompressedBytesWithHeader.length); - } else { - onDiskBytesWithHeader = uncompressedBytesWithHeader; - } - } - - /** - * Encodes this block if it is a data block and encoding is turned on in - * {@link #dataBlockEncoder}. - */ - private void encodeDataBlockForDisk() throws IOException { - if (blockType != BlockType.DATA) { - return; // skip any non-data block - } - - // do data block encoding, if data block encoder is set - ByteBuffer rawKeyValues = ByteBuffer.wrap(uncompressedBytesWithHeader, - HEADER_SIZE, uncompressedBytesWithHeader.length - - HEADER_SIZE).slice(); - Pair encodingResult = - dataBlockEncoder.beforeWriteToDisk(rawKeyValues, - includesMemstoreTS); - - BlockType encodedBlockType = encodingResult.getSecond(); - if (encodedBlockType == BlockType.ENCODED_DATA) { - // Save the unencoded block in case we need to cache it on write. - // We don't know the final on-disk size at this point, because - // compression has not been done yet, to set it to uncompressed size - // and override later. - int uncompressedSizeWithoutHeader = getUncompressedSizeWithoutHeader(); - unencodedDataBlockForCaching = new HFileBlock(blockType, - uncompressedSizeWithoutHeader, // will override this later - uncompressedSizeWithoutHeader, prevOffset, - getUncompressedBufferWithHeader(), FILL_HEADER, startOffset, - includesMemstoreTS); - uncompressedBytesWithHeader = encodingResult.getFirst().array(); - blockType = encodedBlockType; - } else { - // There is no encoding configured. Do some extra sanity-checking. - if (encodedBlockType != BlockType.DATA) { - throw new IOException("Unexpected block type coming out of data " + - "block encoder: " + encodedBlockType); - } - if (userDataStream.size() != + if (uncompressedSizeWithoutHeader != uncompressedBytesWithHeader.length - HEADER_SIZE) { throw new IOException("Uncompressed size mismatch: " - + userDataStream.size() + " vs. " + + uncompressedSizeWithoutHeader + " vs. " + (uncompressedBytesWithHeader.length - HEADER_SIZE)); } + + // Write the header into the beginning of the uncompressed byte array. + putHeader(uncompressedBytesWithHeader, 0); } } - /** - * Put the header into the given byte array at the given offset. - * @param onDiskSize size of the block on disk - * @param uncompressedSize size of the block after decompression (but - * before optional data block decoding) - */ - private void putHeader(byte[] dest, int offset, int onDiskSize, - int uncompressedSize) { + /** Put the header into the given byte array at the given offset. */ + private void putHeader(byte[] dest, int offset) { offset = blockType.put(dest, offset); - offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE); - offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE); + offset = Bytes.putInt(dest, offset, onDiskBytesWithHeader.length + - HEADER_SIZE); + offset = Bytes.putInt(dest, offset, uncompressedSizeWithoutHeader); Bytes.putLong(dest, offset, prevOffset); } @@ -884,7 +793,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ public int getUncompressedSizeWithoutHeader() { expectState(State.BLOCK_READY); - return uncompressedBytesWithHeader.length - HEADER_SIZE; + return uncompressedSizeWithoutHeader; } /** @@ -892,7 +801,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ public int getUncompressedSizeWithHeader() { expectState(State.BLOCK_READY); - return uncompressedBytesWithHeader.length; + return uncompressedSizeWithoutHeader + HEADER_SIZE; } /** @return true if a block is being written */ @@ -923,6 +832,15 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { private byte[] getUncompressedDataWithHeader() { expectState(State.BLOCK_READY); + if (compressAlgo == NONE) + return onDiskBytesWithHeader; + + if (!cacheOnWrite) + throw new IllegalStateException("Cache-on-write is turned off"); + + if (uncompressedBytesWithHeader == null) + throw new NullPointerException(); + return uncompressedBytesWithHeader; } @@ -956,18 +874,14 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ public void writeBlock(BlockWritable bw, FSDataOutputStream out) throws IOException { - bw.writeToBlock(startWriting(bw.getBlockType())); + bw.writeToBlock(startWriting(bw.getBlockType(), false)); writeHeaderAndData(out); } public HFileBlock getBlockForCaching() { - if (unencodedDataBlockForCaching != null) { - return unencodedDataBlockForCaching; - } - return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(), - getUncompressedSizeWithoutHeader(), prevOffset, - getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset, - includesMemstoreTS); + return new HFileBlock(blockType, onDiskBytesWithHeader.length + - HEADER_SIZE, uncompressedSizeWithoutHeader, prevOffset, + getUncompressedBufferWithHeader(), false, startOffset); } } @@ -1049,18 +963,14 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { /** The size of the file we are reading from, or -1 if unknown. */ protected long fileSize; - /** Data block encoding used to read from file */ - protected HFileDataBlockEncoder dataBlockEncoder; - /** The default buffer size for our buffered streams */ public static final int DEFAULT_BUFFER_SIZE = 1 << 20; public AbstractFSReader(FSDataInputStream istream, Algorithm compressAlgo, - long fileSize, HFileDataBlockEncoder dataBlockEncoder) { + long fileSize) { this.istream = istream; this.compressAlgo = compressAlgo; this.fileSize = fileSize; - this.dataBlockEncoder = dataBlockEncoder; } @Override @@ -1223,12 +1133,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { public FSReaderV1(FSDataInputStream istream, Algorithm compressAlgo, long fileSize) { - this(istream, compressAlgo, fileSize, new NoOpDataBlockEncoder()); - } - - public FSReaderV1(FSDataInputStream istream, Algorithm compressAlgo, - long fileSize, HFileDataBlockEncoder blockEncoder) { - super(istream, compressAlgo, fileSize, blockEncoder); + super(istream, compressAlgo, fileSize); } /** @@ -1251,8 +1156,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { */ @Override public HFileBlock readBlockData(long offset, long onDiskSizeWithMagic, - int uncompressedSizeWithMagic, boolean pread) - throws IOException { + int uncompressedSizeWithMagic, boolean pread) throws IOException { if (uncompressedSizeWithMagic <= 0) { throw new IOException("Invalid uncompressedSize=" + uncompressedSizeWithMagic + " for a version 1 block"); @@ -1310,8 +1214,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { // to the size of the data portion of the block without the magic record, // since the magic record gets moved to the header. HFileBlock b = new HFileBlock(newBlockType, onDiskSizeWithoutHeader, - uncompressedSizeWithMagic - MAGIC_LENGTH, -1L, buf, FILL_HEADER, - offset, MemStore.NO_PERSISTENT_TS); + uncompressedSizeWithMagic - MAGIC_LENGTH, -1L, buf, true, offset); return b; } } @@ -1329,9 +1232,6 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { /** Reads version 2 blocks from the filesystem. */ public static class FSReaderV2 extends AbstractFSReader { - /** Whether we include memstore timestamp in data blocks */ - protected boolean includesMemstoreTS; - private ThreadLocal prefetchedHeaderForThread = new ThreadLocal() { @Override @@ -1342,12 +1242,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { public FSReaderV2(FSDataInputStream istream, Algorithm compressAlgo, long fileSize) { - this(istream, compressAlgo, fileSize, new NoOpDataBlockEncoder()); - } - - public FSReaderV2(FSDataInputStream istream, Algorithm compressAlgo, - long fileSize, HFileDataBlockEncoder dataBlockEncoder) { - super(istream, compressAlgo, fileSize, dataBlockEncoder); + super(istream, compressAlgo, fileSize); } /** @@ -1543,13 +1438,6 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { } } } - - b.includesMemstoreTS = includesMemstoreTS; - - if (b.getBlockType() == BlockType.ENCODED_DATA) { - b = dataBlockEncoder.afterReadFromDisk(b); - } - b.offset = offset; return b; } @@ -1563,10 +1451,6 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { prefetchedHeader.header, 0, HEADER_SIZE); } - void setIncludesMemstoreTS(boolean enabled) { - includesMemstoreTS = enabled; - } - } @Override @@ -1634,9 +1518,5 @@ public class HFileBlock extends SchemaConfigured implements Cacheable { return true; } - public boolean doesIncludeMemstoreTS() { - return includesMemstoreTS; - } - } diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index 033eb870442..3f6ccb6fece 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -210,8 +210,7 @@ public class HFileBlockIndex { } // Found a data block, break the loop and check our level in the tree. - if (block.getBlockType().equals(BlockType.DATA) || - block.getBlockType().equals(BlockType.ENCODED_DATA)) { + if (block.getBlockType().equals(BlockType.DATA)) { break; } @@ -734,8 +733,8 @@ public class HFileBlockIndex { long rootLevelIndexPos = out.getPos(); { - DataOutput blockStream = - blockWriter.startWriting(BlockType.ROOT_INDEX); + DataOutput blockStream = blockWriter.startWriting(BlockType.ROOT_INDEX, + false); rootChunk.writeRoot(blockStream); if (midKeyMetadata != null) blockStream.write(midKeyMetadata); @@ -830,7 +829,7 @@ public class HFileBlockIndex { BlockIndexChunk parent, BlockIndexChunk curChunk) throws IOException { long beginOffset = out.getPos(); DataOutputStream dos = blockWriter.startWriting( - BlockType.INTERMEDIATE_INDEX); + BlockType.INTERMEDIATE_INDEX, cacheOnWrite()); curChunk.writeNonRoot(dos); byte[] curFirstKey = curChunk.getBlockKey(0); blockWriter.writeHeaderAndData(out); diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoder.java deleted file mode 100644 index 801ec653a2a..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoder.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.hfile; - -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.util.Pair; - -/** - * Controls what kind of data block encoding is used. If data block encoding is - * not set, methods should just return unmodified block. All of the methods do - * something meaningful if BlockType is DATA_BLOCK or ENCODED_DATA. Otherwise - * they just return the unmodified block. - *

    - * Read path: [parsed from disk] -> {@link #afterReadFromDisk(HFileBlock)} -> - * [caching] -> - * {@link #afterReadFromDiskAndPuttingInCache(HFileBlock, boolean)} -> [used - * somewhere] - *

    - * where [caching] looks: - *

    - * ------------------------------------>
    - *   \----> {@link #beforeBlockCache(HFileBlock)}
    - * 
    - *

    - * Write path: [sorted KeyValues have been created] -> - * {@link #beforeWriteToDisk(ByteBuffer)} -> [(optional) compress] -> [write to - * disk] - *

    - * Reading from cache path: [get from cache] -> - * {@link #afterBlockCache(HFileBlock, boolean)} - *

    - * Storing data in file info: {@link #saveMetadata(StoreFile.Writer)} - *

    - * Creating algorithm specific Scanner: {@link #useEncodedScanner()} - */ -public interface HFileDataBlockEncoder { - /** - * Should be called after each HFileBlock of type DATA_BLOCK or - * ENCODED_DATA_BLOCK is read from disk, but before it is put into the cache. - * @param block Block read from HFile stored on disk. - * @return non null block which is coded according to the settings. - */ - public HFileBlock afterReadFromDisk(HFileBlock block); - - /** - * Should be called after each HFileBlock of type DATA_BLOCK or - * ENCODED_DATA_BLOCK is read from disk and after it is saved in cache - * @param block Block read from HFile stored on disk. - * @param isCompaction Will block be used for compaction. - * @return non null block which is coded according to the settings. - */ - public HFileBlock afterReadFromDiskAndPuttingInCache(HFileBlock block, - boolean isCompaction, boolean includesMemsoreTS); - - /** - * Should be called before an encoded or unencoded data block is written to - * disk. - * @param in KeyValues next to each other - * @return a non-null on-heap buffer containing the contents of the - * HFileBlock with unfilled header and block type - */ - public Pair beforeWriteToDisk( - ByteBuffer in, boolean includesMemstoreTS); - - /** - * Should always be called before putting a block into cache. - * @param block block that needs to be put into cache. - * @return the block to put into cache instead (possibly the same) - */ - public HFileBlock beforeBlockCache(HFileBlock block, - boolean includesMemstoreTS); - - /** - * After getting block from cache. - * @param block block which was returned from cache, may be null. - * @param isCompaction Will block be used for compaction. - * @param includesMemstoreTS whether we have a memstore timestamp encoded - * as a variable-length integer after each key-value pair - * @return HFileBlock to use. Can be null, even if argument is not null. - */ - public HFileBlock afterBlockCache(HFileBlock block, - boolean isCompaction, boolean includesMemstoreTS); - - /** - * Should special version of scanner be used. - * @param isCompaction Will scanner be used for compaction. - * @return Whether to use encoded scanner. - */ - public boolean useEncodedScanner(boolean isCompaction); - - /** - * Save metadata in StoreFile which will be written to disk - * @param storeFileWriter writer for a given StoreFile - * @exception IOException on disk problems - */ - public void saveMetadata(StoreFile.Writer storeFileWriter) throws IOException; -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java deleted file mode 100644 index 610d3dbf524..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java +++ /dev/null @@ -1,366 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.hfile; - -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings.Algorithm; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.zookeeper.server.ByteBufferInputStream; - -/** - * Do different kinds of data block encoding according to column family - * options. - */ -public class HFileDataBlockEncoderImpl implements HFileDataBlockEncoder { - private final DataBlockEncodings.Algorithm onDisk; - private final DataBlockEncodings.Algorithm inCache; - - public static final boolean NO_ENCODED_SEEK = false; - - private final boolean encodedSeek; - - /** - * Do data block encoding as with specified options. - * @param onDisk What kind of data block encoding will be used before writing - * HFileBlock to disk. - * @param inCache What kind of data block encoding will be used in block - * cache. - * @param encodedSeek should we seek over encoded data blocks (true) or - * decode blocks first and use normal seek operations (false) - */ - public HFileDataBlockEncoderImpl(Algorithm onDisk, Algorithm inCache, - boolean encodedSeek) { - this.onDisk = onDisk != null ? - onDisk : DataBlockEncodings.Algorithm.NONE; - this.inCache = inCache != null ? - inCache : DataBlockEncodings.Algorithm.NONE; - this.encodedSeek = encodedSeek; - } - - /** - * @return the data block encoding algorithm used on disk - */ - public DataBlockEncodings.Algorithm getOnDisk() { - return onDisk; - } - - /** - * @return the data block encoding algorithm used in the block cache - */ - public DataBlockEncodings.Algorithm getInCache() { - return inCache; - } - - /** - * @return whether we should do seek operations on encoded blocks - */ - public boolean useEncodedSeek() { - return encodedSeek - && inCache != DataBlockEncodings.Algorithm.NONE; - } - - // Preconditions: any HFileBlock format - // Postconditions: HFileBlock not encoded - // or encoded same format as inCache - @Override - public HFileBlock afterReadFromDisk(HFileBlock block) { - if (ignoreBlock(block)) { - return block; // non DATA block, skip it - } - - // is already encoded in desired encoding - if (block.getBlockType() == BlockType.ENCODED_DATA && - block.getDataBlockEncodingId() == inCache.getId()) { - return block; - } - - // decode if we need it - HFileBlock decompressedBlock; - if (block.getBlockType() == BlockType.ENCODED_DATA) { - decompressedBlock = decodeDataBlock(block, false, (short) 0, - block.doesIncludeMemstoreTS()); - } else { - decompressedBlock = block; - } - - // check if we want to encode it here - if (encodedSeek && inCache != DataBlockEncodings.Algorithm.NONE && - onDisk != DataBlockEncodings.Algorithm.NONE) { - return encodeDataBlock(decompressedBlock, inCache, - block.doesIncludeMemstoreTS()); - } - - return decompressedBlock; - } - - /** - * Preconditions: HFileBlock not encoded or encoded in the {@link #inCache} - * format. - *

    - * Postconditions: - *

      - *
    • if isCompaction is set and {@link #onDisk} is NONE there is no - * encoding
    • - *
    • if {@link #encodedSeek} is set there is same encoding as inCache - * Otherwise there is no encoding
    • - *
    - */ - @Override - public HFileBlock afterReadFromDiskAndPuttingInCache(HFileBlock block, - boolean isCompaction, boolean includesMemstoreTS) { - if (ignoreBlock(block)) { - return block; // non DATA block, skip it - } - - // use decoded buffer in case of compaction - if (dontEncodeBeforeCompaction(isCompaction)) { - if (block.getBlockType() != BlockType.DATA) { - return decodeDataBlock(block, true, inCache.getId(), - includesMemstoreTS); - } - return block; - } - - if (!encodedSeek) { - // we need to have it decoded in memory - if (block.getBlockType() != BlockType.DATA) { - return decodeDataBlock(block, true, inCache.getId(), - includesMemstoreTS); - } - return block; - } - - // got already data in desired format - if (block.getBlockType() == BlockType.ENCODED_DATA && - block.getDataBlockEncodingId() == inCache.getId()) { - return block; - } - - if (block.getBlockType() == BlockType.ENCODED_DATA) { - throw new IllegalStateException("Unexpected encoding"); - } - - // need to encode it - if (inCache != DataBlockEncodings.Algorithm.NONE) { - return encodeDataBlock(block, inCache, includesMemstoreTS); - } - - return block; - } - - // Precondition: not encoded buffer - // Postcondition: same encoding as onDisk - @Override - public Pair beforeWriteToDisk(ByteBuffer in, - boolean includesMemstoreTS) { - if (onDisk == DataBlockEncodings.Algorithm.NONE) { - // there is no need to encode the block before writing it to disk - return new Pair(in, BlockType.DATA); - } - - ByteBuffer encodedBuffer = encodeBufferToHFileBlockBuffer(in, - onDisk, includesMemstoreTS); - return new Pair(encodedBuffer, - BlockType.ENCODED_DATA); - } - - // Precondition: an unencoded block or the same encoding as inCache - // Postcondition: same encoding as inCache - @Override - public HFileBlock beforeBlockCache(HFileBlock block, - boolean includesMemstoreTS) { - if (ignoreBlock(block)) { - return block; // non DATA block skip it - } - - if (block.getBlockType() == BlockType.ENCODED_DATA) { - if (block.getDataBlockEncodingId() == inCache.getId()) { - // is already encoded in right format - return block; - } - - // expecting either the "in-cache" encoding or no encoding - throw new IllegalStateException(String.format( - "Expected the in-cache encoding ('%s') or no encoding, " + - "but got encoding '%s'", inCache.toString(), - DataBlockEncodings.getNameFromId( - block.getDataBlockEncodingId()))); - } - - if (inCache != DataBlockEncodings.Algorithm.NONE) { - // encode data - HFileBlock encodedBlock = encodeDataBlock(block, inCache, - includesMemstoreTS); - block.passSchemaMetricsTo(encodedBlock); - return encodedBlock; - } - - return block; - } - - /** - * Precondition: same encoding as in inCache - *

    - * Postcondition: if (isCompaction is set and {@link #onDisk} is not NONE) or - * {@link #encodedSeek} is not set -> don't encode. - */ - @Override - public HFileBlock afterBlockCache(HFileBlock block, boolean isCompaction, - boolean includesMemstoreTS) { - if (block == null || ignoreBlock(block)) { - return block; // skip no DATA block - } - - if (inCache == DataBlockEncodings.Algorithm.NONE) { - // no need of decoding - if (block.getBlockType() == BlockType.ENCODED_DATA) { - throw new IllegalStateException("Expected non-encoded data in cache."); - } - return block; - } - - if (block.getBlockType() != BlockType.ENCODED_DATA) { - throw new IllegalStateException("Expected encoded data in cache."); - } - - if (dontEncodeBeforeCompaction(isCompaction)) { - // If we don't use dataBlockEncoding on disk, - // we would also avoid using it for compactions. - // That way we don't change disk format. - return null; - } - - if (encodedSeek) { - // we use encoding in memory - return block; - } - - return decodeDataBlock(block, true, inCache.getId(), includesMemstoreTS); - } - - @Override - public boolean useEncodedScanner(boolean isCompaction) { - if (isCompaction && onDisk == DataBlockEncodings.Algorithm.NONE) { - return false; - } - return encodedSeek && inCache != DataBlockEncodings.Algorithm.NONE; - } - - @Override - public void saveMetadata(StoreFile.Writer storeFileWriter) - throws IOException { - storeFileWriter.appendFileInfo(StoreFile.DATA_BLOCK_ENCODING, - onDisk.getNameInBytes()); - } - - private HFileBlock decodeDataBlock(HFileBlock block, boolean verifyEncoding, - short expectedEncoderId, boolean includesMemstoreTS) { - assert block.getBlockType() == BlockType.ENCODED_DATA; - short dataBlockEncoderId = block.getDataBlockEncodingId(); - - // (optional) sanity check of encoder type - if (verifyEncoding && expectedEncoderId != dataBlockEncoderId) { - throw new IllegalStateException(String.format( - "Expected encoding type '%d', but found '%d'", - expectedEncoderId, dataBlockEncoderId)); - } - - ByteBuffer originalBuf = block.getBufferReadOnly(); - ByteBuffer withoutEncodedHeader = ByteBuffer.wrap(originalBuf.array(), - originalBuf.arrayOffset() + HFileBlock.ENCODED_HEADER_SIZE, - originalBuf.limit() - HFileBlock.ENCODED_HEADER_SIZE).slice(); - ByteBufferInputStream bbis = - new ByteBufferInputStream(withoutEncodedHeader); - DataInputStream dis; - ByteBuffer newBuf; - DataBlockEncoder dataBlockEncoder = null; - - try { - dis = new DataInputStream(bbis); - dataBlockEncoder = - DataBlockEncodings.getDataBlockEncoderFromId(dataBlockEncoderId); - int preReadLength = originalBuf.limit() - - HFileBlock.HEADER_SIZE - block.getUncompressedSizeWithoutHeader(); - // Sometimes buffer is larger, because it also contains next's block - // header. In that case we want to skip it. - newBuf = dataBlockEncoder.uncompressKeyValues(dis, HFileBlock.HEADER_SIZE, - preReadLength, includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException(String.format( - "Bug while decoding the block using '%s'", dataBlockEncoder), e); - } - - // Create a decoded HFileBlock. Offset will be set later. - return new HFileBlock(BlockType.DATA, block.getOnDiskSizeWithoutHeader(), - newBuf.limit() - HFileBlock.HEADER_SIZE, block.getPrevBlockOffset(), - newBuf, HFileBlock.FILL_HEADER, 0, includesMemstoreTS); - } - - private ByteBuffer encodeBufferToHFileBlockBuffer(ByteBuffer in, - DataBlockEncodings.Algorithm algo, boolean includesMemstoreTS) { - ByteArrayOutputStream encodedStream = new ByteArrayOutputStream(); - DataOutputStream dataOut = new DataOutputStream(encodedStream); - DataBlockEncoder encoder = algo.getEncoder(); - try { - encodedStream.write(HFileBlock.DUMMY_HEADER); - algo.writeIdInBytes(dataOut); - encoder.compressKeyValues(dataOut, in, - includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException(String.format("Bug in data block encoder " + - "'%s', it probably requested too much data", algo.toString()), e); - } - return ByteBuffer.wrap(encodedStream.toByteArray()); - } - - private HFileBlock encodeDataBlock(HFileBlock block, - DataBlockEncodings.Algorithm algo, boolean includesMemstoreTS) { - ByteBuffer compressedBuffer = encodeBufferToHFileBlockBuffer( - block.getBufferWithoutHeader(), algo, includesMemstoreTS); - int sizeWithoutHeader = compressedBuffer.limit() - HFileBlock.HEADER_SIZE; - return new HFileBlock(BlockType.ENCODED_DATA, - block.getOnDiskSizeWithoutHeader(), - sizeWithoutHeader, block.getPrevBlockOffset(), - compressedBuffer, HFileBlock.FILL_HEADER, block.getOffset(), - includesMemstoreTS); - } - - private boolean ignoreBlock(HFileBlock block) { - BlockType type = block.getBlockType(); - return type != BlockType.DATA && type != BlockType.ENCODED_DATA; - } - - private boolean dontEncodeBeforeCompaction(boolean isCompaction) { - return isCompaction - && onDisk == DataBlockEncodings.Algorithm.NONE; - } - - @Override - public String toString() { - return String.format(getClass().getSimpleName() - + " onDisk='%s' inCache='%s' encodedSeek=%s", onDisk.toString(), - inCache.toString(), encodedSeek); - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index b68e2af66e1..faca02da15c 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.BloomFilter; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.ByteBloomFilter; @@ -171,7 +170,6 @@ public class HFilePrettyPrinter { conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); conf.set("fs.default.name", conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR)); - SchemaMetrics.configureGlobally(conf); try { if (!parseOptions(args)) return 1; diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java index a175fdb0386..7d63c4f123a 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java @@ -30,12 +30,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.io.hfile.HFile.Reader; import org.apache.hadoop.hbase.io.hfile.HFile.Writer; -import org.apache.hadoop.hbase.regionserver.MemStore; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.IOUtils; @@ -59,16 +57,12 @@ public class HFileReaderV1 extends AbstractHFileReader { * stream. * @param size Length of the stream. * @param cacheConf cache references and configuration - * @param blockEncoder what kind of data block encoding will be used - * @throws IOException */ public HFileReaderV1(Path path, FixedFileTrailer trailer, final FSDataInputStream fsdis, final long size, final boolean closeIStream, - final CacheConfig cacheConf, - final HFileDataBlockEncoder blockEncoder) { - super(path, trailer, fsdis, size, closeIStream, cacheConf, - blockEncoder); + final CacheConfig cacheConf) { + super(path, trailer, fsdis, size, closeIStream, cacheConf); trailer.expectVersion(1); fsBlockReader = new HFileBlock.FSReaderV1(fsdis, compressAlgo, fileSize); @@ -173,9 +167,6 @@ public class HFileReaderV1 extends AbstractHFileReader { @Override public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction) { - if (blockEncoder.useEncodedScanner(isCompaction)) { - return new EncodedScannerV1(this, cacheBlocks, pread, isCompaction); - } return new ScannerV1(this, cacheBlocks, pread, isCompaction); } @@ -304,8 +295,6 @@ public class HFileReaderV1 extends AbstractHFileReader { HFileBlock cachedBlock = (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheConf.shouldCacheDataOnRead()); - cachedBlock = blockEncoder.afterBlockCache(cachedBlock, - isCompaction, MemStore.NO_PERSISTENT_TS); if (cachedBlock != null) { cacheHits.incrementAndGet(); getSchemaMetrics().updateOnCacheHit(cachedBlock.getBlockType().getCategory(), @@ -333,7 +322,7 @@ public class HFileReaderV1 extends AbstractHFileReader { - offset, dataBlockIndexReader.getRootBlockDataSize(block), pread); passSchemaMetricsTo(hfileBlock); hfileBlock.expectType(BlockType.DATA); - hfileBlock = blockEncoder.afterReadFromDisk(hfileBlock); + ByteBuffer buf = hfileBlock.getBufferWithoutHeader(); long delta = System.nanoTime() - startTimeNs; if (pread) { @@ -349,17 +338,10 @@ public class HFileReaderV1 extends AbstractHFileReader { // Cache the block if (cacheBlock && cacheConf.shouldCacheBlockOnRead( hfileBlock.getBlockType().getCategory())) { - hfileBlock = blockEncoder.beforeBlockCache(hfileBlock, - MemStore.NO_PERSISTENT_TS); cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock, cacheConf.isInMemory()); } - hfileBlock = blockEncoder.afterReadFromDiskAndPuttingInCache( - hfileBlock, isCompaction, MemStore.NO_PERSISTENT_TS); - - ByteBuffer buf = hfileBlock.getBufferWithoutHeader(); - return buf; } } @@ -414,101 +396,16 @@ public class HFileReaderV1 extends AbstractHFileReader { } } - protected abstract static class AbstractScannerV1 - extends AbstractHFileReader.Scanner { - protected final HFileReaderV1 readerV1; - protected int currBlock; - - public AbstractScannerV1(HFileReaderV1 reader, boolean cacheBlocks, - final boolean pread, final boolean isCompaction) { - super(reader, cacheBlocks, pread, isCompaction); - readerV1 = reader; - } - - /** - * Within a loaded block, seek looking for the first key - * that is smaller than (or equal to?) the key we are interested in. - * - * A note on the seekBefore - if you have seekBefore = true, AND the - * first key in the block = key, then you'll get thrown exceptions. - * @param key to find - * @param seekBefore find the key before the exact match. - * @return - */ - protected abstract int blockSeek(byte[] key, int offset, int length, - boolean seekBefore); - - protected abstract void loadBlock(int bloc, boolean rewind) - throws IOException; - - @Override - public int seekTo(byte[] key, int offset, int length) throws IOException { - int b = readerV1.blockContainingKey(key, offset, length); - if (b < 0) return -1; // falls before the beginning of the file! :-( - // Avoid re-reading the same block (that'd be dumb). - loadBlock(b, true); - return blockSeek(key, offset, length, false); - } - - @Override - public int reseekTo(byte[] key, int offset, int length) - throws IOException { - if (blockBuffer != null && currKeyLen != 0) { - ByteBuffer bb = getKey(); - int compared = reader.getComparator().compare(key, offset, - length, bb.array(), bb.arrayOffset(), bb.limit()); - if (compared < 1) { - // If the required key is less than or equal to current key, then - // don't do anything. - return compared; - } - } - - int b = readerV1.blockContainingKey(key, offset, length); - if (b < 0) { - return -1; - } - loadBlock(b, false); - return blockSeek(key, offset, length, false); - } - - @Override - public boolean seekBefore(byte[] key, int offset, int length) - throws IOException { - int b = readerV1.blockContainingKey(key, offset, length); - if (b < 0) - return false; // key is before the start of the file. - - // Question: does this block begin with 'key'? - byte[] firstkKey = reader.getDataBlockIndexReader().getRootBlockKey(b); - if (reader.getComparator().compare(firstkKey, 0, firstkKey.length, - key, offset, length) == 0) { - // Ok the key we're interested in is the first of the block, so go back - // by one. - if (b == 0) { - // we have a 'problem', the key we want is the first of the file. - return false; - } - b--; - // TODO shortcut: seek forward in this block to the last key of the - // block. - } - loadBlock(b, true); - blockSeek(key, offset, length, true); - return true; - } - } - /** * Implementation of {@link HFileScanner} interface. */ - - protected static class ScannerV1 extends AbstractScannerV1 { - private HFileReaderV1 reader; + protected static class ScannerV1 extends AbstractHFileReader.Scanner { + private final HFileReaderV1 reader; + private int currBlock; public ScannerV1(HFileReaderV1 reader, boolean cacheBlocks, final boolean pread, final boolean isCompaction) { - super(reader, cacheBlocks, pread, isCompaction); + super(cacheBlocks, pread, isCompaction); this.reader = reader; } @@ -589,7 +486,57 @@ public class HFileReaderV1 extends AbstractHFileReader { } @Override - protected int blockSeek(byte[] key, int offset, int length, + public int seekTo(byte[] key) throws IOException { + return seekTo(key, 0, key.length); + } + + @Override + public int seekTo(byte[] key, int offset, int length) throws IOException { + int b = reader.blockContainingKey(key, offset, length); + if (b < 0) return -1; // falls before the beginning of the file! :-( + // Avoid re-reading the same block (that'd be dumb). + loadBlock(b, true); + return blockSeek(key, offset, length, false); + } + + @Override + public int reseekTo(byte[] key) throws IOException { + return reseekTo(key, 0, key.length); + } + + @Override + public int reseekTo(byte[] key, int offset, int length) + throws IOException { + if (blockBuffer != null && currKeyLen != 0) { + ByteBuffer bb = getKey(); + int compared = reader.getComparator().compare(key, offset, + length, bb.array(), bb.arrayOffset(), bb.limit()); + if (compared <= 0) { + // If the required key is less than or equal to current key, then + // don't do anything. + return compared; + } + } + + int b = reader.blockContainingKey(key, offset, length); + if (b < 0) { + return -1; + } + loadBlock(b, false); + return blockSeek(key, offset, length, false); + } + + /** + * Within a loaded block, seek looking for the first key + * that is smaller than (or equal to?) the key we are interested in. + * + * A note on the seekBefore - if you have seekBefore = true, AND the + * first key in the block = key, then you'll get thrown exceptions. + * @param key to find + * @param seekBefore find the key before the exact match. + * @return + */ + private int blockSeek(byte[] key, int offset, int length, boolean seekBefore) { int klen, vlen; int lastLen = 0; @@ -630,6 +577,37 @@ public class HFileReaderV1 extends AbstractHFileReader { return 1; // didn't exactly find it. } + @Override + public boolean seekBefore(byte[] key) throws IOException { + return seekBefore(key, 0, key.length); + } + + @Override + public boolean seekBefore(byte[] key, int offset, int length) + throws IOException { + int b = reader.blockContainingKey(key, offset, length); + if (b < 0) + return false; // key is before the start of the file. + + // Question: does this block begin with 'key'? + byte[] firstkKey = reader.getDataBlockIndexReader().getRootBlockKey(b); + if (reader.getComparator().compare(firstkKey, 0, firstkKey.length, + key, offset, length) == 0) { + // Ok the key we're interested in is the first of the block, so go back + // by one. + if (b == 0) { + // we have a 'problem', the key we want is the first of the file. + return false; + } + b--; + // TODO shortcut: seek forward in this block to the last key of the + // block. + } + loadBlock(b, true); + blockSeek(key, offset, length, true); + return true; + } + @Override public String getKeyString() { return Bytes.toStringBinary(blockBuffer.array(), @@ -642,6 +620,11 @@ public class HFileReaderV1 extends AbstractHFileReader { blockBuffer.position() + currKeyLen, currValueLen); } + @Override + public Reader getReader() { + return reader; + } + @Override public boolean seekTo() throws IOException { if (reader.getDataBlockIndexReader().isEmpty()) { @@ -662,8 +645,7 @@ public class HFileReaderV1 extends AbstractHFileReader { return true; } - @Override - protected void loadBlock(int bloc, boolean rewind) throws IOException { + private void loadBlock(int bloc, boolean rewind) throws IOException { if (blockBuffer == null) { blockBuffer = reader.readBlockBuffer(bloc, cacheBlocks, pread, isCompaction); @@ -690,115 +672,6 @@ public class HFileReaderV1 extends AbstractHFileReader { } - protected static class EncodedScannerV1 extends AbstractScannerV1 { - private DataBlockEncoder.EncodedSeeker seeker = null; - private DataBlockEncoder dataBlockEncoder = null; - - public EncodedScannerV1(HFileReaderV1 reader, boolean cacheBlocks, - boolean pread, boolean isCompaction) { - super(reader, cacheBlocks, pread, isCompaction); - } - - @Override - public boolean seekTo() throws IOException { - if (reader.getDataBlockIndexReader().isEmpty()) { - return false; - } - - loadBlock(0, true); - return true; - } - - @Override - public boolean next() throws IOException { - if (blockBuffer == null) { - throw new IOException("Next called on non-seeked scanner"); - } - - boolean ok = seeker.next(); - - if (!ok) { - if (currBlock + 1 >= - reader.getDataBlockIndexReader().getRootBlockCount()) { - // damn we are at the end - currBlock = 0; - blockBuffer = null; - return false; - } - loadBlock(currBlock + 1, false); - ok = true; - } - - return ok; - } - - @Override - public ByteBuffer getKey() { - return seeker.getKey(); - } - - @Override - public ByteBuffer getValue() { - return seeker.getValue(); - } - - @Override - public KeyValue getKeyValue() { - if (blockBuffer == null) { - return null; - } - return seeker.getKeyValueObject(); - } - - @Override - public String getKeyString() { - ByteBuffer keyBuffer = seeker.getKey(); - return Bytes.toStringBinary(keyBuffer.array(), - keyBuffer.arrayOffset(), keyBuffer.limit()); - } - - @Override - public String getValueString() { - ByteBuffer valueBuffer = seeker.getValue(); - return Bytes.toStringBinary(valueBuffer.array(), - valueBuffer.arrayOffset(), valueBuffer.limit()); - } - - @Override - protected int blockSeek(byte[] key, int offset, int length, - boolean seekBefore) { - return seeker.blockSeekTo(key, offset, length, seekBefore); - } - - @Override - protected void loadBlock(int bloc, boolean rewind) throws IOException { - if (blockBuffer == null || bloc != currBlock) { - blockBuffer = readerV1.readBlockBuffer(bloc, cacheBlocks, pread, - isCompaction); - currBlock = bloc; - blockFetches++; - short dataBlockEncoderId = blockBuffer.getShort(); - blockBuffer = blockBuffer.slice(); - - if (seeker == null || - DataBlockEncodings.isCorrectEncoder( - dataBlockEncoder, dataBlockEncoderId)) { - dataBlockEncoder = - DataBlockEncodings.getDataBlockEncoderFromId(dataBlockEncoderId); - seeker = dataBlockEncoder.createSeeker(reader.getComparator(), - MemStore.NO_PERSISTENT_TS); - } - seeker.setCurrentBuffer(blockBuffer); - - } else { - // we are already in the same block, just rewind to seek again. - if (rewind) { - seeker.rewind(); - } - } - } - } - @Override public HFileBlock readBlock(long offset, long onDiskBlockSize, boolean cacheBlock, boolean pread, boolean isCompaction) { diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java index 0052d8c0990..59033f42e00 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java @@ -30,8 +30,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.util.Bytes; @@ -82,20 +80,17 @@ public class HFileReaderV2 extends AbstractHFileReader { * @param size Length of the stream. * @param closeIStream Whether to close the stream. * @param cacheConf Cache configuration. - * @param blockEncoder what kind of data block encoding will be used * @throws IOException */ public HFileReaderV2(Path path, FixedFileTrailer trailer, final FSDataInputStream fsdis, final long size, - final boolean closeIStream, final CacheConfig cacheConf, - final HFileDataBlockEncoder blockEncoder) + final boolean closeIStream, final CacheConfig cacheConf) throws IOException { - super(path, trailer, fsdis, size, closeIStream, cacheConf, - blockEncoder); + super(path, trailer, fsdis, size, closeIStream, cacheConf); + trailer.expectVersion(2); - HFileBlock.FSReaderV2 fsBlockReader = new HFileBlock.FSReaderV2(fsdis, - compressAlgo, fileSize, blockEncoder); - this.fsBlockReader = fsBlockReader; + fsBlockReader = new HFileBlock.FSReaderV2(fsdis, compressAlgo, + fileSize); // Comparator class name is stored in the trailer in version 2. comparator = trailer.createComparator(); @@ -128,10 +123,8 @@ public class HFileReaderV2 extends AbstractHFileReader { avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN)); avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN)); byte [] keyValueFormatVersion = fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION); - includesMemstoreTS = keyValueFormatVersion != null && - Bytes.toInt(keyValueFormatVersion) == - HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE; - fsBlockReader.setIncludesMemstoreTS(includesMemstoreTS); + includesMemstoreTS = (keyValueFormatVersion != null && + Bytes.toInt(keyValueFormatVersion) == HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE); // Store all other load-on-open blocks for further consumption. HFileBlock b; @@ -152,15 +145,9 @@ public class HFileReaderV2 extends AbstractHFileReader { * @param isCompaction is scanner being used for a compaction? * @return Scanner on this file. */ - @Override - public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, + @Override + public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction) { - // check if we want to use data block encoding in memory - if (blockEncoder.useEncodedScanner(isCompaction)) { - return new EncodedScannerV2(this, cacheBlocks, pread, isCompaction, - includesMemstoreTS); - } - return new ScannerV2(this, cacheBlocks, pread, isCompaction); } @@ -271,8 +258,6 @@ public class HFileReaderV2 extends AbstractHFileReader { if (cacheConf.isBlockCacheEnabled()) { HFileBlock cachedBlock = (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock); - cachedBlock = blockEncoder.afterBlockCache(cachedBlock, - isCompaction, shouldIncludeMemstoreTS()); if (cachedBlock != null) { BlockCategory blockCategory = cachedBlock.getBlockType().getCategory(); @@ -280,9 +265,8 @@ public class HFileReaderV2 extends AbstractHFileReader { getSchemaMetrics().updateOnCacheHit(blockCategory, isCompaction); - if (cachedBlock.getBlockType() == BlockType.DATA) { + if (cachedBlock.getBlockType() == BlockType.DATA) HFile.dataBlockReadCnt.incrementAndGet(); - } return cachedBlock; } // Carry on, please load. @@ -308,8 +292,6 @@ public class HFileReaderV2 extends AbstractHFileReader { // Cache the block if (cacheBlock && cacheConf.shouldCacheBlockOnRead( hfileBlock.getBlockType().getCategory())) { - hfileBlock = blockEncoder.beforeBlockCache( - hfileBlock, includesMemstoreTS); cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock, cacheConf.isInMemory()); } @@ -318,8 +300,7 @@ public class HFileReaderV2 extends AbstractHFileReader { HFile.dataBlockReadCnt.incrementAndGet(); } - return blockEncoder.afterReadFromDiskAndPuttingInCache(hfileBlock, - isCompaction, includesMemstoreTS); + return hfileBlock; } finally { offsetLock.releaseLockEntry(lockEntry); } @@ -364,153 +345,24 @@ public class HFileReaderV2 extends AbstractHFileReader { } } - protected abstract static class AbstractScannerV2 - extends AbstractHFileReader.Scanner { - protected HFileBlock block; - - public AbstractScannerV2(HFileReaderV2 r, boolean cacheBlocks, - final boolean pread, final boolean isCompaction) { - super(r, cacheBlocks, pread, isCompaction); - } - - /** - * An internal API function. Seek to the given key, optionally rewinding to - * the first key of the block before doing the seek. - * - * @param key key byte array - * @param offset key offset in the key byte array - * @param length key length - * @param rewind whether to rewind to the first key of the block before - * doing the seek. If this is false, we are assuming we never go - * back, otherwise the result is undefined. - * @return -1 if the key is earlier than the first key of the file, - * 0 if we are at the given key, and 1 if we are past the given key - * @throws IOException - */ - protected int seekTo(byte[] key, int offset, int length, boolean rewind) - throws IOException { - HFileBlockIndex.BlockIndexReader indexReader = - reader.getDataBlockIndexReader(); - HFileBlock seekToBlock = indexReader.seekToDataBlock(key, offset, length, - block, cacheBlocks, pread, isCompaction); - if (seekToBlock == null) { - // This happens if the key e.g. falls before the beginning of the file. - return -1; - } - return loadBlockAndSeekToKey(seekToBlock, rewind, key, offset, length, - false); - } - - protected abstract ByteBuffer getFirstKeyInBlock(HFileBlock curBlock); - - protected abstract int loadBlockAndSeekToKey(HFileBlock seekToBlock, - boolean rewind, byte[] key, int offset, int length, boolean seekBefore) - throws IOException; - - @Override - public int seekTo(byte[] key, int offset, int length) throws IOException { - // Always rewind to the first key of the block, because the given key - // might be before or after the current key. - return seekTo(key, offset, length, true); - } - - @Override - public int reseekTo(byte[] key, int offset, int length) throws IOException { - if (isSeeked()) { - ByteBuffer bb = getKey(); - int compared = reader.getComparator().compare(key, offset, - length, bb.array(), bb.arrayOffset(), bb.limit()); - if (compared < 1) { - // If the required key is less than or equal to current key, then - // don't do anything. - return compared; - } - } - - // Don't rewind on a reseek operation, because reseek implies that we are - // always going forward in the file. - return seekTo(key, offset, length, false); - } - - @Override - public boolean seekBefore(byte[] key, int offset, int length) - throws IOException { - HFileBlock seekToBlock = - reader.getDataBlockIndexReader().seekToDataBlock(key, offset, length, - block, cacheBlocks, pread, isCompaction); - if (seekToBlock == null) { - return false; - } - ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock); - if (reader.getComparator().compare(firstKey.array(), - firstKey.arrayOffset(), firstKey.limit(), key, offset, length) == 0) - { - long previousBlockOffset = seekToBlock.getPrevBlockOffset(); - // The key we are interested in - if (previousBlockOffset == -1) { - // we have a 'problem', the key we want is the first of the file. - return false; - } - - // It is important that we compute and pass onDiskSize to the block - // reader so that it does not have to read the header separately to - // figure out the size. - seekToBlock = reader.readBlock(previousBlockOffset, - seekToBlock.getOffset() - previousBlockOffset, cacheBlocks, - pread, isCompaction); - - // TODO shortcut: seek forward in this block to the last key of the - // block. - } - loadBlockAndSeekToKey(seekToBlock, true, key, offset, length, true); - return true; - } - - - /** - * Scans blocks in the "scanned" section of the {@link HFile} until the next - * data block is found. - * - * @return the next block, or null if there are no more data blocks - * @throws IOException - */ - protected HFileBlock readNextDataBlock() throws IOException { - long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); - if (block == null) - return null; - - HFileBlock curBlock = block; - - do { - if (curBlock.getOffset() >= lastDataBlockOffset) - return null; - - if (curBlock.getOffset() < 0) { - throw new IOException("Invalid block file offset: " + block); - } - curBlock = reader.readBlock(curBlock.getOffset() - + curBlock.getOnDiskSizeWithHeader(), - curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, - isCompaction); - } while (!(curBlock.getBlockType().equals(BlockType.DATA) || - curBlock.getBlockType().equals(BlockType.ENCODED_DATA))); - - return curBlock; - } - } - /** * Implementation of {@link HFileScanner} interface. */ - protected static class ScannerV2 extends AbstractScannerV2 { + protected static class ScannerV2 extends AbstractHFileReader.Scanner { + private HFileBlock block; private HFileReaderV2 reader; public ScannerV2(HFileReaderV2 r, boolean cacheBlocks, final boolean pread, final boolean isCompaction) { - super(r, cacheBlocks, pread, isCompaction); + super(cacheBlocks, pread, isCompaction); this.reader = r; } + @Override + public HFileReaderV2 getReader() { + return reader; + } + @Override public KeyValue getKeyValue() { if (!isSeeked()) @@ -599,6 +451,36 @@ public class HFileReaderV2 extends AbstractHFileReader { return true; } + /** + * Scans blocks in the "scanned" section of the {@link HFile} until the next + * data block is found. + * + * @return the next block, or null if there are no more data blocks + * @throws IOException + */ + private HFileBlock readNextDataBlock() throws IOException { + long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); + if (block == null) + return null; + + HFileBlock curBlock = block; + + do { + if (curBlock.getOffset() >= lastDataBlockOffset) + return null; + + if (curBlock.getOffset() < 0) { + throw new IOException("Invalid block file offset: " + block); + } + curBlock = reader.readBlock(curBlock.getOffset() + + curBlock.getOnDiskSizeWithHeader(), + curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, + isCompaction); + } while (!curBlock.getBlockType().equals(BlockType.DATA)); + + return curBlock; + } + /** * Positions this scanner at the start of the file. * @@ -635,7 +517,70 @@ public class HFileReaderV2 extends AbstractHFileReader { } @Override - protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind, + public int seekTo(byte[] key) throws IOException { + return seekTo(key, 0, key.length); + } + + /** + * An internal API function. Seek to the given key, optionally rewinding to + * the first key of the block before doing the seek. + * + * @param key key byte array + * @param offset key offset in the key byte array + * @param length key length + * @param rewind whether to rewind to the first key of the block before + * doing the seek. If this is false, we are assuming we never go + * back, otherwise the result is undefined. + * @return -1 if the key is earlier than the first key of the file, + * 0 if we are at the given key, and 1 if we are past the given key + * @throws IOException + */ + private int seekTo(byte[] key, int offset, int length, boolean rewind) + throws IOException { + HFileBlockIndex.BlockIndexReader indexReader = + reader.getDataBlockIndexReader(); + HFileBlock seekToBlock = indexReader.seekToDataBlock(key, offset, length, + block, cacheBlocks, pread, isCompaction); + + if (seekToBlock == null) { + // This happens if the key e.g. falls before the beginning of the file. + return -1; + } + return loadBlockAndSeekToKey(seekToBlock, rewind, key, offset, length, + false); + } + + @Override + public int seekTo(byte[] key, int offset, int length) throws IOException { + // Always rewind to the first key of the block, because the given key + // might be before or after the current key. + return seekTo(key, offset, length, true); + } + + @Override + public int reseekTo(byte[] key) throws IOException { + return reseekTo(key, 0, key.length); + } + + @Override + public int reseekTo(byte[] key, int offset, int length) throws IOException { + if (isSeeked()) { + ByteBuffer bb = getKey(); + int compared = reader.getComparator().compare(key, offset, + length, bb.array(), bb.arrayOffset(), bb.limit()); + if (compared < 1) { + // If the required key is less than or equal to current key, then + // don't do anything. + return compared; + } + } + + // Don't rewind on a reseek operation, because reseek implies that we are + // always going forward in the file. + return seekTo(key, offset, length, false); + } + + private int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind, byte[] key, int offset, int length, boolean seekBefore) throws IOException { if (block == null || block.getOffset() != seekToBlock.getOffset()) { @@ -654,13 +599,6 @@ public class HFileReaderV2 extends AbstractHFileReader { */ private void updateCurrBlock(HFileBlock newBlock) { block = newBlock; - - // sanity check - if (block.getBlockType() != BlockType.DATA) { - throw new IllegalStateException( - "ScannerV2 works only on data blocks"); - } - blockBuffer = block.getBufferWithoutHeader(); readKeyValueLen(); blockFetches++; @@ -775,7 +713,11 @@ public class HFileReaderV2 extends AbstractHFileReader { } @Override - protected ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) { + public boolean seekBefore(byte[] key) throws IOException { + return seekBefore(key, 0, key.length); + } + + private ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) { ByteBuffer buffer = curBlock.getBufferWithoutHeader(); // It is safe to manipulate this buffer because we own the buffer object. buffer.rewind(); @@ -787,6 +729,40 @@ public class HFileReaderV2 extends AbstractHFileReader { return keyBuff; } + @Override + public boolean seekBefore(byte[] key, int offset, int length) + throws IOException { + HFileBlock seekToBlock = + reader.getDataBlockIndexReader().seekToDataBlock(key, offset, + length, block, cacheBlocks, pread, isCompaction); + if (seekToBlock == null) { + return false; + } + ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock); + if (reader.getComparator().compare(firstKey.array(), + firstKey.arrayOffset(), firstKey.limit(), key, offset, length) == 0) + { + long previousBlockOffset = seekToBlock.getPrevBlockOffset(); + // The key we are interested in + if (previousBlockOffset == -1) { + // we have a 'problem', the key we want is the first of the file. + return false; + } + + // It is important that we compute and pass onDiskSize to the block + // reader so that it does not have to read the header separately to + // figure out the size. + seekToBlock = reader.readBlock(previousBlockOffset, + seekToBlock.getOffset() - previousBlockOffset, cacheBlocks, + pread, isCompaction); + + // TODO shortcut: seek forward in this block to the last key of the + // block. + } + loadBlockAndSeekToKey(seekToBlock, true, key, offset, length, true); + return true; + } + @Override public String getKeyString() { return Bytes.toStringBinary(blockBuffer.array(), @@ -800,162 +776,7 @@ public class HFileReaderV2 extends AbstractHFileReader { + blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen, currValueLen); } - } - /** - * ScannerV2 which operate on encoded data blocks. - */ - protected static class EncodedScannerV2 extends AbstractScannerV2 { - private DataBlockEncoder.EncodedSeeker seeker = null; - private DataBlockEncoder dataBlockEncoder = null; - private final boolean includesMemstoreTS; - - public EncodedScannerV2(HFileReaderV2 reader, boolean cacheBlocks, - boolean pread, boolean isCompaction, boolean includesMemstoreTS) { - super(reader, cacheBlocks, pread, isCompaction); - this.includesMemstoreTS = includesMemstoreTS; - } - - private void setDataBlockEncoder(DataBlockEncoder dataBlockEncoder) { - this.dataBlockEncoder = dataBlockEncoder; - seeker = dataBlockEncoder.createSeeker(reader.getComparator(), - includesMemstoreTS); - } - - /** - * Updates the current block to be the given {@link HFileBlock}. Seeks to - * the the first key/value pair. - * - * @param newBlock the block to make current - */ - private void updateCurrentBlock(HFileBlock newBlock) { - block = newBlock; - - // sanity checks - if (block.getBlockType() != BlockType.ENCODED_DATA) { - throw new IllegalStateException( - "EncodedScannerV2 works only on encoded data blocks"); - } - - short dataBlockEncoderId = block.getDataBlockEncodingId(); - if (dataBlockEncoder == null - || !DataBlockEncodings.isCorrectEncoder(dataBlockEncoder, - dataBlockEncoderId)) { - DataBlockEncoder encoder = - DataBlockEncodings.getDataBlockEncoderFromId(dataBlockEncoderId); - setDataBlockEncoder(encoder); - } - - seeker.setCurrentBuffer(getEncodedBuffer(newBlock)); - blockFetches++; - } - - private ByteBuffer getEncodedBuffer(HFileBlock newBlock) { - ByteBuffer origBlock = newBlock.getBufferReadOnly(); - ByteBuffer encodedBlock = ByteBuffer.wrap(origBlock.array(), - origBlock.arrayOffset() + HFileBlock.HEADER_SIZE + - DataBlockEncodings.ID_SIZE, - origBlock.limit() - HFileBlock.HEADER_SIZE - - DataBlockEncodings.ID_SIZE).slice(); - return encodedBlock; - } - - @Override - public boolean seekTo() throws IOException { - if (reader == null) { - return false; - } - - if (reader.getTrailer().getEntryCount() == 0) { - // No data blocks. - return false; - } - - long firstDataBlockOffset = - reader.getTrailer().getFirstDataBlockOffset(); - if (block != null && block.getOffset() == firstDataBlockOffset) { - seeker.rewind(); - return true; - } - - block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, - isCompaction); - if (block.getOffset() < 0) { - throw new IOException("Invalid block offset: " + block.getOffset()); - } - updateCurrentBlock(block); - return true; - } - - @Override - public boolean next() throws IOException { - boolean isValid = seeker.next(); - if (!isValid) { - block = readNextDataBlock(); - isValid = block != null; - if (isValid) { - updateCurrentBlock(block); - } - } - return isValid; - } - - @Override - public ByteBuffer getKey() { - assertValidSeek(); - return seeker.getKey(); - } - - @Override - public ByteBuffer getValue() { - assertValidSeek(); - return seeker.getValue(); - } - - @Override - public KeyValue getKeyValue() { - if (block == null) { - return null; - } - return seeker.getKeyValueObject(); - } - - @Override - public String getKeyString() { - ByteBuffer keyBuffer = getKey(); - return Bytes.toStringBinary(keyBuffer.array(), - keyBuffer.arrayOffset(), keyBuffer.limit()); - } - - @Override - public String getValueString() { - ByteBuffer valueBuffer = getValue(); - return Bytes.toStringBinary(valueBuffer.array(), - valueBuffer.arrayOffset(), valueBuffer.limit()); - } - - private void assertValidSeek() { - if (block == null) { - throw new NotSeekedException(); - } - } - - @Override - protected ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) { - return dataBlockEncoder.getFirstKeyInBlock(getEncodedBuffer(curBlock)); - } - - @Override - protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind, - byte[] key, int offset, int length, boolean seekBefore) - throws IOException { - if (block == null || block.getOffset() != seekToBlock.getOffset()) { - updateCurrentBlock(seekToBlock); - } else if (rewind) { - seeker.rewind(); - } - return seeker.blockSeekTo(key, offset, length, seekBefore); - } } /** diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java index 5f438c96b71..e58d0cc935f 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java @@ -35,10 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KeyComparator; -import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.io.hfile.HFile.Writer; -import org.apache.hadoop.hbase.regionserver.MemStore; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.Bytes; @@ -46,7 +44,7 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.compress.Compressor; /** - * Writes version 1 HFiles. Mainly used for testing backwards-compatibility. + * Writes version 1 HFiles. Mainly used for testing backwards-compatibilty. */ public class HFileWriterV1 extends AbstractHFileWriter { @@ -93,17 +91,16 @@ public class HFileWriterV1 extends AbstractHFileWriter { @Override public Writer createWriter(FileSystem fs, Path path, int blockSize, - Algorithm compressAlgo, HFileDataBlockEncoder dataBlockEncoder, - KeyComparator comparator) + Compression.Algorithm compressAlgo, final KeyComparator comparator) throws IOException { return new HFileWriterV1(conf, cacheConf, fs, path, blockSize, - compressAlgo, dataBlockEncoder, comparator); + compressAlgo, comparator); } @Override public Writer createWriter(FileSystem fs, Path path, int blockSize, - String compressAlgoName, KeyComparator comparator) - throws IOException { + String compressAlgoName, + final KeyComparator comparator) throws IOException { return new HFileWriterV1(conf, cacheConf, fs, path, blockSize, compressAlgoName, comparator); } @@ -120,8 +117,7 @@ public class HFileWriterV1 extends AbstractHFileWriter { public Writer createWriter(final FSDataOutputStream ostream, final int blockSize, final Compression.Algorithm compress, final KeyComparator c) throws IOException { - return new HFileWriterV1(cacheConf, ostream, blockSize, compress, - new NoOpDataBlockEncoder(), c); + return new HFileWriterV1(cacheConf, ostream, blockSize, compress, c); } } @@ -131,7 +127,7 @@ public class HFileWriterV1 extends AbstractHFileWriter { throws IOException { this(conf, cacheConf, fs, path, HFile.DEFAULT_BLOCKSIZE, HFile.DEFAULT_COMPRESSION_ALGORITHM, - new NoOpDataBlockEncoder(), null); + null); } /** @@ -142,18 +138,15 @@ public class HFileWriterV1 extends AbstractHFileWriter { Path path, int blockSize, String compressAlgoName, final KeyComparator comparator) throws IOException { this(conf, cacheConf, fs, path, blockSize, - compressionByName(compressAlgoName), new NoOpDataBlockEncoder(), - comparator); + compressionByName(compressAlgoName), comparator); } /** Constructor that takes a path, creates and closes the output stream. */ - public HFileWriterV1(Configuration conf, CacheConfig cacheConf, - FileSystem fs, Path path, - int blockSize, Compression.Algorithm compress, - HFileDataBlockEncoder blockEncoder, + public HFileWriterV1(Configuration conf, CacheConfig cacheConf, FileSystem fs, + Path path, int blockSize, Compression.Algorithm compress, final KeyComparator comparator) throws IOException { super(cacheConf, createOutputStream(conf, fs, path), path, - blockSize, compress, blockEncoder, comparator); + blockSize, compress, comparator); SchemaMetrics.configureGlobally(conf); } @@ -164,17 +157,15 @@ public class HFileWriterV1 extends AbstractHFileWriter { throws IOException { this(cacheConf, outputStream, blockSize, Compression.getCompressionAlgorithmByName(compressAlgoName), - new NoOpDataBlockEncoder(), comparator); + comparator); } /** Constructor that takes a stream. */ public HFileWriterV1(CacheConfig cacheConf, final FSDataOutputStream outputStream, final int blockSize, - final Compression.Algorithm compress, - HFileDataBlockEncoder blockEncoder, final KeyComparator comparator) + final Compression.Algorithm compress, final KeyComparator comparator) throws IOException { - super(cacheConf, outputStream, null, blockSize, compress, - blockEncoder, comparator); + super(cacheConf, outputStream, null, blockSize, compress, comparator); } /** @@ -211,17 +202,13 @@ public class HFileWriterV1 extends AbstractHFileWriter { if (cacheConf.shouldCacheDataOnWrite()) { baosDos.flush(); - // we do not do dataBlockEncoding on disk HFile V2. byte[] bytes = baos.toByteArray(); HFileBlock cBlock = new HFileBlock(BlockType.DATA, (int) (outputStream.getPos() - blockBegin), bytes.length, -1, - ByteBuffer.wrap(bytes, 0, bytes.length), HFileBlock.FILL_HEADER, - blockBegin, MemStore.NO_PERSISTENT_TS); - HFileBlock codedBlock = blockEncoder.beforeBlockCache(cBlock, - false); - passSchemaMetricsTo(codedBlock); + ByteBuffer.wrap(bytes, 0, bytes.length), true, blockBegin); + passSchemaMetricsTo(cBlock); cacheConf.getBlockCache().cacheBlock( - HFile.getBlockCacheKey(name, blockBegin), codedBlock); + HFile.getBlockCacheKey(name, blockBegin), cBlock); baosDos.close(); } blockNumber++; diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java index 0897ebabdde..d44a9954067 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java @@ -49,13 +49,9 @@ public class HFileWriterV2 extends AbstractHFileWriter { static final Log LOG = LogFactory.getLog(HFileWriterV2.class); /** Max memstore (mvcc) timestamp in FileInfo */ - public static final byte [] MAX_MEMSTORE_TS_KEY = - Bytes.toBytes("MAX_MEMSTORE_TS_KEY"); - + public static final byte [] MAX_MEMSTORE_TS_KEY = Bytes.toBytes("MAX_MEMSTORE_TS_KEY"); /** KeyValue version in FileInfo */ - public static final byte [] KEY_VALUE_VERSION = - Bytes.toBytes("KEY_VALUE_VERSION"); - + public static final byte [] KEY_VALUE_VERSION = Bytes.toBytes("KEY_VALUE_VERSION"); /** Version for KeyValue which includes memstore timestamp */ public static final int KEY_VALUE_VER_WITH_MEMSTORE = 1; @@ -96,10 +92,10 @@ public class HFileWriterV2 extends AbstractHFileWriter { @Override public Writer createWriter(FileSystem fs, Path path, int blockSize, - Compression.Algorithm compress, HFileDataBlockEncoder blockEncoder, + Compression.Algorithm compress, final KeyComparator comparator) throws IOException { return new HFileWriterV2(conf, cacheConf, fs, path, blockSize, - compress, blockEncoder, comparator); + compress, comparator); } @Override @@ -132,7 +128,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { FileSystem fs, Path path) throws IOException { this(conf, cacheConf, fs, path, HFile.DEFAULT_BLOCKSIZE, - HFile.DEFAULT_COMPRESSION_ALGORITHM, null, null); + HFile.DEFAULT_COMPRESSION_ALGORITHM, null); } /** @@ -143,16 +139,15 @@ public class HFileWriterV2 extends AbstractHFileWriter { Path path, int blockSize, String compressAlgoName, final KeyComparator comparator) throws IOException { this(conf, cacheConf, fs, path, blockSize, - compressionByName(compressAlgoName), null, comparator); + compressionByName(compressAlgoName), comparator); } /** Constructor that takes a path, creates and closes the output stream. */ public HFileWriterV2(Configuration conf, CacheConfig cacheConf, FileSystem fs, Path path, int blockSize, Compression.Algorithm compressAlgo, - HFileDataBlockEncoder blockEncoder, final KeyComparator comparator) throws IOException { super(cacheConf, createOutputStream(conf, fs, path), path, - blockSize, compressAlgo, blockEncoder, comparator); + blockSize, compressAlgo, comparator); SchemaMetrics.configureGlobally(conf); finishInit(conf); } @@ -172,8 +167,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { final FSDataOutputStream outputStream, final int blockSize, final Compression.Algorithm compress, final KeyComparator comparator) throws IOException { - super(cacheConf, outputStream, null, blockSize, compress, null, - comparator); + super(cacheConf, outputStream, null, blockSize, compress, comparator); finishInit(conf); } @@ -183,8 +177,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { throw new IllegalStateException("finishInit called twice"); // HFile filesystem-level (non-caching) block writer - fsBlockWriter = new HFileBlock.Writer(compressAlgo, blockEncoder, - includeMemstoreTS); + fsBlockWriter = new HFileBlock.Writer(compressAlgo); // Data block index writer boolean cacheIndexesOnWrite = cacheConf.shouldCacheIndexesOnWrite(); @@ -232,9 +225,8 @@ public class HFileWriterV2 extends AbstractHFileWriter { long startTimeNs = System.nanoTime(); // Update the first data block offset for scanning. - if (firstDataBlockOffset == -1) { + if (firstDataBlockOffset == -1) firstDataBlockOffset = outputStream.getPos(); - } // Update the last data block offset lastDataBlockOffset = outputStream.getPos(); @@ -250,12 +242,10 @@ public class HFileWriterV2 extends AbstractHFileWriter { HFile.writeOps.incrementAndGet(); if (cacheConf.shouldCacheDataOnWrite()) { - HFileBlock cBlock = fsBlockWriter.getBlockForCaching(); - HFileBlock codedBlock = blockEncoder.beforeBlockCache(cBlock, - includeMemstoreTS); - passSchemaMetricsTo(codedBlock); + HFileBlock blockForCaching = fsBlockWriter.getBlockForCaching(); + passSchemaMetricsTo(blockForCaching); cacheConf.getBlockCache().cacheBlock( - HFile.getBlockCacheKey(name, lastDataBlockOffset), codedBlock); + HFile.getBlockCacheKey(name, lastDataBlockOffset), blockForCaching); } } @@ -266,7 +256,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { long offset = outputStream.getPos(); boolean cacheThisBlock = ibw.cacheOnWrite(); ibw.writeInlineBlock(fsBlockWriter.startWriting( - ibw.getInlineBlockType())); + ibw.getInlineBlockType(), cacheThisBlock)); fsBlockWriter.writeHeaderAndData(outputStream); ibw.blockWritten(offset, fsBlockWriter.getOnDiskSizeWithHeader(), fsBlockWriter.getUncompressedSizeWithoutHeader()); @@ -275,11 +265,9 @@ public class HFileWriterV2 extends AbstractHFileWriter { if (cacheThisBlock) { // Cache this block on write. HFileBlock cBlock = fsBlockWriter.getBlockForCaching(); - HFileBlock codedBlock = blockEncoder.beforeBlockCache(cBlock, - includeMemstoreTS); - passSchemaMetricsTo(codedBlock); + passSchemaMetricsTo(cBlock); cacheConf.getBlockCache().cacheBlock( - HFile.getBlockCacheKey(name, offset), codedBlock); + HFile.getBlockCacheKey(name, offset), cBlock); } } } @@ -292,7 +280,8 @@ public class HFileWriterV2 extends AbstractHFileWriter { */ private void newBlock() throws IOException { // This is where the next block begins. - fsBlockWriter.startWriting(BlockType.DATA); + fsBlockWriter.startWriting(BlockType.DATA, + cacheConf.shouldCacheDataOnWrite()); firstKeyInBlock = null; } @@ -424,7 +413,8 @@ public class HFileWriterV2 extends AbstractHFileWriter { // store the beginning offset long offset = outputStream.getPos(); // write the metadata content - DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META); + DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META, + cacheConf.shouldCacheDataOnWrite()); metaData.get(i).write(dos); fsBlockWriter.writeHeaderAndData(outputStream); @@ -450,7 +440,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { // Meta block index. metaBlockIndexWriter.writeSingleLevelIndex(fsBlockWriter.startWriting( - BlockType.ROOT_INDEX), "meta"); + BlockType.ROOT_INDEX, false), "meta"); fsBlockWriter.writeHeaderAndData(outputStream); totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader(); @@ -460,7 +450,8 @@ public class HFileWriterV2 extends AbstractHFileWriter { } // File info - writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO)); + writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO, + false)); fsBlockWriter.writeHeaderAndData(outputStream); totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader(); diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java index 7ba5ea3c74f..0e20440d51c 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java @@ -722,10 +722,4 @@ public class LruBlockCache implements BlockCache, HeapSize { public void shutdown() { this.scheduleThreadPool.shutdown(); } - - /** Clears the cache. Used in tests. */ - public void clearCache() { - map.clear(); - } - } diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java deleted file mode 100644 index 237e51b0a96..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.hfile; - -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.util.Pair; - -/** - * Does not perform any kind of encoding/decoding. - */ -public class NoOpDataBlockEncoder implements HFileDataBlockEncoder { - - @Override - public HFileBlock afterReadFromDisk(HFileBlock block) { - if (block.getBlockType() == BlockType.ENCODED_DATA) { - throw new IllegalStateException("Unexpected encoded block"); - } - return block; - } - - @Override - public HFileBlock afterReadFromDiskAndPuttingInCache(HFileBlock block, - boolean isCompaction, boolean includesMemstoreTS) { - return block; - } - - @Override - public Pair beforeWriteToDisk( - ByteBuffer in, boolean includesMemstoreTS) { - return new Pair(in, BlockType.DATA); - } - - @Override - public HFileBlock beforeBlockCache(HFileBlock block, - boolean includesMemstoreTS) { - return block; - } - - @Override - public HFileBlock afterBlockCache(HFileBlock block, boolean isCompaction, - boolean includesMemstoreTS) { - return block; - } - - @Override - public boolean useEncodedScanner(boolean isCompaction) { - return false; - } - - @Override - public void saveMetadata(StoreFile.Writer storeFileWriter) { - } -} diff --git a/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 3e75b04d40a..bd574b2adda 100644 --- a/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ b/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -65,8 +65,6 @@ import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference.Range; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -530,13 +528,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool { CacheConfig cacheConf = new CacheConfig(conf); HalfStoreFileReader halfReader = null; StoreFile.Writer halfWriter = null; - HFileDataBlockEncoder dataBlockEncoder = new HFileDataBlockEncoderImpl( - familyDescriptor.getDataBlockEncodingOnDisk(), - familyDescriptor.getDataBlockEncodingInCache(), - familyDescriptor.useEncodedDataBlockSeek()); try { halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, - reference, dataBlockEncoder); + reference); Map fileInfo = halfReader.loadFileInfo(); int blocksize = familyDescriptor.getBlocksize(); @@ -544,8 +538,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { BloomType bloomFilterType = familyDescriptor.getBloomFilterType(); halfWriter = new StoreFile.Writer( - fs, outFile, blocksize, compression, dataBlockEncoder, - conf, cacheConf, + fs, outFile, blocksize, compression, conf, cacheConf, KeyValue.COMPARATOR, bloomFilterType, 0); HFileScanner scanner = halfReader.getScanner(false, false, false); scanner.seekTo(); @@ -645,6 +638,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir)); for (Path hfile : hfiles) { if (hfile.getName().startsWith("_")) continue; + HFile.Reader reader = HFile.createReader(fs, hfile, new CacheConfig(getConf())); final byte[] first, last; diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java index b1d42d08658..5c2d72cf50f 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -870,10 +870,6 @@ public class MemStore implements HeapSize { ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST + (2 * ClassSize.CONCURRENT_SKIPLISTMAP)); - // Constants for whether to serialize memstore timestamp. - public static final boolean NO_PERSISTENT_TS = false; - public static final boolean PERSISTENT_TS = true; - /* * Calculate how the MemStore size has changed. Includes overhead of the * backing Map. diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 49a7a1cdd8f..97c2d46462e 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.Compression; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.InvalidHFileException; @@ -146,7 +144,6 @@ public class Store extends SchemaConfigured implements HeapSize { private final Compression.Algorithm compression; /** Compression algorithm for major compaction */ private final Compression.Algorithm compactionCompression; - private HFileDataBlockEncoder dataBlockEncoder; // Comparing KeyValues final KeyValue.KVComparator comparator; @@ -184,12 +181,6 @@ public class Store extends SchemaConfigured implements HeapSize { this.compactionCompression = (family.getCompactionCompression() != Compression.Algorithm.NONE) ? family.getCompactionCompression() : this.compression; - - this.dataBlockEncoder = - new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(), - family.getDataBlockEncodingInCache(), - family.useEncodedDataBlockSeek()); - this.comparator = info.getComparator(); // getTimeToLive returns ttl in seconds. Convert to milliseconds. this.ttl = family.getTimeToLive(); @@ -279,21 +270,6 @@ public class Store extends SchemaConfigured implements HeapSize { public Path getHomedir() { return homedir; } - - /** - * @return the data block encoder - */ - public HFileDataBlockEncoder getDataBlockEncoder() { - return dataBlockEncoder; - } - - /** - * Should be used only in tests. - * @param blockEncoder the block delta encoder to use - */ - public void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) { - this.dataBlockEncoder = blockEncoder; - } /* * Creates an unsorted list of StoreFile loaded from the given directory. @@ -316,9 +292,8 @@ public class Store extends SchemaConfigured implements HeapSize { continue; } StoreFile curfile = new StoreFile(fs, p, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); + this.family.getBloomFilterType()); passSchemaMetricsTo(curfile); - curfile.createReader(); long length = curfile.getReader().length(); this.storeSize += length; @@ -472,9 +447,8 @@ public class Store extends SchemaConfigured implements HeapSize { StoreFile.rename(fs, srcPath, dstPath); StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); + this.family.getBloomFilterType()); passSchemaMetricsTo(sf); - sf.createReader(); LOG.info("Moved hfile " + srcPath + " into store directory " + @@ -581,6 +555,7 @@ public class Store extends SchemaConfigured implements HeapSize { MonitoredTask status) throws IOException { StoreFile.Writer writer; + String fileName; // Find the smallest read point across all the Scanners. long smallestReadPoint = region.getSmallestReadPoint(); long flushed = 0; @@ -676,9 +651,8 @@ public class Store extends SchemaConfigured implements HeapSize { status.setStatus("Flushing " + this + ": reopening flushed file"); StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); + this.family.getBloomFilterType()); passSchemaMetricsTo(sf); - StoreFile.Reader r = sf.createReader(); this.storeSize += r.length(); this.totalUncompressedBytes += r.getTotalUncompressedBytes(); @@ -716,7 +690,7 @@ public class Store extends SchemaConfigured implements HeapSize { Compression.Algorithm compression) throws IOException { StoreFile.Writer w = StoreFile.createWriter(fs, region.getTmpDir(), - blocksize, compression, dataBlockEncoder, comparator, conf, cacheConf, + blocksize, compression, comparator, conf, cacheConf, family.getBloomFilterType(), maxKeyCount); // The store file writer's path does not include the CF name, so we need // to configure the HFile writer directly. @@ -1442,7 +1416,7 @@ public class Store extends SchemaConfigured implements HeapSize { StoreFile storeFile = null; try { storeFile = new StoreFile(this.fs, path, this.conf, - this.cacheConf, this.family.getBloomFilterType(), null); + this.cacheConf, this.family.getBloomFilterType()); passSchemaMetricsTo(storeFile); storeFile.createReader(); } catch (IOException e) { @@ -1494,7 +1468,7 @@ public class Store extends SchemaConfigured implements HeapSize { " to " + destPath); } result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); + this.family.getBloomFilterType()); passSchemaMetricsTo(result); result.createReader(); } @@ -2082,8 +2056,8 @@ public class Store extends SchemaConfigured implements HeapSize { } public static final long FIXED_OVERHEAD = - ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE + - + (19 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG) + ClassSize.align(new SchemaConfigured().heapSize() + + (18 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG) + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN); public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 6af87ffdcab..550b01b1c41 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -56,8 +56,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileWriterV1; import org.apache.hadoop.hbase.io.hfile.HFileWriterV2; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; import org.apache.hadoop.hbase.util.BloomFilter; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterWriter; @@ -132,10 +130,6 @@ public class StoreFile extends SchemaConfigured { /** Key for timestamp of earliest-put in metadata*/ public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS"); - /** Type of encoding used for data blocks in HFile. Stored in file info. */ - public static final byte[] DATA_BLOCK_ENCODING = - Bytes.toBytes("DATA_BLOCK_ENCODING"); - // Make default block size for StoreFiles 8k while testing. TODO: FIX! // Need to make it 8k for testing. public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024; @@ -154,10 +148,7 @@ public class StoreFile extends SchemaConfigured { // Block cache configuration and reference. private final CacheConfig cacheConf; - // What kind of data block encoding will be used - private HFileDataBlockEncoder dataBlockEncoder; - - // HDFS blocks distribution information + // HDFS blocks distribuion information private HDFSBlocksDistribution hdfsBlocksDistribution; // Keys for metadata stored in backing HFile. @@ -215,23 +206,6 @@ public class StoreFile extends SchemaConfigured { // the last modification time stamp private long modificationTimeStamp = 0L; - /** - * Ignore bloom filters, don't use option inMemory - * and dataBlockEncoding in memory. - * @param fs The current file system to use - * @param p The path of the file. - * @param conf The current configuration. - * @throws IOException When opening the reader fails. - */ - StoreFile(final FileSystem fs, - final Path p, - final Configuration conf, - final CacheConfig cacheConf) - throws IOException { - this(fs, p, conf, cacheConf, BloomType.NONE, - new NoOpDataBlockEncoder()); - } - /** * Constructor, loads a reader and it's indices, etc. May allocate a * substantial amount of ram depending on the underlying files (10-20MB?). @@ -246,20 +220,17 @@ public class StoreFile extends SchemaConfigured { * as the Bloom filter type actually present in the HFile, because * column family configuration might change. If this is * {@link BloomType#NONE}, the existing Bloom filter is ignored. - * @param dataBlockEncoder data block encoding algorithm. * @throws IOException When opening the reader fails. */ StoreFile(final FileSystem fs, final Path p, final Configuration conf, final CacheConfig cacheConf, - final BloomType cfBloomType, - final HFileDataBlockEncoder dataBlockEncoder) + final BloomType cfBloomType) throws IOException { this.fs = fs; this.path = p; this.cacheConf = cacheConf; - this.dataBlockEncoder = dataBlockEncoder; if (isReference(p)) { this.reference = Reference.read(fs, p); this.referencePath = getReferredToFile(this.path); @@ -522,10 +493,9 @@ public class StoreFile extends SchemaConfigured { } if (isReference()) { this.reader = new HalfStoreFileReader(this.fs, this.referencePath, - this.cacheConf, this.reference, this.dataBlockEncoder); + this.cacheConf, this.reference); } else { - this.reader = new Reader(this.fs, this.path, this.cacheConf, - this.dataBlockEncoder); + this.reader = new Reader(this.fs, this.path, this.cacheConf); } if (isSchemaConfigured()) { @@ -707,8 +677,8 @@ public class StoreFile extends SchemaConfigured { public static Writer createWriter(final FileSystem fs, final Path dir, final int blocksize, Configuration conf, CacheConfig cacheConf) throws IOException { - return createWriter(fs, dir, blocksize, null, new NoOpDataBlockEncoder(), - null, conf, cacheConf, BloomType.NONE, 0); + return createWriter(fs, dir, blocksize, null, null, conf, cacheConf, + BloomType.NONE, 0); } /** @@ -719,7 +689,6 @@ public class StoreFile extends SchemaConfigured { * Creates a file with a unique name in this directory. * @param blocksize * @param algorithm Pass null to get default. - * @param dataBlockEncoder Pass null to disable data block encoding. * @param c Pass null to get default. * @param conf HBase system configuration. used with bloom filters * @param cacheConf Cache configuration and reference. @@ -732,7 +701,6 @@ public class StoreFile extends SchemaConfigured { final Path dir, final int blocksize, final Compression.Algorithm algorithm, - final HFileDataBlockEncoder dataBlockEncoder, final KeyValue.KVComparator c, final Configuration conf, final CacheConfig cacheConf, @@ -750,7 +718,7 @@ public class StoreFile extends SchemaConfigured { return new Writer(fs, path, blocksize, algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm, - dataBlockEncoder, conf, cacheConf, c == null ? KeyValue.COMPARATOR : c, bloomType, + conf, cacheConf, c == null ? KeyValue.COMPARATOR: c, bloomType, maxKeyCount); } @@ -846,8 +814,6 @@ public class StoreFile extends SchemaConfigured { private KeyValue lastDeleteFamilyKV = null; private long deleteFamilyCnt = 0; - protected HFileDataBlockEncoder dataBlockEncoder; - TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); /* isTimeRangeTrackerSet keeps track if the timeRange has already been set * When flushing a memstore, we set TimeRange and use this variable to @@ -872,16 +838,13 @@ public class StoreFile extends SchemaConfigured { * @throws IOException problem writing to FS */ public Writer(FileSystem fs, Path path, int blocksize, - Compression.Algorithm compress, - HFileDataBlockEncoder dataBlockEncoder, final Configuration conf, + Compression.Algorithm compress, final Configuration conf, CacheConfig cacheConf, final KVComparator comparator, BloomType bloomType, long maxKeys) throws IOException { - this.dataBlockEncoder = dataBlockEncoder != null ? - dataBlockEncoder : new NoOpDataBlockEncoder(); writer = HFile.getWriterFactory(conf, cacheConf).createWriter( fs, path, blocksize, - compress, this.dataBlockEncoder, comparator.getRawComparator()); + compress, comparator.getRawComparator()); this.kvComparator = comparator; @@ -1118,10 +1081,6 @@ public class StoreFile extends SchemaConfigured { } public void close() throws IOException { - // (optional) Add data block encoding used to save this file - // It is mostly for statistics and debugging purpose. - dataBlockEncoder.saveMetadata(this); - boolean hasGeneralBloom = this.closeGeneralBloomFilter(); boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter(); @@ -1160,12 +1119,10 @@ public class StoreFile extends SchemaConfigured { private byte[] lastBloomKey; private long deleteFamilyCnt = -1; - public Reader(FileSystem fs, Path path, CacheConfig cacheConf, - HFileDataBlockEncoder dataBlockEncoder) + public Reader(FileSystem fs, Path path, CacheConfig cacheConf) throws IOException { super(path); - reader = HFile.createReader(fs, path, cacheConf, - dataBlockEncoder); + reader = HFile.createReader(fs, path, cacheConf); bloomFilterType = BloomType.NONE; } @@ -1305,7 +1262,7 @@ public class StoreFile extends SchemaConfigured { default: return true; - } + } } public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, @@ -1355,7 +1312,7 @@ public class StoreFile extends SchemaConfigured { return true; byte[] key; - switch (bloomFilterType) { + switch (bloomFilterType) { case ROW: if (col != null) { throw new RuntimeException("Row-only Bloom filter called with " + diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index b3d767082ca..97d55de4f31 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -177,7 +177,7 @@ class StoreFileScanner implements KeyValueScanner { realSeekDone = true; } } catch (IOException ioe) { - throw new IOException("Could not seek " + this + " " + key, ioe); + throw new IOException("Could not seek " + this, ioe); } } diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java b/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java index 4821b53b233..ae45a26d798 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java @@ -82,15 +82,6 @@ public class SchemaConfigured implements HeapSize, SchemaAware { } } - /** - * Creates an instance corresponding to an unknown table and column family. - * Used in unit tests. - */ - public static SchemaConfigured createUnknown() { - return new SchemaConfigured(null, SchemaMetrics.UNKNOWN, - SchemaMetrics.UNKNOWN); - } - /** * Default constructor. Only use when column/family name are not known at * construction (i.e. for HFile blocks). diff --git a/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java deleted file mode 100644 index 3250f86b28a..00000000000 --- a/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ /dev/null @@ -1,494 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.util; - -import java.io.DataInput; -import java.io.DataInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.io.encoding. - EncoderBufferTooSmallException; -import org.apache.hadoop.io.WritableUtils; - -/** - * Utility functions for working with byte buffers, such as reading/writing - * variable-length long numbers. - */ -public final class ByteBufferUtils { - - // "Compressed integer" serialization helper constants. - private final static int VALUE_MASK = 0x7f; - private final static int NEXT_BIT_SHIFT = 7; - private final static int NEXT_BIT_MASK = 1 << 7; - - private ByteBufferUtils() { - } - - /** - * Similar to {@link WritableUtils#writeVLong(java.io.DataOutput, long)}, - * but writes to a {@link ByteBuffer}. - */ - public static void writeVLong(ByteBuffer out, long i) { - if (i >= -112 && i <= 127) { - out.put((byte) i); - return; - } - - int len = -112; - if (i < 0) { - i ^= -1L; // take one's complement' - len = -120; - } - - long tmp = i; - while (tmp != 0) { - tmp = tmp >> 8; - len--; - } - - out.put((byte) len); - - len = (len < -120) ? -(len + 120) : -(len + 112); - - for (int idx = len; idx != 0; idx--) { - int shiftbits = (idx - 1) * 8; - long mask = 0xFFL << shiftbits; - out.put((byte) ((i & mask) >> shiftbits)); - } - } - - /** - * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a - * {@link ByteBuffer}. - */ - public static long readVLong(ByteBuffer in) { - byte firstByte = in.get(); - int len = WritableUtils.decodeVIntSize(firstByte); - if (len == 1) { - return firstByte; - } - long i = 0; - for (int idx = 0; idx < len-1; idx++) { - byte b = in.get(); - i = i << 8; - i = i | (b & 0xFF); - } - return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i); - } - - - /** - * Put in buffer integer using 7 bit encoding. For each written byte: - * 7 bits are used to store value - * 1 bit is used to indicate whether there is next bit. - * @param value Int to be compressed. - * @param out Where to put compressed data - * @return Number of bytes written. - * @throws IOException on stream error - */ - public static int putCompressedInt(OutputStream out, final int value) - throws IOException { - int i = 0; - int tmpvalue = value; - do { - byte b = (byte) (tmpvalue & VALUE_MASK); - tmpvalue >>>= NEXT_BIT_SHIFT; - if (tmpvalue != 0) { - b |= (byte) NEXT_BIT_MASK; - } - out.write(b); - i++; - } while (tmpvalue != 0); - return i; - } - - /** - * Put in output stream 32 bit integer (Big Endian byte order). - * @param out Where to put integer. - * @param value Value of integer. - * @throws IOException On stream error. - */ - public static void putInt(OutputStream out, final int value) - throws IOException { - for (int i = Bytes.SIZEOF_INT - 1 ; i >= 0 ; --i) { - out.write((byte) (value >>> (i * 8))); - } - } - - /** - * Copy byte to the output stream. - * @param b byte to be copied - * @param out Where to put compressed data - * @return Number of written bytes. - * @throws IOException on stream error - */ - public static int copyToStream(OutputStream out, byte b) - throws IOException { - out.write(b); - return Bytes.SIZEOF_BYTE; - } - - /** - * Copy the data to the output stream and update position in buffer. - * @param out Write bytes here. - * @param buffer Source buffer in certain position. - * @param length Length of copy. - * @return Number of written bytes. - * @throws IOException on stream error - */ - public static int copyToStream(OutputStream out, ByteBuffer buffer, - int length) throws IOException { - if (buffer.hasArray()) { - out.write(buffer.array(), buffer.arrayOffset() + buffer.position(), - length); - skip(buffer, length); - } else { - for (int i = 0 ; i < length ; ++i) { - out.write(buffer.get()); - } - } - return length; - } - - /** - * Copy the data to the output stream - * @param out Write bytes here. - * @param in Source buffer in certain position. - * @param offset In the buffer from position 0. - * @param length Length of copy. - * @return Number of written bytes. - * @throws IOException on stream error - */ - public static int copyToStream(OutputStream out, ByteBuffer in, - int offset, int length) throws IOException { - if (in.hasArray()) { - out.write(in.array(), in.arrayOffset() + offset, - length); - } else { - for (int i = 0 ; i < length ; ++i) { - out.write(in.get(offset + i)); - } - } - return length; - } - - public static int putLong(OutputStream out, final long length, - final int fitInBytes) throws IOException { - long tmpLength = length; - for (int i = 0 ; i < fitInBytes ; ++i) { - out.write((byte) (tmpLength & 0xff)); - tmpLength >>>= 8; - } - return fitInBytes; - } - - /** - * Check how many bytes are required to store value. - * @param value Value which size will be tested. - * @return How many bytes are required to store value. - */ - public static int longFitsIn(final long value) { - if (value < 0) { - return 8; - } - - if (value < (1l << 4 * 8)) { - // no more than 4 bytes - if (value < (1l << 2 * 8)) { - if (value < (1l << 1 * 8)) { - return 1; - } - return 2; - } - if (value < (1l << 3 * 8)) { - return 3; - } - return 4; - } - // more than 4 bytes - if (value < (1l << 6 * 8)) { - if (value < (1l << 5 * 8)) { - return 5; - } - return 6; - } - if (value < (1l << 7 * 8)) { - return 7; - } - return 8; - } - - /** - * Check how many bytes is required to store value. - * @param value Value which size will be tested. - * @return How many bytes are required to store value. - */ - public static int intFitsIn(final int value) { - if (value < 0) { - return 4; - } - - if (value < (1 << 2 * 8)) { - if (value < (1 << 1 * 8)) { - return 1; - } - return 2; - } - if (value <= (1 << 3 * 8)) { - return 3; - } - return 4; - } - - /** - * Read integer from stream coded in 7 bits and increment position. - * @return Read integer. - * @throws IOException - */ - public static int readCompressedInt(InputStream input) - throws IOException { - int result = 0; - int i = 0; - byte b; - do { - b = (byte) input.read(); - result += (b & VALUE_MASK) << (NEXT_BIT_SHIFT * i); - i++; - if (i > Bytes.SIZEOF_INT + 1) { - throw new IllegalStateException( - "Corrupted compressed int (too long: " + (i + 1) + " bytes)"); - } - } while (0 != (b & NEXT_BIT_MASK)); - return result; - } - - /** - * Read integer from buffer coded in 7 bits and increment position. - * @return Read integer. - */ - public static int readCompressedInt(ByteBuffer buffer) { - byte b = buffer.get(); - if ((b & NEXT_BIT_MASK) != 0) { - return (b & VALUE_MASK) + (readCompressedInt(buffer) << NEXT_BIT_SHIFT); - } - return b & VALUE_MASK; - } - - /** - * Read long which was written to fitInBytes bytes and increment position. - * @param fitInBytes In how many bytes given long is stored. - * @return The value of parsed long. - * @throws IOException - */ - public static long readLong(InputStream input, final int fitInBytes) - throws IOException { - long tmpLong = 0; - for (int i = 0 ; i < fitInBytes ; ++i) { - tmpLong |= (input.read() & 0xffl) << (8 * i); - } - return tmpLong; - } - - /** - * Read long which was written to fitInBytes bytes and increment position. - * @param fitInBytes In how many bytes given long is stored. - * @return The value of parsed long. - */ - public static long readLong(ByteBuffer buffer, final int fitInBytes) { - long tmpLength = 0; - for (int i = 0 ; i < fitInBytes ; ++i) { - tmpLength |= (buffer.get() & 0xffl) << (8l * i); - } - return tmpLength; - } - - /** - * Asserts that we have 'length' bytes remaining in 'buffer'. - * @param buffer Where are we looking for remaining bytes. - * @param length How many bytes do we need. - * @throws EncoderBufferTooSmallException If there are no enough bytes. - */ - public static void ensureSpace(ByteBuffer buffer, int length) - throws EncoderBufferTooSmallException { - if (buffer.position() + length > buffer.limit()) { - throw new EncoderBufferTooSmallException( - "Buffer position=" + buffer.position() + - ", buffer limit=" + buffer.limit() + - ", length to be written=" + length); - } - } - - /** - * Copy 'length' bytes from 'source' and put it at the current position of - * 'buffer'. Update position in 'buffer' afterwards. - * @param source From where data should be read. - * @param buffer Write data here. - * @param length Read that many bytes. - * @throws IOException If there is problem in source. - */ - public static void copyFromStream(DataInputStream source, - ByteBuffer buffer, int length) throws IOException { - if (buffer.hasArray()) { - source.readFully(buffer.array(), buffer.position() + buffer.arrayOffset(), - length); - skip(buffer, length); - } else { - for (int i = 0 ; i < length ; ++i) { - buffer.put(source.readByte()); - } - } - } - - /** - * Copy from one buffer to another from given offset - * @param source From where copy. - * @param destination Where to copy. - * @param sourceOffset Offset in the source buffer - * @param length How many bytes will be copied. - * @throws IOException - */ - public static void copyFromBuffer(ByteBuffer source, - ByteBuffer destination, int sourceOffset, int length) { - if (source.hasArray() && destination.hasArray()) { - System.arraycopy(source.array(), sourceOffset + source.arrayOffset(), - destination.array(), destination.position() + - destination.arrayOffset(), length); - skip(destination, length); - } else { - for (int i = 0 ; i < length ; ++i) { - destination.put(source.get(sourceOffset + i)); - } - } - } - - /** - * Find length of common prefix of two parts in the buffer - * @param buffer Where parts are located. - * @param offsetLeft Offset of the first part. - * @param offsetRight Offset of the second part. - * @param limit Maximal length of common prefix. - * @return Length of prefix. - */ - public static int findCommonPrefix(ByteBuffer buffer, int offsetLeft, - int offsetRight, int limit) { - int prefix = 0; - - for (; prefix < limit ; ++prefix) { - if (buffer.get(offsetLeft + prefix) != buffer.get(offsetRight + prefix)) { - break; - } - } - - return prefix; - } - - /** - * Find length of common prefix in two arrays. - * @param left Array to be compared. - * @param leftOffset Offset in left array. - * @param leftLength Length of left array. - * @param right Array to be compared. - * @param rightArray Offset in right array. - * @param rightLength Length of right array. - */ - public static int findCommonPrefix( - byte[] left, int leftOffset, int leftLength, - byte[] right, int rightOffset, int rightLength) { - int length = Math.min(leftLength, rightLength); - int result = 0; - - while (result < length && - left[leftOffset + result] == right[rightOffset + result]) { - result++; - } - - return result; - } - - /** - * Check whether two parts in the same buffer are equal. - * @param buffer In which buffer there are parts - * @param offsetLeft Beginning of first part. - * @param lengthLeft Length of the first part. - * @param offsetRight Beginning of the second part. - * @param lengthRight Length of the second part. - * @return - */ - public static boolean arePartsEqual(ByteBuffer buffer, - int offsetLeft, int lengthLeft, - int offsetRight, int lengthRight) { - if (lengthLeft != lengthRight) { - return false; - } - - if (buffer.hasArray()) { - return 0 == Bytes.compareTo( - buffer.array(), buffer.arrayOffset() + offsetLeft, lengthLeft, - buffer.array(), buffer.arrayOffset() + offsetRight, lengthRight); - } - - for (int i = 0 ; i < lengthRight ; ++i) { - if (buffer.get(offsetLeft + i) != buffer.get(offsetRight + i)) { - return false; - } - } - return true; - } - - /** - * Increment position in buffer. - * @param buffer In this buffer. - * @param length By that many bytes. - */ - public static void skip(ByteBuffer buffer, int length) { - buffer.position(buffer.position() + length); - } - - /** - * Read int, assuming it is stored in N bytes with no special encoding. - * @param source From where read bytes. - * @param intLength How long is the integer - * @return The value of the integer. - * @throws IOException On IO error. - */ - public static int readCompressedInt(InputStream source, int intLength) - throws IOException { - int result = 0; - for (int i = 0 ; i < intLength ; ++i) { - result = (result << 8) + (source.read() & 0xff); - } - return result; - } - - /** - * Read int, assuming it is stored in N bytes with no special encoding. - * @param buffer Read bytes from this buffer. - * @param intLength The lenght of the integer in bytes. - * @return The value of the integer. - */ - public static int readCompressedInt(ByteBuffer buffer, int intLength) { - int result = 0; - for (int i = 0 ; i < intLength ; ++i) { - result = (result << 8) + (buffer.get() & 0xff); - } - return result; - } - -} diff --git a/src/main/ruby/hbase/admin.rb b/src/main/ruby/hbase/admin.rb index 64562e28839..72c8259702c 100644 --- a/src/main/ruby/hbase/admin.rb +++ b/src/main/ruby/hbase/admin.rb @@ -532,9 +532,6 @@ module Hbase family.setInMemory(JBoolean.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::IN_MEMORY) family.setTimeToLive(JInteger.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::TTL])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::TTL) family.setCompressionType(org.apache.hadoop.hbase.io.hfile.Compression::Algorithm.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::COMPRESSION])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::COMPRESSION) - family.setDataBlockEncodingOnDisk(org.apache.hadoop.hbase.io.encoding.DataBlockEncodingAlgorithms::Algorithm.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING_ON_DISK])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING_ON_DISK) - family.setDataBlockEncodingInCache(org.apache.hadoop.hbase.io.encoding.DataBlockEncodingAlgorithms::Algorithm.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING_IN_CACHE])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING_IN_CACHE) - family.setEncodedDataBlockSeek(JBoolean.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::ENCODED_DATA_BLOCK_SEEK])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::ENCODED_DATA_BLOCK_SEEK) family.setBlocksize(JInteger.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::BLOCKSIZE])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::BLOCKSIZE) family.setMaxVersions(JInteger.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::VERSIONS])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::VERSIONS) family.setMinVersions(JInteger.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::MIN_VERSIONS])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::MIN_VERSIONS) diff --git a/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index 0802fb33ab4..0b1d2586163 100644 --- a/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -221,33 +221,18 @@ public abstract class HBaseTestCase extends TestCase { final int minVersions, final int versions, final int ttl, boolean keepDeleted) { HTableDescriptor htd = new HTableDescriptor(name); htd.addFamily(new HColumnDescriptor(fam1, minVersions, versions, - keepDeleted, - HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, - false, false, - HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl, - HColumnDescriptor.DEFAULT_BLOOMFILTER, - HConstants.REPLICATION_SCOPE_LOCAL)); + keepDeleted, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, + HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl, + HColumnDescriptor.DEFAULT_BLOOMFILTER, + HConstants.REPLICATION_SCOPE_LOCAL)); htd.addFamily(new HColumnDescriptor(fam2, minVersions, versions, - keepDeleted, - HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, - false, false, + keepDeleted, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl, HColumnDescriptor.DEFAULT_BLOOMFILTER, HConstants.REPLICATION_SCOPE_LOCAL)); htd.addFamily(new HColumnDescriptor(fam3, minVersions, versions, - keepDeleted, - HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, - false, false, - HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl, + keepDeleted, HColumnDescriptor.DEFAULT_COMPRESSION, false, false, + HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl, HColumnDescriptor.DEFAULT_BLOOMFILTER, HConstants.REPLICATION_SCOPE_LOCAL)); return htd; diff --git a/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java index 4099f40055f..a9ef57a35ca 100644 --- a/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java +++ b/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java @@ -191,8 +191,7 @@ public class HFilePerformanceEvaluation { void setUp() throws Exception { writer = HFile.getWriterFactoryNoCache(conf).createWriter(this.fs, - this.mf, RFILE_BLOCKSIZE, (Compression.Algorithm) null, null, - null); + this.mf, RFILE_BLOCKSIZE, (Compression.Algorithm) null, null); } @Override @@ -366,4 +365,4 @@ public class HFilePerformanceEvaluation { public static void main(String[] args) throws Exception { new HFilePerformanceEvaluation().runBenchmarks(); } -} +} \ No newline at end of file diff --git a/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 9734e6609ed..ab80020b1e3 100644 --- a/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -143,9 +143,6 @@ public class TestFromClientSide { HColumnDescriptor.DEFAULT_VERSIONS, true, HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, HColumnDescriptor.DEFAULT_IN_MEMORY, HColumnDescriptor.DEFAULT_BLOCKCACHE, HColumnDescriptor.DEFAULT_BLOCKSIZE, diff --git a/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index 4ac9797a372..e3061d01f01 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ b/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.util.Bytes; @@ -99,7 +98,7 @@ public class TestHalfStoreFileReader { CacheConfig cacheConf) throws IOException { final HalfStoreFileReader halfreader = - new HalfStoreFileReader(fs, p, cacheConf, bottom, null); + new HalfStoreFileReader(fs, p, cacheConf, bottom); halfreader.loadFileInfo(); final HFileScanner scanner = halfreader.getScanner(false, false); diff --git a/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java index 3a07dc05baf..8b7d016b001 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java +++ b/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java @@ -295,14 +295,7 @@ public class TestHeapSize extends TestCase { assertEquals(expected, actual); } - // SchemaConfigured - LOG.debug("Heap size for: " + SchemaConfigured.class.getName()); - SchemaConfigured sc = new SchemaConfigured(null, "myTable", "myCF"); - assertEquals(ClassSize.estimateBase(SchemaConfigured.class, true), - sc.heapSize()); - // Store Overhead - LOG.debug("Heap size for: " + Store.class.getName()); cl = Store.class; actual = Store.FIXED_OVERHEAD; expected = ClassSize.estimateBase(cl, false); @@ -326,6 +319,10 @@ public class TestHeapSize extends TestCase { // accounted for. But we have satisfied our two core requirements. // Sizing is quite accurate now, and our tests will throw errors if // any of these classes are modified without updating overhead sizes. + + SchemaConfigured sc = new SchemaConfigured(null, "myTable", "myCF"); + assertEquals(ClassSize.estimateBase(SchemaConfigured.class, true), + sc.heapSize()); } @org.junit.Rule diff --git a/src/test/java/org/apache/hadoop/hbase/io/encoding/RedundantKVGenerator.java b/src/test/java/org/apache/hadoop/hbase/io/encoding/RedundantKVGenerator.java deleted file mode 100644 index b2fe516e37a..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/io/encoding/RedundantKVGenerator.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; - -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.io.WritableUtils; - -/** - * Generate list of key values which are very useful to test data block encoding - * and compression. - */ -public class RedundantKVGenerator { - // row settings - static int DEFAULT_NUMBER_OF_ROW_PREFIXES = 10; - static int DEFAULT_AVERAGE_PREFIX_LENGTH = 6; - static int DEFAULT_PREFIX_LENGTH_VARIANCE = 3; - static int DEFAULT_AVERAGE_SUFFIX_LENGTH = 3; - static int DEFAULT_SUFFIX_LENGTH_VARIANCE = 3; - static int DEFAULT_NUMBER_OF_ROW = 500; - - // qualifier - static float DEFAULT_CHANCE_FOR_SAME_QUALIFIER = 0.5f; - static float DEFAULT_CHANCE_FOR_SIMILIAR_QUALIFIER = 0.4f; - static int DEFAULT_AVERAGE_QUALIFIER_LENGTH = 9; - static int DEFAULT_QUALIFIER_LENGTH_VARIANCE = 3; - - static int DEFAULT_COLUMN_FAMILY_LENGTH = 9; - static int DEFAULT_VALUE_LENGTH = 8; - static float DEFAULT_CHANCE_FOR_ZERO_VALUE = 0.5f; - - static int DEFAULT_BASE_TIMESTAMP_DIVIDE = 1000000; - static int DEFAULT_TIMESTAMP_DIFF_SIZE = 100000000; - - /** - * Default constructor, assumes all parameters from class constants. - */ - public RedundantKVGenerator() { - this(new Random(42L), - DEFAULT_NUMBER_OF_ROW_PREFIXES, - DEFAULT_AVERAGE_PREFIX_LENGTH, - DEFAULT_PREFIX_LENGTH_VARIANCE, - DEFAULT_AVERAGE_SUFFIX_LENGTH, - DEFAULT_SUFFIX_LENGTH_VARIANCE, - DEFAULT_NUMBER_OF_ROW, - - DEFAULT_CHANCE_FOR_SAME_QUALIFIER, - DEFAULT_CHANCE_FOR_SIMILIAR_QUALIFIER, - DEFAULT_AVERAGE_QUALIFIER_LENGTH, - DEFAULT_QUALIFIER_LENGTH_VARIANCE, - - DEFAULT_COLUMN_FAMILY_LENGTH, - DEFAULT_VALUE_LENGTH, - DEFAULT_CHANCE_FOR_ZERO_VALUE, - - DEFAULT_BASE_TIMESTAMP_DIVIDE, - DEFAULT_TIMESTAMP_DIFF_SIZE - ); - } - - - /** - * Various configuration options for generating key values - * @param randomizer pick things by random - */ - public RedundantKVGenerator(Random randomizer, - int numberOfRowPrefixes, - int averagePrefixLength, - int prefixLengthVariance, - int averageSuffixLength, - int suffixLengthVariance, - int numberOfRows, - - float chanceForSameQualifier, - float chanceForSimiliarQualifier, - int averageQualifierLength, - int qualifierLengthVariance, - - int columnFamilyLength, - int valueLength, - float chanceForZeroValue, - - int baseTimestampDivide, - int timestampDiffSize - ) { - this.randomizer = randomizer; - - this.numberOfRowPrefixes = numberOfRowPrefixes; - this.averagePrefixLength = averagePrefixLength; - this.prefixLengthVariance = prefixLengthVariance; - this.averageSuffixLength = averageSuffixLength; - this.suffixLengthVariance = suffixLengthVariance; - this.numberOfRows = numberOfRows; - - this.chanceForSameQualifier = chanceForSameQualifier; - this.chanceForSimiliarQualifier = chanceForSimiliarQualifier; - this.averageQualifierLength = averageQualifierLength; - this.qualifierLengthVariance = qualifierLengthVariance; - - this.columnFamilyLength = columnFamilyLength; - this.valueLength = valueLength; - this.chanceForZeroValue = chanceForZeroValue; - - this.baseTimestampDivide = baseTimestampDivide; - this.timestampDiffSize = timestampDiffSize; - } - - /** Used to generate dataset */ - private Random randomizer; - - // row settings - private int numberOfRowPrefixes; - private int averagePrefixLength = 6; - private int prefixLengthVariance = 3; - private int averageSuffixLength = 3; - private int suffixLengthVariance = 3; - private int numberOfRows = 500; - - // qualifier - private float chanceForSameQualifier = 0.5f; - private float chanceForSimiliarQualifier = 0.4f; - private int averageQualifierLength = 9; - private int qualifierLengthVariance = 3; - - private int columnFamilyLength = 9; - private int valueLength = 8; - private float chanceForZeroValue = 0.5f; - - private int baseTimestampDivide = 1000000; - private int timestampDiffSize = 100000000; - - private List generateRows() { - // generate prefixes - List prefixes = new ArrayList(); - prefixes.add(new byte[0]); - for (int i = 1 ; i < numberOfRowPrefixes ; ++i) { - int prefixLength = averagePrefixLength; - prefixLength += randomizer.nextInt(2 * prefixLengthVariance + 1) - - prefixLengthVariance; - byte[] newPrefix = new byte[prefixLength]; - randomizer.nextBytes(newPrefix); - prefixes.add(newPrefix); - } - - // generate rest of the row - List rows = new ArrayList(); - for (int i = 0 ; i < numberOfRows ; ++i) { - int suffixLength = averageSuffixLength; - suffixLength += randomizer.nextInt(2 * suffixLengthVariance + 1) - - suffixLengthVariance; - int randomPrefix = randomizer.nextInt(prefixes.size()); - byte[] row = new byte[prefixes.get(randomPrefix).length + - suffixLength]; - rows.add(row); - } - - return rows; - } - - /** - * Generate test data useful to test encoders. - * @param howMany How many Key values should be generated. - * @return sorted list of key values - */ - public List generateTestKeyValues(int howMany) { - List result = new ArrayList(); - - List rows = generateRows(); - Map> rowsToQualifier = - new HashMap>(); - - byte[] family = new byte[columnFamilyLength]; - randomizer.nextBytes(family); - - long baseTimestamp = Math.abs(randomizer.nextLong()) / - baseTimestampDivide; - - byte[] value = new byte[valueLength]; - - for (int i = 0 ; i < howMany ; ++i) { - long timestamp = baseTimestamp + randomizer.nextInt( - timestampDiffSize); - Integer rowId = randomizer.nextInt(rows.size()); - byte[] row = rows.get(rowId); - - // generate qualifier, sometimes it is same, sometimes similar, - // occasionally completely different - byte[] qualifier; - float qualifierChance = randomizer.nextFloat(); - if (!rowsToQualifier.containsKey(rowId) || - qualifierChance > chanceForSameQualifier + - chanceForSimiliarQualifier) { - int qualifierLength = averageQualifierLength; - qualifierLength += - randomizer.nextInt(2 * qualifierLengthVariance + 1) - - qualifierLengthVariance; - qualifier = new byte[qualifierLength]; - randomizer.nextBytes(qualifier); - - // add it to map - if (!rowsToQualifier.containsKey(rowId)) { - rowsToQualifier.put(rowId, new ArrayList()); - } - rowsToQualifier.get(rowId).add(qualifier); - } else if (qualifierChance > chanceForSameQualifier) { - // similar qualifier - List previousQualifiers = rowsToQualifier.get(rowId); - byte[] originalQualifier = previousQualifiers.get( - randomizer.nextInt(previousQualifiers.size())); - - qualifier = new byte[originalQualifier.length]; - int commonPrefix = randomizer.nextInt(qualifier.length); - System.arraycopy(originalQualifier, 0, qualifier, 0, commonPrefix); - for (int j = commonPrefix ; j < qualifier.length ; ++j) { - qualifier[j] = (byte) (randomizer.nextInt() & 0xff); - } - - rowsToQualifier.get(rowId).add(qualifier); - } else { - // same qualifier - List previousQualifiers = rowsToQualifier.get(rowId); - qualifier = previousQualifiers.get( - randomizer.nextInt(previousQualifiers.size())); - } - - if (randomizer.nextFloat() < chanceForZeroValue) { - for (int j = 0 ; j < value.length ; ++j) { - value[j] = (byte) 0; - } - } else { - randomizer.nextBytes(value); - } - - result.add(new KeyValue(row, family, qualifier, timestamp, value)); - } - - Collections.sort(result, KeyValue.COMPARATOR); - - return result; - } - - /** - * Convert list of KeyValues to byte buffer. - * @param keyValues list of KeyValues to be converted. - * @return buffer with content from key values - */ - public static ByteBuffer convertKvToByteBuffer(List keyValues, - boolean includesMemstoreTS) { - int totalSize = 0; - for (KeyValue kv : keyValues) { - totalSize += kv.getLength(); - if (includesMemstoreTS) { - totalSize += WritableUtils.getVIntSize(kv.getMemstoreTS()); - } - } - - ByteBuffer result = ByteBuffer.allocate(totalSize); - for (KeyValue kv : keyValues) { - result.put(kv.getBuffer(), kv.getOffset(), kv.getLength()); - if (includesMemstoreTS) { - ByteBufferUtils.writeVLong(result, kv.getMemstoreTS()); - } - } - - return result; - } - -} \ No newline at end of file diff --git a/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java b/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java deleted file mode 100644 index 2e7de2f1d2b..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import static org.junit.Assert.assertEquals; - -import org.apache.hadoop.hbase.SmallTests; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(SmallTests.class) -public class TestBufferedDataBlockEncoder { - - @Test - public void testEnsureSpaceForKey() { - BufferedDataBlockEncoder.SeekerState state = - new BufferedDataBlockEncoder.SeekerState(); - for (int i = 1; i <= 65536; ++i) { - state.keyLength = i; - state.ensureSpaceForKey(); - state.keyBuffer[state.keyLength - 1] = (byte) ((i - 1) % 0xff); - for (int j = 0; j < i - 1; ++j) { - // Check that earlier bytes were preserved as the buffer grew. - assertEquals((byte) (j % 0xff), state.keyBuffer[j]); - } - } - } - -} diff --git a/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java deleted file mode 100644 index c81f9fddbeb..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java +++ /dev/null @@ -1,343 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.encoding; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Random; - -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -/** - * Test all of the data block encoding algorithms for correctness. - * Most of the class generate data which will test different branches in code. - */ -@Category(LargeTests.class) -@RunWith(Parameterized.class) -public class TestDataBlockEncoders { - static int NUMBER_OF_KV = 10000; - static int NUM_RANDOM_SEEKS = 10000; - - private RedundantKVGenerator generator = new RedundantKVGenerator(); - private Random randomizer = new Random(42l); - - private final boolean includesMemstoreTS; - - @Parameters - public static Collection parameters() { - return HBaseTestingUtility.BOOLEAN_PARAMETERIZED; - } - - public TestDataBlockEncoders(boolean includesMemstoreTS) { - this.includesMemstoreTS = includesMemstoreTS; - } - - private void testAlgorithm(ByteBuffer dataset, DataBlockEncoder encoder) - throws IOException { - // encode - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dataOut = new DataOutputStream(baos); - encoder.compressKeyValues(dataOut, dataset, includesMemstoreTS); - - // decode - ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - DataInputStream dis = new DataInputStream(bais); - ByteBuffer actualDataset; - actualDataset = encoder.uncompressKeyValues(dis, includesMemstoreTS); - - dataset.rewind(); - actualDataset.rewind(); - - assertEquals("Encoding -> decoding gives different results for " + encoder, - dataset, actualDataset); - } - - /** - * Test data block encoding of empty KeyValue. - * @throws IOException On test failure. - */ - @Test - public void testEmptyKeyValues() throws IOException { - List kvList = new ArrayList(); - byte[] row = new byte[0]; - byte[] family = new byte[0]; - byte[] qualifier = new byte[0]; - byte[] value = new byte[0]; - kvList.add(new KeyValue(row, family, qualifier, 0l, Type.Put, value)); - kvList.add(new KeyValue(row, family, qualifier, 0l, Type.Put, value)); - testEncodersOnDataset(RedundantKVGenerator.convertKvToByteBuffer(kvList, - includesMemstoreTS)); - } - - /** - * Test KeyValues with negative timestamp. - * @throws IOException On test failure. - */ - @Test - public void testNegativeTimestamps() throws IOException { - List kvList = new ArrayList(); - byte[] row = new byte[0]; - byte[] family = new byte[0]; - byte[] qualifier = new byte[0]; - byte[] value = new byte[0]; - kvList.add(new KeyValue(row, family, qualifier, -1l, Type.Put, value)); - kvList.add(new KeyValue(row, family, qualifier, -2l, Type.Put, value)); - testEncodersOnDataset( - RedundantKVGenerator.convertKvToByteBuffer(kvList, - includesMemstoreTS)); - } - - /** - * Test whether compression -> decompression gives the consistent results on - * pseudorandom sample. - * @throws IOException On test failure. - */ - @Test - public void testExecutionOnSample() throws IOException { - testEncodersOnDataset( - RedundantKVGenerator.convertKvToByteBuffer( - generator.generateTestKeyValues(NUMBER_OF_KV), - includesMemstoreTS)); - } - - /** - * Test seeking while file is encoded. - */ - @Test - public void testSeekingOnSample() throws IOException{ - List sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV); - ByteBuffer originalBuffer = - RedundantKVGenerator.convertKvToByteBuffer(sampleKv, - includesMemstoreTS); - List dataBlockEncoders = DataBlockEncodings.getAllEncoders(); - - // create all seekers - List encodedSeekers = - new ArrayList(); - for (DataBlockEncoder encoder : dataBlockEncoders) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dataOut = new DataOutputStream(baos); - encoder.compressKeyValues(dataOut, originalBuffer, includesMemstoreTS); - ByteBuffer encodedBuffer = ByteBuffer.wrap(baos.toByteArray()); - DataBlockEncoder.EncodedSeeker seeker = - encoder.createSeeker(KeyValue.KEY_COMPARATOR, includesMemstoreTS); - seeker.setCurrentBuffer(encodedBuffer); - encodedSeekers.add(seeker); - } - - // test it! - // try a few random seeks - for (boolean seekBefore : new boolean[] {false, true}) { - for (int i = 0 ; i < NUM_RANDOM_SEEKS ; ++i) { - int keyValueId; - if (!seekBefore) { - keyValueId = randomizer.nextInt(sampleKv.size()); - } else { - keyValueId = randomizer.nextInt(sampleKv.size() - 1) + 1; - } - - KeyValue keyValue = sampleKv.get(keyValueId); - checkSeekingConsistency(encodedSeekers, seekBefore, keyValue); - } - } - - // check edge cases - checkSeekingConsistency(encodedSeekers, false, sampleKv.get(0)); - for (boolean seekBefore : new boolean[] {false, true}) { - checkSeekingConsistency(encodedSeekers, seekBefore, - sampleKv.get(sampleKv.size() - 1)); - KeyValue midKv = sampleKv.get(sampleKv.size() / 2); - KeyValue lastMidKv = midKv.createLastOnRowCol(); - checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv); - } - } - - /** - * Test iterating on encoded buffers. - */ - @Test - public void testNextOnSample() { - List sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV); - ByteBuffer originalBuffer = - RedundantKVGenerator.convertKvToByteBuffer(sampleKv, - includesMemstoreTS); - List dataBlockEncoders = DataBlockEncodings.getAllEncoders(); - - for (DataBlockEncoder encoder : dataBlockEncoders) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dataOut = new DataOutputStream(baos); - try { - encoder.compressKeyValues(dataOut, originalBuffer, includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException(String.format( - "Bug while encoding using '%s'", encoder.toString()), e); - } - - ByteBuffer encodedBuffer = ByteBuffer.wrap(baos.toByteArray()); - DataBlockEncoder.EncodedSeeker seeker = - encoder.createSeeker(KeyValue.KEY_COMPARATOR, includesMemstoreTS); - seeker.setCurrentBuffer(encodedBuffer); - int i = 0; - do { - KeyValue expectedKeyValue = sampleKv.get(i); - ByteBuffer keyValue = seeker.getKeyValue(); - if (0 != Bytes.compareTo( - keyValue.array(), keyValue.arrayOffset(), keyValue.limit(), - expectedKeyValue.getBuffer(), expectedKeyValue.getOffset(), - expectedKeyValue.getLength())) { - - int commonPrefix = 0; - byte[] left = keyValue.array(); - byte[] right = expectedKeyValue.getBuffer(); - int leftOff = keyValue.arrayOffset(); - int rightOff = expectedKeyValue.getOffset(); - int length = Math.min(keyValue.limit(), expectedKeyValue.getLength()); - while (commonPrefix < length && - left[commonPrefix + leftOff] == right[commonPrefix + rightOff]) { - commonPrefix++; - } - - fail(String.format( - "next() produces wrong results " + - "encoder: %s i: %d commonPrefix: %d" + - "\n expected %s\n actual %s", - encoder.toString(), i, commonPrefix, - Bytes.toStringBinary(expectedKeyValue.getBuffer(), - expectedKeyValue.getOffset(), expectedKeyValue.getLength()), - Bytes.toStringBinary(keyValue))); - } - i++; - } while (seeker.next()); - } - } - - /** - * Test whether the decompression of first key is implemented correctly. - */ - @Test - public void testFirstKeyInBlockOnSample() { - List sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV); - ByteBuffer originalBuffer = - RedundantKVGenerator.convertKvToByteBuffer(sampleKv, - includesMemstoreTS); - List dataBlockEncoders = DataBlockEncodings.getAllEncoders(); - - for (DataBlockEncoder encoder : dataBlockEncoders) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dataOut = new DataOutputStream(baos); - try { - encoder.compressKeyValues(dataOut, originalBuffer, includesMemstoreTS); - } catch (IOException e) { - throw new RuntimeException(String.format( - "Bug while encoding using '%s'", encoder.toString()), e); - } - - ByteBuffer encodedBuffer = ByteBuffer.wrap(baos.toByteArray()); - ByteBuffer keyBuffer = encoder.getFirstKeyInBlock(encodedBuffer); - KeyValue firstKv = sampleKv.get(0); - if (0 != Bytes.compareTo( - keyBuffer.array(), keyBuffer.arrayOffset(), keyBuffer.limit(), - firstKv.getBuffer(), firstKv.getKeyOffset(), - firstKv.getKeyLength())) { - - int commonPrefix = 0; - int length = Math.min(keyBuffer.limit(), firstKv.getKeyLength()); - while (commonPrefix < length && - keyBuffer.array()[keyBuffer.arrayOffset() + commonPrefix] == - firstKv.getBuffer()[firstKv.getKeyOffset() + commonPrefix]) { - commonPrefix++; - } - fail(String.format("Bug in '%s' commonPrefix %d", - encoder.toString(), commonPrefix)); - } - } - } - - private void checkSeekingConsistency( - List encodedSeekers, boolean seekBefore, - KeyValue keyValue) { - ByteBuffer expectedKeyValue = null; - ByteBuffer expectedKey = null; - ByteBuffer expectedValue = null; - - for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) { - seeker.blockSeekTo(keyValue.getBuffer(), - keyValue.getKeyOffset(), keyValue.getKeyLength(), seekBefore); - seeker.rewind(); - - ByteBuffer actualKeyValue = seeker.getKeyValue(); - ByteBuffer actualKey = seeker.getKey(); - ByteBuffer actualValue = seeker.getValue(); - - if (expectedKeyValue != null) { - assertEquals(expectedKeyValue, actualKeyValue); - } else { - expectedKeyValue = actualKeyValue; - } - - if (expectedKey != null) { - assertEquals(expectedKey, actualKey); - } else { - expectedKey = actualKey; - } - - if (expectedValue != null) { - assertEquals(expectedValue, actualValue); - } else { - expectedValue = actualValue; - } - } - } - - private void testEncodersOnDataset(ByteBuffer onDataset) - throws IOException{ - List dataBlockEncoders = - DataBlockEncodings.getAllEncoders(); - ByteBuffer dataset = ByteBuffer.allocate(onDataset.capacity()); - onDataset.rewind(); - dataset.put(onDataset); - onDataset.rewind(); - dataset.flip(); - - for (DataBlockEncoder encoder : dataBlockEncoders) { - testAlgorithm(dataset, encoder); - - // ensure that dataset is unchanged - dataset.rewind(); - assertEquals("Input of two methods is changed", onDataset, dataset); - } - } -} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java index 61ce0775cab..7a4b153f0cd 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java @@ -41,15 +41,9 @@ import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; public class CacheTestUtils { - private static final boolean includesMemstoreTS = true; + /*Just checks if heapsize grows when something is cached, and gets smaller when the same object is evicted*/ - /** - * Just checks if heapsize grows when something is cached, and gets smaller - * when the same object is evicted - */ - - public static void testHeapSizeChanges(final BlockCache toBeTested, - final int blockSize) { + public static void testHeapSizeChanges(final BlockCache toBeTested, final int blockSize){ HFileBlockPair[] blocks = generateHFileBlocks(blockSize, 1); long heapSize = ((HeapSize) toBeTested).heapSize(); toBeTested.cacheBlock(blocks[0].blockName, blocks[0].block); @@ -322,8 +316,7 @@ public class CacheTestUtils { HFileBlock generated = new HFileBlock(BlockType.DATA, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, - prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER, - blockSize, includesMemstoreTS); + prevBlockOffset, cachedBuffer, false, blockSize); String strKey; /* No conflicting keys */ diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index cc5772b5f5f..5ee825bb85e 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -33,7 +33,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.junit.After; @@ -43,7 +42,6 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; - import static org.junit.Assert.*; /** @@ -63,12 +61,10 @@ public class TestCacheOnWrite { private FileSystem fs; private Random rand = new Random(12983177L); private Path storeFilePath; + private Compression.Algorithm compress; + private CacheOnWriteType cowType; private BlockCache blockCache; - private String testDescription; - - private final CacheOnWriteType cowType; - private final Compression.Algorithm compress; - private final BlockEncoderTestType encoderType; + private String testName; private static final int DATA_BLOCK_SIZE = 2048; private static final int NUM_KV = 25000; @@ -80,90 +76,49 @@ public class TestCacheOnWrite { KeyValue.Type.values().length - 2; private static enum CacheOnWriteType { - DATA_BLOCKS(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, - BlockType.DATA, BlockType.ENCODED_DATA), - BLOOM_BLOCKS(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, - BlockType.BLOOM_CHUNK), - INDEX_BLOCKS(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, - BlockType.LEAF_INDEX, BlockType.INTERMEDIATE_INDEX); + DATA_BLOCKS(BlockType.DATA, CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY), + BLOOM_BLOCKS(BlockType.BLOOM_CHUNK, + CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY), + INDEX_BLOCKS(BlockType.LEAF_INDEX, + CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY); private final String confKey; - private final BlockType blockType1; - private final BlockType blockType2; + private final BlockType inlineBlockType; - private CacheOnWriteType(String confKey, BlockType blockType) { - this(confKey, blockType, blockType); - } - - private CacheOnWriteType(String confKey, BlockType blockType1, - BlockType blockType2) { - this.blockType1 = blockType1; - this.blockType2 = blockType2; + private CacheOnWriteType(BlockType inlineBlockType, String confKey) { + this.inlineBlockType = inlineBlockType; this.confKey = confKey; } public boolean shouldBeCached(BlockType blockType) { - return blockType == blockType1 || blockType == blockType2; + return blockType == inlineBlockType + || blockType == BlockType.INTERMEDIATE_INDEX + && inlineBlockType == BlockType.LEAF_INDEX; } public void modifyConf(Configuration conf) { - for (CacheOnWriteType cowType : CacheOnWriteType.values()) { + for (CacheOnWriteType cowType : CacheOnWriteType.values()) conf.setBoolean(cowType.confKey, cowType == this); - } } } - private static final DataBlockEncodings.Algorithm ENCODING_ALGO = - DataBlockEncodings.Algorithm.PREFIX; - - /** Provides fancy names for four combinations of two booleans */ - private static enum BlockEncoderTestType { - NO_BLOCK_ENCODING(false, false), - BLOCK_ENCODING_IN_CACHE_ONLY(false, true), - BLOCK_ENCODING_ON_DISK_ONLY(true, false), - BLOCK_ENCODING_EVERYWHERE(true, true); - - private final boolean encodeOnDisk; - private final boolean encodeInCache; - - BlockEncoderTestType(boolean encodeOnDisk, boolean encodeInCache) { - this.encodeOnDisk = encodeOnDisk; - this.encodeInCache = encodeInCache; - } - - public HFileDataBlockEncoder getEncoder() { - // We always use an encoded seeker. It should not have effect if there - // is no encoding in cache. - return new HFileDataBlockEncoderImpl( - encodeOnDisk ? ENCODING_ALGO : DataBlockEncodings.Algorithm.NONE, - encodeInCache ? ENCODING_ALGO : DataBlockEncodings.Algorithm.NONE, - true); - } - } - public TestCacheOnWrite(CacheOnWriteType cowType, - Compression.Algorithm compress, BlockEncoderTestType encoderType) { + Compression.Algorithm compress) { this.cowType = cowType; this.compress = compress; - this.encoderType = encoderType; - testDescription = "[cacheOnWrite=" + cowType + ", compress=" + compress + - ", encoderType=" + encoderType + "]"; - System.out.println(testDescription); + testName = "[cacheOnWrite=" + cowType + ", compress=" + compress + "]"; + System.out.println(testName); } @Parameters public static Collection getParameters() { List cowTypes = new ArrayList(); - for (CacheOnWriteType cowType : CacheOnWriteType.values()) { + for (CacheOnWriteType cowType : CacheOnWriteType.values()) for (Compression.Algorithm compress : HBaseTestingUtility.COMPRESSION_ALGORITHMS) { - for (BlockEncoderTestType encoderType : - BlockEncoderTestType.values()) { - cowTypes.add(new Object[] { cowType, compress, encoderType }); - } + cowTypes.add(new Object[] { cowType, compress }); } - } return cowTypes; } @@ -201,10 +156,10 @@ public class TestCacheOnWrite { private void readStoreFile() throws IOException { HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs, - storeFilePath, cacheConf, encoderType.getEncoder()); + storeFilePath, cacheConf); LOG.info("HFile information: " + reader); HFileScanner scanner = reader.getScanner(false, false); - assertTrue(testDescription, scanner.seekTo()); + assertTrue(testName, scanner.seekTo()); long offset = 0; HFileBlock prevBlock = null; @@ -219,11 +174,10 @@ public class TestCacheOnWrite { // Flags: don't cache the block, use pread, this is not a compaction. HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false); - BlockCacheKey blockCacheKey = HFile.getBlockCacheKey(reader.getName(), - offset); + BlockCacheKey blockCacheKey = HFile.getBlockCacheKey(reader.getName(), offset); boolean isCached = blockCache.getBlock(blockCacheKey, true) != null; boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType()); - assertEquals(testDescription + " " + block, shouldBeCached, isCached); + assertEquals(testName + " " + block, shouldBeCached, isCached); prevBlock = block; offset += block.getOnDiskSizeWithHeader(); BlockType bt = block.getBlockType(); @@ -233,10 +187,8 @@ public class TestCacheOnWrite { LOG.info("Block count by type: " + blockCountByType); String countByType = blockCountByType.toString(); - BlockType cachedDataBlockType = - encoderType.encodeInCache ? BlockType.ENCODED_DATA : BlockType.DATA; - assertEquals("{" + cachedDataBlockType - + "=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}", + assertEquals( + "{DATA=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}", countByType); reader.close(); @@ -262,9 +214,8 @@ public class TestCacheOnWrite { Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(), "test_cache_on_write"); StoreFile.Writer sfw = StoreFile.createWriter(fs, storeFileParentDir, - DATA_BLOCK_SIZE, compress, encoderType.getEncoder(), - KeyValue.COMPARATOR, conf, cacheConf, StoreFile.BloomType.ROWCOL, - NUM_KV); + DATA_BLOCK_SIZE, compress, KeyValue.COMPARATOR, conf, + cacheConf, StoreFile.BloomType.ROWCOL, NUM_KV); final int rowLen = 32; for (int i = 0; i < NUM_KV; ++i) { @@ -285,6 +236,7 @@ public class TestCacheOnWrite { storeFilePath = sfw.getPath(); } + @org.junit.Rule public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java index dd5695bbf6d..742ffe8ed84 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -27,8 +27,6 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -47,24 +45,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.DoubleOutputStream; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.Compressor; import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; @Category(MediumTests.class) -@RunWith(Parameterized.class) public class TestHFileBlock { // change this value to activate more logs private static final boolean detailedLogging = false; @@ -79,29 +69,14 @@ public class TestHFileBlock { static final Compression.Algorithm[] GZIP_ONLY = { GZ }; private static final int NUM_TEST_BLOCKS = 1000; - private static final int NUM_READER_THREADS = 26; - // Used to generate KeyValues - private static int NUM_KEYVALUES = 50; - private static int FIELD_LENGTH = 10; - private static float CHANCE_TO_REPEAT = 0.6f; + private static final int NUM_READER_THREADS = 26; private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private FileSystem fs; private int uncompressedSizeV1; - private final boolean includesMemstoreTS; - - public TestHFileBlock(boolean includesMemstoreTS) { - this.includesMemstoreTS = includesMemstoreTS; - } - - @Parameters - public static Collection parameters() { - return HBaseTestingUtility.BOOLEAN_PARAMETERIZED; - } - @Before public void setUp() throws IOException { fs = FileSystem.get(TEST_UTIL.getConfiguration()); @@ -113,74 +88,6 @@ public class TestHFileBlock { dos.writeInt(i / 100); } - private int writeTestKeyValues(OutputStream dos, int seed) - throws IOException { - List keyValues = new ArrayList(); - Random randomizer = new Random(42l + seed); // just any fixed number - - // generate keyValues - for (int i = 0 ; i < NUM_KEYVALUES ; ++i) { - byte[] row; - long timestamp; - byte[] family; - byte[] qualifier; - byte[] value; - - // generate it or repeat, it should compress well - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - row = keyValues.get(randomizer.nextInt(keyValues.size())).getRow(); - } else { - row = new byte[FIELD_LENGTH]; - randomizer.nextBytes(row); - } - if (0 == i) { - family = new byte[FIELD_LENGTH]; - randomizer.nextBytes(family); - } else { - family = keyValues.get(0).getFamily(); - } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - qualifier = keyValues.get( - randomizer.nextInt(keyValues.size())).getQualifier(); - } else { - qualifier = new byte[FIELD_LENGTH]; - randomizer.nextBytes(qualifier); - } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - value = keyValues.get(randomizer.nextInt(keyValues.size())).getValue(); - } else { - value = new byte[FIELD_LENGTH]; - randomizer.nextBytes(value); - } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - timestamp = keyValues.get( - randomizer.nextInt(keyValues.size())).getTimestamp(); - } else { - timestamp = randomizer.nextLong(); - } - - keyValues.add(new KeyValue(row, family, qualifier, timestamp, value)); - } - - // sort it and write to stream - int totalSize = 0; - Collections.sort(keyValues, KeyValue.COMPARATOR); - DataOutputStream dataOutputStream = new DataOutputStream(dos); - for (KeyValue kv : keyValues) { - totalSize += kv.getLength(); - dataOutputStream.write(kv.getBuffer(), kv.getOffset(), kv.getLength()); - if (includesMemstoreTS) { - long memstoreTS = randomizer.nextLong(); - WritableUtils.writeVLong(dataOutputStream, memstoreTS); - totalSize += WritableUtils.getVIntSize(memstoreTS); - } - } - - return totalSize; - } - - - public byte[] createTestV1Block(Compression.Algorithm algo) throws IOException { Compressor compressor = algo.getCompressor(); @@ -198,9 +105,8 @@ public class TestHFileBlock { private byte[] createTestV2Block(Compression.Algorithm algo) throws IOException { final BlockType blockType = BlockType.DATA; - HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null, - includesMemstoreTS); - DataOutputStream dos = hbw.startWriting(blockType); + HFileBlock.Writer hbw = new HFileBlock.Writer(algo); + DataOutputStream dos = hbw.startWriting(blockType, false); writeTestBlockContents(dos); byte[] headerAndData = hbw.getHeaderAndData(); assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader()); @@ -288,11 +194,10 @@ public class TestHFileBlock { Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_" + algo); FSDataOutputStream os = fs.create(path); - HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null, - includesMemstoreTS); + HFileBlock.Writer hbw = new HFileBlock.Writer(algo); long totalSize = 0; for (int blockId = 0; blockId < 2; ++blockId) { - DataOutputStream dos = hbw.startWriting(BlockType.DATA); + DataOutputStream dos = hbw.startWriting(BlockType.DATA, false); for (int i = 0; i < 1234; ++i) dos.writeInt(i); hbw.writeHeaderAndData(os); @@ -335,97 +240,6 @@ public class TestHFileBlock { } } - /** - * Test encoding/decoding data blocks. - * @throws IOException a bug or a problem with temporary files. - */ - @Test - public void testDataBlockEncoding() throws IOException { - for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) { - for (boolean pread : new boolean[] { false, true }) { - for (DataBlockEncodings.Algorithm dataBlockEncoderAlgo : - DataBlockEncodings.Algorithm.values()) { - Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_" - + algo + "_" + dataBlockEncoderAlgo.toString()); - FSDataOutputStream os = fs.create(path); - HFileDataBlockEncoder dataBlockEncoder = new HFileDataBlockEncoderImpl( - dataBlockEncoderAlgo, - DataBlockEncodings.Algorithm.NONE, - HFileDataBlockEncoderImpl.NO_ENCODED_SEEK); - HFileBlock.Writer hbw = new HFileBlock.Writer(algo, dataBlockEncoder, - includesMemstoreTS); - long totalSize = 0; - List blockSizes = new ArrayList(); - List blockContent = new ArrayList(); - for (int blockId = 0; blockId < 2; ++blockId) { - DataOutputStream dos = hbw.startWriting(BlockType.DATA); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DoubleOutputStream doubleOutputStream = - new DoubleOutputStream(dos, baos); - - blockSizes.add(writeTestKeyValues(doubleOutputStream, blockId)); - - ByteBuffer buf = ByteBuffer.wrap(baos.toByteArray()); - buf.rewind(); - blockContent.add(buf); - - hbw.writeHeaderAndData(os); - totalSize += hbw.getOnDiskSizeWithHeader(); - } - os.close(); - - FSDataInputStream is = fs.open(path); - HFileBlock.FSReaderV2 hbr = new HFileBlock.FSReaderV2(is, algo, - totalSize, dataBlockEncoder); - hbr.setIncludesMemstoreTS(includesMemstoreTS); - - HFileBlock b; - int pos = 0; - for (int blockId = 0; blockId < 2; ++blockId) { - b = hbr.readBlockData(pos, -1, -1, pread); - b.sanityCheck(); - pos += b.getOnDiskSizeWithHeader(); - - assertEquals((int) blockSizes.get(blockId), - b.getUncompressedSizeWithoutHeader()); - ByteBuffer actualBuffer = b.getBufferWithoutHeader(); - ByteBuffer expectedBuffer = blockContent.get(blockId); - expectedBuffer.rewind(); - - // test if content matches, produce nice message - if (!actualBuffer.equals(expectedBuffer)) { - int prefix = 0; - while (prefix < expectedBuffer.limit() && - expectedBuffer.get(prefix) == actualBuffer.get(prefix)) { - prefix++; - } - - int kvCount = 0; - - while (actualBuffer.position() + 2 * Bytes.SIZEOF_INT < - actualBuffer.limit()) { - int keyLength = actualBuffer.getInt(); - int valueLength = actualBuffer.getInt(); - kvCount++; - actualBuffer.position(actualBuffer.position() + - keyLength + valueLength); - } - - fail(String.format( - "Content mismath compression: %s encoding: %s" + - " pread: %s commonPrefix: %d kvCount: %d" + - " expected char: %s actual char %s", algo.getName(), - dataBlockEncoderAlgo.toString(), pread, prefix, kvCount, - expectedBuffer.get(prefix), - actualBuffer.get(prefix))); - } - } - is.close(); - } - } - } - } - @Test public void testPreviousOffset() throws IOException { for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) { @@ -632,17 +446,13 @@ public class TestHFileBlock { ) throws IOException { boolean cacheOnWrite = expectedContents != null; FSDataOutputStream os = fs.create(path); - HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo, null, - includesMemstoreTS); + HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo); Map prevOffsetByType = new HashMap(); long totalSize = 0; for (int i = 0; i < NUM_TEST_BLOCKS; ++i) { int blockTypeOrdinal = rand.nextInt(BlockType.values().length); - if (blockTypeOrdinal == BlockType.ENCODED_DATA.ordinal()) { - blockTypeOrdinal = BlockType.DATA.ordinal(); - } BlockType bt = BlockType.values()[blockTypeOrdinal]; - DataOutputStream dos = hbw.startWriting(bt); + DataOutputStream dos = hbw.startWriting(bt, cacheOnWrite); for (int j = 0; j < rand.nextInt(500); ++j) { // This might compress well. dos.writeShort(i + 1); @@ -691,7 +501,7 @@ public class TestHFileBlock { byte[] byteArr = new byte[HFileBlock.HEADER_SIZE + size]; ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size); HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, -1, includesMemstoreTS); + true, -1); long byteBufferExpectedSize = ClassSize.align(ClassSize.estimateBase(buf.getClass(), true) + HFileBlock.HEADER_SIZE + size); diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java index 91947b95c44..d8730867651 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -20,10 +20,6 @@ package org.apache.hadoop.hbase.io.hfile; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -48,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader; import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; + import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,6 +52,8 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import static org.junit.Assert.*; + @RunWith(Parameterized.class) @Category(MediumTests.class) public class TestHFileBlockIndex { @@ -93,8 +92,6 @@ public class TestHFileBlockIndex { private static final int[] UNCOMPRESSED_INDEX_SIZES = { 19187, 21813, 23086 }; - private static final boolean includesMemstoreTS = true; - static { assert INDEX_CHUNK_SIZES.length == EXPECTED_NUM_LEVELS.length; assert INDEX_CHUNK_SIZES.length == UNCOMPRESSED_INDEX_SIZES.length; @@ -213,14 +210,13 @@ public class TestHFileBlockIndex { private void writeWholeIndex() throws IOException { assertEquals(0, keys.size()); - HFileBlock.Writer hbw = new HFileBlock.Writer(compr, null, - includesMemstoreTS); + HFileBlock.Writer hbw = new HFileBlock.Writer(compr); FSDataOutputStream outputStream = fs.create(path); HFileBlockIndex.BlockIndexWriter biw = new HFileBlockIndex.BlockIndexWriter(hbw, null, null); for (int i = 0; i < NUM_DATA_BLOCKS; ++i) { - hbw.startWriting(BlockType.DATA).write( + hbw.startWriting(BlockType.DATA, false).write( String.valueOf(rand.nextInt(1000)).getBytes()); long blockOffset = outputStream.getPos(); hbw.writeHeaderAndData(outputStream); @@ -255,7 +251,7 @@ public class TestHFileBlockIndex { boolean isClosing) throws IOException { while (biw.shouldWriteBlock(isClosing)) { long offset = outputStream.getPos(); - biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType())); + biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType(), false)); hbw.writeHeaderAndData(outputStream); biw.blockWritten(offset, hbw.getOnDiskSizeWithHeader(), hbw.getUncompressedSizeWithoutHeader()); @@ -483,7 +479,7 @@ public class TestHFileBlockIndex { { HFile.Writer writer = HFile.getWriterFactory(conf, cacheConf).createWriter(fs, - hfilePath, SMALL_BLOCK_SIZE, compr, null, KeyValue.KEY_COMPARATOR); + hfilePath, SMALL_BLOCK_SIZE, compr, KeyValue.KEY_COMPARATOR); Random rand = new Random(19231737); for (int i = 0; i < NUM_KV; ++i) { diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java deleted file mode 100644 index 85f17db1f16..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.io.hfile; - -import static org.junit.Assert.*; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.io.HeapSize; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings.Algorithm; -import org.apache.hadoop.hbase.io.encoding.RedundantKVGenerator; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; -import org.apache.hadoop.hbase.util.Pair; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestHFileDataBlockEncoder { - private Configuration conf; - private final HBaseTestingUtility TEST_UTIL = - new HBaseTestingUtility(); - private HFileDataBlockEncoderImpl blockEncoder; - private RedundantKVGenerator generator = new RedundantKVGenerator(); - private SchemaConfigured UNKNOWN_TABLE_AND_CF = - SchemaConfigured.createUnknown(); - private boolean includesMemstoreTS; - - /** - * Create test for given data block encoding configuration. - * @param blockEncoder What kind of encoding policy will be used. - */ - public TestHFileDataBlockEncoder(HFileDataBlockEncoderImpl blockEncoder, - boolean includesMemstoreTS) { - this.blockEncoder = blockEncoder; - this.includesMemstoreTS = includesMemstoreTS; - } - - /** - * Preparation before JUnit test. - */ - @Before - public void setUp() { - conf = TEST_UTIL.getConfiguration(); - SchemaMetrics.configureGlobally(conf); - } - - /** - * Cleanup after JUnit test. - */ - @After - public void tearDown() throws IOException { - TEST_UTIL.cleanupTestDir(); - } - - /** - * Test putting and taking out blocks into cache with different - * encoding options. - */ - @Test - public void testEncodingWithCache() { - HFileBlock block = getSampleHFileBlock(); - LruBlockCache blockCache = - new LruBlockCache(8 * 1024 * 1024, 32 * 1024); - - HFileBlock cacheBlock = blockEncoder.beforeBlockCache(block, - includesMemstoreTS); - BlockCacheKey cacheKey = new BlockCacheKey("test", 0); - blockCache.cacheBlock(cacheKey, cacheBlock); - - HeapSize heapSize = blockCache.getBlock(cacheKey, false); - assertTrue(heapSize instanceof HFileBlock); - - HFileBlock afterCache = (HFileBlock) heapSize; - HFileBlock returnedBlock = blockEncoder.afterBlockCache(afterCache, - false, includesMemstoreTS); - - if (!blockEncoder.useEncodedSeek() || - blockEncoder.getInCache() == Algorithm.NONE) { - assertEquals(block.getBufferWithHeader(), - returnedBlock.getBufferWithHeader()); - } else { - if (BlockType.ENCODED_DATA != returnedBlock.getBlockType()) { - System.out.println(blockEncoder); - } - assertEquals(BlockType.ENCODED_DATA, returnedBlock.getBlockType()); - } - } - - /** - * Test writing to disk. - */ - @Test - public void testEncodingWritePath() { - // usually we have just block without headers, but don't complicate that - HFileBlock block = getSampleHFileBlock(); - Pair result = - blockEncoder.beforeWriteToDisk(block.getBufferWithoutHeader(), - includesMemstoreTS); - - int size = result.getFirst().limit() - HFileBlock.HEADER_SIZE; - HFileBlock blockOnDisk = new HFileBlock(result.getSecond(), - size, size, -1, result.getFirst(), HFileBlock.FILL_HEADER, 0, - includesMemstoreTS); - - if (blockEncoder.getOnDisk() != - DataBlockEncodings.Algorithm.NONE) { - assertEquals(BlockType.ENCODED_DATA, blockOnDisk.getBlockType()); - assertEquals(blockEncoder.getOnDisk().getId(), - blockOnDisk.getDataBlockEncodingId()); - } else { - assertEquals(BlockType.DATA, blockOnDisk.getBlockType()); - } - } - - /** - * Test reading from disk. - */ - @Test - public void testEncodingReadPath() { - HFileBlock origBlock = getSampleHFileBlock(); - HFileBlock afterDisk = blockEncoder.afterReadFromDisk(origBlock); - blockEncoder.afterReadFromDiskAndPuttingInCache(afterDisk, false, - includesMemstoreTS); - } - - private HFileBlock getSampleHFileBlock() { - ByteBuffer keyValues = RedundantKVGenerator.convertKvToByteBuffer( - generator.generateTestKeyValues(60), includesMemstoreTS); - int size = keyValues.limit(); - ByteBuffer buf = ByteBuffer.allocate(size + HFileBlock.HEADER_SIZE); - buf.position(HFileBlock.HEADER_SIZE); - keyValues.rewind(); - buf.put(keyValues); - HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, 0, includesMemstoreTS); - UNKNOWN_TABLE_AND_CF.passSchemaMetricsTo(b); - return b; - } - - /** - * @return All possible data block encoding configurations - */ - @Parameters - public static Collection getAllConfigurations() { - List configurations = - new ArrayList(); - - for (Algorithm diskAlgo : DataBlockEncodings.Algorithm.values()) { - for (Algorithm cacheAlgo : DataBlockEncodings.Algorithm.values()) { - for (boolean useEncodedSeek : new boolean[] {false, true}) { - for (boolean includesMemstoreTS : new boolean[] {false, true}) { - configurations.add( new Object[] { - new HFileDataBlockEncoderImpl(diskAlgo, cacheAlgo, - useEncodedSeek), - new Boolean(includesMemstoreTS)}); - } - } - } - } - - return configurations; - } -} diff --git a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java index 5ff4dfda20a..474cbcf973f 100644 --- a/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java +++ b/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java @@ -76,7 +76,7 @@ public class TestHFileWriterV2 { final Compression.Algorithm COMPRESS_ALGO = Compression.Algorithm.GZ; HFileWriterV2 writer = new HFileWriterV2(conf, new CacheConfig(conf), fs, - hfilePath, 4096, COMPRESS_ALGO, null, KeyValue.KEY_COMPARATOR); + hfilePath, 4096, COMPRESS_ALGO, KeyValue.KEY_COMPARATOR); long totalKeyLength = 0; long totalValueLength = 0; @@ -125,12 +125,10 @@ public class TestHFileWriterV2 { new HFileBlock.FSReaderV2(fsdis, COMPRESS_ALGO, fileSize); // Comparator class name is stored in the trailer in version 2. RawComparator comparator = trailer.createComparator(); - HFileBlockIndex.BlockIndexReader dataBlockIndexReader = - new HFileBlockIndex.BlockIndexReader(comparator, - trailer.getNumDataIndexLevels()); - HFileBlockIndex.BlockIndexReader metaBlockIndexReader = - new HFileBlockIndex.BlockIndexReader( - Bytes.BYTES_RAWCOMPARATOR, 1); + HFileBlockIndex.BlockIndexReader dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator, + trailer.getNumDataIndexLevels()); + HFileBlockIndex.BlockIndexReader metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader( + Bytes.BYTES_RAWCOMPARATOR, 1); HFileBlock.BlockIterator blockIter = blockReader.blockRange( trailer.getLoadOnOpenDataOffset(), @@ -148,10 +146,8 @@ public class TestHFileWriterV2 { // File info FileInfo fileInfo = new FileInfo(); fileInfo.readFields(blockIter.nextBlockAsStream(BlockType.FILE_INFO)); - byte [] keyValueFormatVersion = fileInfo.get( - HFileWriterV2.KEY_VALUE_VERSION); - boolean includeMemstoreTS = keyValueFormatVersion != null && - Bytes.toInt(keyValueFormatVersion) > 0; + byte [] keyValueFormatVersion = fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION); + boolean includeMemstoreTS = (keyValueFormatVersion != null && Bytes.toInt(keyValueFormatVersion) > 0); // Counters for the number of key/value pairs and the number of blocks int entriesRead = 0; diff --git a/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 778a27318db..e3bc3233f37 100644 --- a/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -133,9 +133,6 @@ public class TestImportExport { 5, /* versions */ true /* keep deleted cells */, HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, HColumnDescriptor.DEFAULT_IN_MEMORY, HColumnDescriptor.DEFAULT_BLOCKCACHE, HColumnDescriptor.DEFAULT_BLOCKSIZE, @@ -182,9 +179,6 @@ public class TestImportExport { 5, /* versions */ true /* keep deleted cells */, HColumnDescriptor.DEFAULT_COMPRESSION, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_ON_DISK, - HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING_IN_CACHE, - HColumnDescriptor.DEFAULT_ENCODED_DATA_BLOCK_SEEK, HColumnDescriptor.DEFAULT_IN_MEMORY, HColumnDescriptor.DEFAULT_BLOCKCACHE, HColumnDescriptor.DEFAULT_BLOCKSIZE, diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java b/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java index 456eb77ade2..45aed4ded08 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java @@ -184,8 +184,8 @@ public class CreateRandomStoreFile { } StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir, blockSize, - compr, null, KeyValue.COMPARATOR, conf, new CacheConfig(conf), - bloomType, numKV); + compr, KeyValue.COMPARATOR, conf, new CacheConfig(conf), bloomType, + numKV); rand = new Random(); LOG.info("Writing " + numKV + " key/value pairs"); diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java deleted file mode 100644 index 5c11e17ce1a..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java +++ /dev/null @@ -1,585 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.regionserver; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.cli.PosixParser; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.encoding.EncodedDataBlock; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.Compression; -import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.compress.Compressor; -import org.apache.hadoop.io.compress.Decompressor; - -/** - * Tests various algorithms for key compression on an existing HFile. Useful - * for testing, debugging and benchmarking. - */ -public class DataBlockEncodingTool { - private static final Log LOG = LogFactory.getLog( - DataBlockEncodingTool.class); - - private static final boolean includesMemstoreTS = true; - - /** - * How many times should benchmark run. - * More times means better data in terms of statistics. - * It has to be larger than BENCHMARK_N_OMIT. - */ - public static int BENCHMARK_N_TIMES = 12; - - /** - * How many first runs should omit benchmark. - * Usually it is one in order to exclude setup cost. - * Has to be 0 or larger. - */ - public static int BENCHMARK_N_OMIT = 2; - - private List codecs = new ArrayList(); - private int totalPrefixLength = 0; - private int totalKeyLength = 0; - private int totalValueLength = 0; - private int totalKeyRedundancyLength = 0; - - final private String compressionAlgorithmName; - final private Algorithm compressionAlgorithm; - final private Compressor compressor; - final private Decompressor decompressor; - - /** - * @param compressionAlgorithmName What kind of algorithm should be used - * as baseline for comparison (e.g. lzo, gz). - */ - public DataBlockEncodingTool(String compressionAlgorithmName) { - this.compressionAlgorithmName = compressionAlgorithmName; - this.compressionAlgorithm = Compression.getCompressionAlgorithmByName( - compressionAlgorithmName); - this.compressor = this.compressionAlgorithm.getCompressor(); - this.decompressor = this.compressionAlgorithm.getDecompressor(); - } - /** - * Check statistics for given HFile for different data block encoders. - * @param scanner Of file which will be compressed. - * @param kvLimit Maximal count of KeyValue which will be processed. - * @throws IOException thrown if scanner is invalid - */ - public void checkStatistics(final KeyValueScanner scanner, final int kvLimit) - throws IOException { - scanner.seek(KeyValue.LOWESTKEY); - - KeyValue currentKv; - - byte[] previousKey = null; - byte[] currentKey; - - List dataBlockEncoders = - DataBlockEncodings.getAllEncoders(); - - for (DataBlockEncoder d : dataBlockEncoders) { - codecs.add(new EncodedDataBlock(d, includesMemstoreTS)); - } - - int j = 0; - while ((currentKv = scanner.next()) != null && j < kvLimit) { - // Iterates through key/value pairs - j++; - currentKey = currentKv.getKey(); - if (previousKey != null) { - for (int i = 0 ; i < previousKey.length && i < currentKey.length && - previousKey[i] == currentKey[i] ; ++i) { - totalKeyRedundancyLength++; - } - } - - for (EncodedDataBlock codec : codecs) { - codec.addKv(currentKv); - } - - previousKey = currentKey; - - totalPrefixLength += currentKv.getLength() - currentKv.getKeyLength() - - currentKv.getValueLength(); - totalKeyLength += currentKv.getKeyLength(); - totalValueLength += currentKv.getValueLength(); - } - } - - /** - * Verify if all data block encoders are working properly. - * - * @param scanner Of file which was compressed. - * @param kvLimit Maximal count of KeyValue which will be processed. - * @return true if all data block encoders compressed/decompressed correctly. - * @throws IOException thrown if scanner is invalid - */ - public boolean verifyCodecs(final KeyValueScanner scanner, final int kvLimit) - throws IOException { - KeyValue currentKv; - - scanner.seek(KeyValue.LOWESTKEY); - List> codecIterators = - new ArrayList>(); - for(EncodedDataBlock codec : codecs) { - codecIterators.add(codec.getIterator()); - } - - int j = 0; - while ((currentKv = scanner.next()) != null && j < kvLimit) { - // Iterates through key/value pairs - ++j; - for (Iterator it : codecIterators) { - KeyValue codecKv = it.next(); - if (codecKv == null || 0 != Bytes.compareTo( - codecKv.getBuffer(), codecKv.getOffset(), codecKv.getLength(), - currentKv.getBuffer(), currentKv.getOffset(), - currentKv.getLength())) { - if (codecKv == null) { - LOG.error("There is a bug in codec " + it + - " it returned null KeyValue,"); - } else { - int prefix = 0; - int limitLength = 2 * Bytes.SIZEOF_INT + - Math.min(codecKv.getLength(), currentKv.getLength()); - while (prefix < limitLength && - codecKv.getBuffer()[prefix + codecKv.getOffset()] == - currentKv.getBuffer()[prefix + currentKv.getOffset()]) { - prefix++; - } - - LOG.error("There is bug in codec " + it.toString() + - "\n on element " + j + - "\n codecKv.getKeyLength() " + codecKv.getKeyLength() + - "\n codecKv.getValueLength() " + codecKv.getValueLength() + - "\n codecKv.getLength() " + codecKv.getLength() + - "\n currentKv.getKeyLength() " + currentKv.getKeyLength() + - "\n currentKv.getValueLength() " + currentKv.getValueLength() + - "\n codecKv.getLength() " + currentKv.getLength() + - "\n currentKV rowLength " + currentKv.getRowLength() + - " familyName " + currentKv.getFamilyLength() + - " qualifier " + currentKv.getQualifierLength() + - "\n prefix " + prefix + - "\n codecKv '" + Bytes.toStringBinary(codecKv.getBuffer(), - codecKv.getOffset(), prefix) + "' diff '" + - Bytes.toStringBinary(codecKv.getBuffer(), - codecKv.getOffset() + prefix, codecKv.getLength() - - prefix) + "'" + - "\n currentKv '" + Bytes.toStringBinary( - currentKv.getBuffer(), - currentKv.getOffset(), prefix) + "' diff '" + - Bytes.toStringBinary(currentKv.getBuffer(), - currentKv.getOffset() + prefix, currentKv.getLength() - - prefix) + "'" - ); - } - return false; - } - } - } - - LOG.info("Verification was successful!"); - - return true; - } - - /** - * Benchmark codec's speed. - */ - public void benchmarkCodecs() { - int prevTotalSize = -1; - for (EncodedDataBlock codec : codecs) { - prevTotalSize = benchmarkEncoder(prevTotalSize, codec); - } - - byte[] buffer = codecs.get(0).getRawKeyValues(); - - benchmarkDefaultCompression(prevTotalSize, buffer); - } - - /** - * Benchmark compression/decompression throughput. - * @param previousTotalSize Total size used for verification. Use -1 if - * unknown. - * @param codec Tested encoder. - * @return Size of uncompressed data. - */ - private int benchmarkEncoder(int previousTotalSize, EncodedDataBlock codec) { - int prevTotalSize = previousTotalSize; - int totalSize = 0; - - // decompression time - List durations = new ArrayList(); - for (int itTime = 0 ; itTime < BENCHMARK_N_TIMES ; ++itTime) { - totalSize = 0; - - Iterator it; - - it = codec.getIterator(); - - // count only the algorithm time, without memory allocations - // (expect first time) - final long startTime = System.nanoTime(); - while (it.hasNext()) { - totalSize += it.next().getLength(); - } - final long finishTime = System.nanoTime(); - if (itTime >= BENCHMARK_N_OMIT) { - durations.add(finishTime - startTime); - } - - if (prevTotalSize != -1 && prevTotalSize != totalSize) { - throw new IllegalStateException(String.format( - "Algorithm '%s' decoded data to different size", codec.toString())); - } - prevTotalSize = totalSize; - } - - // compression time - List compressDurations = new ArrayList(); - for (int itTime = 0 ; itTime < BENCHMARK_N_TIMES ; ++itTime) { - final long startTime = System.nanoTime(); - codec.doCompressData(); - final long finishTime = System.nanoTime(); - if (itTime >= BENCHMARK_N_OMIT) { - compressDurations.add(finishTime - startTime); - } - } - - System.out.println(codec.toString() + ":"); - printBenchmarkResult(totalSize, compressDurations, false); - printBenchmarkResult(totalSize, durations, true); - - return prevTotalSize; - } - - private void benchmarkDefaultCompression(int totalSize, byte[] rawBuffer) { - benchmarkAlgorithm(compressionAlgorithm, compressor, decompressor, - compressionAlgorithmName.toUpperCase(), rawBuffer, 0, totalSize); - } - - /** - * Check decompress performance of a given algorithm and print it. - * @param algorithm Compression algorithm. - * @param compressorCodec Compressor to be tested. - * @param decompressorCodec Decompressor of the same algorithm. - * @param name Name of algorithm. - * @param buffer Buffer to be compressed. - * @param offset Position of the beginning of the data. - * @param length Length of data in buffer. - */ - public static void benchmarkAlgorithm( - Compression.Algorithm algorithm, - Compressor compressorCodec, - Decompressor decompressorCodec, - String name, - byte[] buffer, int offset, int length) { - System.out.println(name + ":"); - - // compress it - List compressDurations = new ArrayList(); - ByteArrayOutputStream compressedStream = new ByteArrayOutputStream(); - OutputStream compressingStream; - try { - for (int itTime = 0 ; itTime < BENCHMARK_N_TIMES ; ++itTime) { - final long startTime = System.nanoTime(); - compressingStream = algorithm.createCompressionStream( - compressedStream, compressorCodec, 0); - compressingStream.write(buffer, offset, length); - compressingStream.flush(); - compressedStream.toByteArray(); - - final long finishTime = System.nanoTime(); - - // add time record - if (itTime >= BENCHMARK_N_OMIT) { - compressDurations.add(finishTime - startTime); - } - - if (itTime + 1 < BENCHMARK_N_TIMES) { // not the last one - compressedStream.reset(); - } - } - } catch (IOException e) { - throw new RuntimeException(String.format( - "Benchmark, or encoding algorithm '%s' cause some stream problems", - name), e); - } - printBenchmarkResult(length, compressDurations, false); - - - byte[] compBuffer = compressedStream.toByteArray(); - - // uncompress it several times and measure performance - List durations = new ArrayList(); - for (int itTime = 0 ; itTime < BENCHMARK_N_TIMES ; ++itTime) { - final long startTime = System.nanoTime(); - byte[] newBuf = new byte[length + 1]; - - try { - - ByteArrayInputStream downStream = new ByteArrayInputStream(compBuffer, - 0, compBuffer.length); - InputStream decompressedStream = algorithm.createDecompressionStream( - downStream, decompressorCodec, 0); - - int destOffset = 0; - int nextChunk; - while ((nextChunk = decompressedStream.available()) > 0) { - destOffset += decompressedStream.read(newBuf, destOffset, nextChunk); - } - decompressedStream.close(); - - // iterate over KeyValue - KeyValue kv; - for (int pos = 0 ; pos < length ; pos += kv.getLength()) { - kv = new KeyValue(newBuf, pos); - } - - } catch (IOException e) { - throw new RuntimeException(String.format( - "Decoding path in '%s' algorithm cause exception ", name), e); - } - - final long finishTime = System.nanoTime(); - - // check correctness - if (0 != Bytes.compareTo(buffer, 0, length, newBuf, 0, length)) { - int prefix = 0; - for( ; prefix < buffer.length && prefix < newBuf.length ; ++prefix) { - if (buffer[prefix] != newBuf[prefix]) { - break; - } - } - throw new RuntimeException(String.format( - "Algorithm '%s' is corrupting the data", name)); - } - - // add time record - if (itTime >= BENCHMARK_N_OMIT) { - durations.add(finishTime - startTime); - } - } - printBenchmarkResult(length, durations, true); - } - - private static void printBenchmarkResult(int totalSize, - List durationsInNanoSed, boolean isDecompression) { - long meanTime = 0; - for (long time : durationsInNanoSed) { - meanTime += time; - } - meanTime /= durationsInNanoSed.size(); - - long standardDev = 0; - for (long time : durationsInNanoSed) { - standardDev += (time - meanTime) * (time - meanTime); - } - standardDev = (long) Math.sqrt(standardDev / durationsInNanoSed.size()); - - final double million = 1000.0 * 1000.0 * 1000.0; - double mbPerSec = (totalSize * million) / (1024.0 * 1024.0 * meanTime); - double mbPerSecDev = (totalSize * million) / - (1024.0 * 1024.0 * (meanTime - standardDev)); - - System.out.println(String.format( - " %s performance:%s %6.2f MB/s (+/- %.2f MB/s)", - isDecompression ? "Decompression" : "Compression", - isDecompression ? "" : " ", - mbPerSec, mbPerSecDev - mbPerSec)); - } - - /** - * Display statistics of different compression algorithms. - */ - public void displayStatistics() { - int totalLength = totalPrefixLength + totalKeyLength + totalValueLength; - compressor.reset(); - - for(EncodedDataBlock codec : codecs) { - System.out.println(codec.toString()); - int saved = totalKeyLength + totalPrefixLength + totalValueLength - - codec.getSize(); - System.out.println( - String.format(" Saved bytes: %8d", saved)); - double keyRatio = (saved * 100.0) / (totalPrefixLength + totalKeyLength); - double allRatio = (saved * 100.0) / totalLength; - System.out.println( - String.format(" Key compression ratio: %.2f %%", keyRatio)); - System.out.println( - String.format(" All compression ratio: %.2f %%", allRatio)); - int compressedSize = codec.checkCompressedSize(compressor); - System.out.println( - String.format(" %s compressed size: %8d", - compressionAlgorithmName.toUpperCase(), compressedSize)); - double lzoRatio = 100.0 * (1.0 - compressedSize / (0.0 + totalLength)); - System.out.println( - String.format(" %s compression ratio: %.2f %%", - compressionAlgorithmName.toUpperCase(), lzoRatio)); - } - - System.out.println( - String.format("Total KV prefix length: %8d", totalPrefixLength)); - System.out.println( - String.format("Total key length: %8d", totalKeyLength)); - System.out.println( - String.format("Total key redundancy: %8d", - totalKeyRedundancyLength)); - System.out.println( - String.format("Total value length: %8d", totalValueLength)); - } - - /** - * Test a data block encoder on the given HFile. Output results to console. - * @param kvLimit The limit of KeyValue which will be analyzed. - * @param hfilePath an HFile path on the file system. - * @param compressionName Compression algorithm used for comparison. - * @param doBenchmark Run performance benchmarks. - * @param doVerify Verify correctness. - * @throws IOException When pathName is incorrect. - */ - public static void testCodecs(int kvLimit, String hfilePath, - String compressionName, boolean doBenchmark, boolean doVerify) - throws IOException { - // create environment - Path path = new Path(hfilePath); - Configuration conf = HBaseConfiguration.create(); - CacheConfig cacheConf = new CacheConfig(conf); - FileSystem fs = FileSystem.get(conf); - StoreFile hsf = new StoreFile(fs, path, conf, cacheConf, - StoreFile.BloomType.NONE, new NoOpDataBlockEncoder()); - - StoreFile.Reader reader = hsf.createReader(); - reader.loadFileInfo(); - KeyValueScanner scanner = reader.getStoreFileScanner(true, true); - - // run the utilities - DataBlockEncodingTool comp = new DataBlockEncodingTool(compressionName); - comp.checkStatistics(scanner, kvLimit); - if (doVerify) { - comp.verifyCodecs(scanner, kvLimit); - } - if (doBenchmark) { - comp.benchmarkCodecs(); - } - comp.displayStatistics(); - - // cleanup - scanner.close(); - reader.close(cacheConf.shouldEvictOnClose()); - } - - private static void printUsage(Options options) { - System.err.println("Usage:"); - System.err.println(String.format("./hbase %s ", - DataBlockEncodingTool.class.getName())); - System.err.println("Options:"); - for (Object it : options.getOptions()) { - Option opt = (Option) it; - if (opt.hasArg()) { - System.err.println(String.format("-%s %s: %s", opt.getOpt(), - opt.getArgName(), opt.getDescription())); - } else { - System.err.println(String.format("-%s: %s", opt.getOpt(), - opt.getDescription())); - } - } - } - - /** - * A command line interface to benchmarks. - * @param args Should have length at least 1 and holds the file path to HFile. - * @throws IOException If you specified the wrong file. - */ - public static void main(final String[] args) throws IOException { - // set up user arguments - Options options = new Options(); - options.addOption("f", true, "HFile to analyse (REQUIRED)"); - options.getOption("f").setArgName("FILENAME"); - options.addOption("n", true, - "Limit number of KeyValue which will be analysed"); - options.getOption("n").setArgName("NUMBER"); - options.addOption("b", false, "Measure read throughput"); - options.addOption("c", false, "Omit corectness tests."); - options.addOption("a", true, - "What kind of compression algorithm use for comparison."); - - // parse arguments - CommandLineParser parser = new PosixParser(); - CommandLine cmd = null; - try { - cmd = parser.parse(options, args); - } catch (ParseException e) { - System.err.println("Could not parse arguments!"); - System.exit(-1); - return; // avoid warning - } - - int kvLimit = Integer.MAX_VALUE; - if (cmd.hasOption("n")) { - kvLimit = Integer.parseInt(cmd.getOptionValue("n")); - } - - // basic argument sanity checks - if (!cmd.hasOption("f")) { - System.err.println("ERROR: Filename is required!"); - printUsage(options); - System.exit(-1); - } - - if (!(new File(cmd.getOptionValue("f"))).exists()) { - System.err.println(String.format("ERROR: file '%s' doesn't exist!", - cmd.getOptionValue("f"))); - printUsage(options); - System.exit(-1); - } - - String pathName = cmd.getOptionValue("f"); - String compressionName = "gz"; - if (cmd.hasOption("a")) { - compressionName = cmd.getOptionValue("a"); - } - boolean doBenchmark = cmd.hasOption("b"); - boolean doVerify = !cmd.hasOption("c"); - - testCodecs(kvLimit, pathName, compressionName, doBenchmark, doVerify); - } - -} diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java deleted file mode 100644 index 3b6511305bd..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java +++ /dev/null @@ -1,180 +0,0 @@ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings.Algorithm; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.LruBlockCache; -import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType; - -/** - * Test seek performance for encoded data blocks. Read an HFile and do several - * random seeks. - */ -public class EncodedSeekPerformanceTest { - private static final double NANOSEC_IN_SEC = 1000.0 * 1000.0 * 1000.0; - private static final double BYTES_IN_MEGABYTES = 1024.0 * 1024.0; - /** Default number of seeks which will be used in benchmark. */ - public static int DEFAULT_NUMBER_OF_SEEKS = 10000; - - private final HBaseTestingUtility testingUtility = new HBaseTestingUtility(); - private Configuration configuration = testingUtility.getConfiguration(); - private CacheConfig cacheConf = new CacheConfig(configuration); - private Random randomizer; - private int numberOfSeeks; - - /** Use this benchmark with default options */ - public EncodedSeekPerformanceTest() { - configuration.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.5f); - randomizer = new Random(42l); - numberOfSeeks = DEFAULT_NUMBER_OF_SEEKS; - } - - private List prepareListOfTestSeeks(Path path) throws IOException { - List allKeyValues = new ArrayList(); - - // read all of the key values - StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), - path, configuration, cacheConf, BloomType.NONE, null); - - StoreFile.Reader reader = storeFile.createReader(); - StoreFileScanner scanner = reader.getStoreFileScanner(true, false); - KeyValue current; - - scanner.seek(KeyValue.LOWESTKEY); - while (null != (current = scanner.next())) { - allKeyValues.add(current); - } - - storeFile.closeReader(cacheConf.shouldEvictOnClose()); - - // pick seeks by random - List seeks = new ArrayList(); - for (int i = 0 ; i < numberOfSeeks ; ++i) { - KeyValue keyValue = allKeyValues.get( - randomizer.nextInt(allKeyValues.size())); - seeks.add(keyValue); - } - - clearBlockCache(); - - return seeks; - } - - private void runTest(Path path, HFileDataBlockEncoder blockEncoder, - List seeks) throws IOException { - // read all of the key values - StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), - path, configuration, cacheConf, BloomType.NONE, blockEncoder); - - int totalSize = 0; - - StoreFile.Reader reader = storeFile.createReader(); - StoreFileScanner scanner = reader.getStoreFileScanner(true, false); - - long startReadingTime = System.nanoTime(); - KeyValue current; - scanner.seek(KeyValue.LOWESTKEY); - while (null != (current = scanner.next())) { // just iterate it! - totalSize += current.getLength(); - } - long finishReadingTime = System.nanoTime(); - - // do seeks - long startSeeksTime = System.nanoTime(); - for (KeyValue keyValue : seeks) { - scanner.seek(keyValue); - KeyValue toVerify = scanner.next(); - if (!keyValue.equals(toVerify)) { - System.out.println(String.format("KeyValue doesn't match:\n" + - "Orig key: %s\n" + - "Ret key: %s", keyValue.getKeyString(), toVerify.getKeyString())); - break; - } - } - long finishSeeksTime = System.nanoTime(); - - // write some stats - double readInMbPerSec = (totalSize * NANOSEC_IN_SEC) / - (BYTES_IN_MEGABYTES * (finishReadingTime - startReadingTime)); - double seeksPerSec = (seeks.size() * NANOSEC_IN_SEC) / - (finishSeeksTime - startSeeksTime); - - storeFile.closeReader(cacheConf.shouldEvictOnClose()); - clearBlockCache(); - - System.out.println(blockEncoder); - System.out.println(String.format(" Read speed: %8.2f (MB/s)", - readInMbPerSec)); - System.out.println(String.format(" Seeks per second: %8.2f (#/s)", - seeksPerSec)); - } - - /** - * @param path Path to the HFile which will be used. - * @param encoders List of encoders which will be used for tests. - * @throws IOException if there is a bug while reading from disk - */ - public void runTests(Path path, List encoders) - throws IOException { - List seeks = prepareListOfTestSeeks(path); - - for (HFileDataBlockEncoder blockEncoder : encoders) { - runTest(path, blockEncoder, seeks); - } - } - - /** - * Command line interface: - * @param args Takes one argument - file size. - * @throws IOException if there is a bug while reading from disk - */ - public static void main(final String[] args) throws IOException { - if (args.length < 1) { - printUsage(); - System.exit(-1); - } - - //System.setProperty("org.apache.commons.logging.Log", - // "org.apache.commons.logging.impl.NoOpLog"); - - Path path = new Path(args[0]); - List encoders = - new ArrayList(); - - encoders.add(new HFileDataBlockEncoderImpl(Algorithm.NONE, Algorithm.NONE, - false)); - for (Algorithm encodingAlgo : Algorithm.values()) { - encoders.add(new HFileDataBlockEncoderImpl(Algorithm.NONE, encodingAlgo, - false)); - } - - for (Algorithm encodingAlgo : Algorithm.values()) { - encoders.add(new HFileDataBlockEncoderImpl(Algorithm.NONE, encodingAlgo, - true)); - } - - EncodedSeekPerformanceTest utility = new EncodedSeekPerformanceTest(); - utility.runTests(path, encoders); - - System.exit(0); - } - - private static void printUsage() { - System.out.println("Usage: one argument, name of the HFile"); - } - - private void clearBlockCache() { - ((LruBlockCache) cacheConf.getBlockCache()).clearCache(); - } -} diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java index d1ef101d559..f5b4c3ca699 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java @@ -102,8 +102,7 @@ public class TestCompactSelection extends TestCase { MockStoreFile(long length, boolean isRef) throws IOException { super(TEST_UTIL.getTestFileSystem(), TEST_FILE, TEST_UTIL.getConfiguration(), - new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE, - null); + new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE); this.length = length; this.isRef = isRef; } diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 66419dac559..6b2221b1259 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -24,10 +24,8 @@ import static org.mockito.Mockito.spy; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.Map.Entry; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -35,24 +33,20 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestCase; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings.Algorithm; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; +import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -67,7 +61,6 @@ public class TestCompaction extends HBaseTestCase { private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private HRegion r = null; - private HTableDescriptor htd = null; private Path compactionDir = null; private Path regionCompactionDir = null; private static final byte [] COLUMN_FAMILY = fam1; @@ -77,6 +70,7 @@ public class TestCompaction extends HBaseTestCase { private byte[] firstRowBytes, secondRowBytes, thirdRowBytes; final private byte[] col1, col2; + /** constructor */ public TestCompaction() throws Exception { super(); @@ -100,7 +94,7 @@ public class TestCompaction extends HBaseTestCase { @Override public void setUp() throws Exception { super.setUp(); - this.htd = createTableDescriptor(getName()); + HTableDescriptor htd = createTableDescriptor(getName()); this.r = createNewHRegion(htd, null, null); } @@ -152,48 +146,6 @@ public class TestCompaction extends HBaseTestCase { * @throws Exception */ public void testMajorCompaction() throws Exception { - majorCompaction(); - } - - /** - * Test major compaction with block cache. - * @throws Exception - */ - public void testDataBlockEncodingWithNormalSeek() throws Exception { - // block cache only - majorCompactionWithDataBlockEncoding(false); - } - - /** - * Test major compaction with block cache and in memory encoding. - * @throws Exception - */ - public void testDataBlockEncodingWithEncodedSeek() throws Exception { - majorCompactionWithDataBlockEncoding(true); - } - - private void majorCompactionWithDataBlockEncoding(boolean encodedSeek) - throws Exception { - Map replaceBlockCache = - new HashMap(); - for (Entry pair : r.getStores().entrySet()) { - Store store = pair.getValue(); - HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder(); - replaceBlockCache.put(pair.getValue(), blockEncoder); - store.setDataBlockEncoderInTest(new HFileDataBlockEncoderImpl(null, - Algorithm.PREFIX, encodedSeek)); - } - - majorCompaction(); - - // restore settings - for (Entry entry : - replaceBlockCache.entrySet()) { - entry.getKey().setDataBlockEncoderInTest(entry.getValue()); - } - } - - private void majorCompaction() throws Exception { createStoreFile(r); for (int i = 0; i < compactionThreshold; i++) { createStoreFile(r); @@ -222,10 +174,10 @@ public class TestCompaction extends HBaseTestCase { CompactionProgress progress = store.getCompactionProgress(); if( progress != null ) { ++storeCount; - assertTrue(progress.currentCompactedKVs > 0); - assertTrue(progress.totalCompactingKVs > 0); + assert(progress.currentCompactedKVs > 0); + assert(progress.totalCompactingKVs > 0); } - assertTrue(storeCount > 0); + assert(storeCount > 0); } // look at the second row @@ -234,20 +186,14 @@ public class TestCompaction extends HBaseTestCase { secondRowBytes[START_KEY_BYTES.length - 1]++; // Always 3 versions if that is what max versions is. - result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT). - setMaxVersions(100), null); - LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " + - "initial compaction: " + result); - assertEquals("Invalid number of versions of row " - + Bytes.toStringBinary(secondRowBytes) + ".", compactionThreshold, - result.size()); + result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null); + assertEquals(compactionThreshold, result.size()); // Now add deletes to memstore and then flush it. // That will put us over // the compaction threshold of 3 store files. Compacting these store files // should result in a compacted store file that has no references to the // deleted row. - LOG.debug("Adding deletes to memstore and flushing"); Delete delete = new Delete(secondRowBytes, System.currentTimeMillis(), null); byte [][] famAndQf = {COLUMN_FAMILY, null}; delete.deleteFamily(famAndQf[0]); diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 6d4b1d97460..d6570ec4d7f 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.*; 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.NoOpDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType; @@ -195,8 +194,7 @@ public class TestCompoundBloomFilter { private void readStoreFile(int t, BloomType bt, List kvs, Path sfPath) throws IOException { - StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt, - new NoOpDataBlockEncoder()); + StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt); StoreFile.Reader r = sf.createReader(); final boolean pread = true; // does not really matter StoreFileScanner scanner = r.getStoreFileScanner(true, pread); @@ -296,7 +294,7 @@ public class TestCompoundBloomFilter { cacheConf = new CacheConfig(conf); StoreFile.Writer w = StoreFile.createWriter(fs, - TEST_UTIL.getDataTestDir(), BLOCK_SIZES[t], null, null, null, conf, + TEST_UTIL.getDataTestDir(), BLOCK_SIZES[t], null, null, conf, cacheConf, bt, 0); assertTrue(w.hasGeneralBloom()); diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index af85d345bb0..f42f124c055 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -76,8 +76,7 @@ public class TestFSErrorsExposed { writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); StoreFile sf = new StoreFile(fs, writer.getPath(), - util.getConfiguration(), cacheConf, StoreFile.BloomType.NONE, null); - + util.getConfiguration(), cacheConf, StoreFile.BloomType.NONE); StoreFile.Reader reader = sf.createReader(); HFileScanner scanner = reader.getScanner(false, true); @@ -120,8 +119,7 @@ public class TestFSErrorsExposed { writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), - cacheConf, BloomType.NONE, null); - + cacheConf, BloomType.NONE); List scanners = StoreFileScanner.getScannersForStoreFiles( Collections.singletonList(sf), false, true, false); KeyValueScanner scanner = scanners.get(0); diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 00c57c61228..e078630208b 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.TreeSet; import org.apache.commons.logging.Log; @@ -34,23 +35,18 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestCase; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.Reference.Range; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -93,7 +89,7 @@ public class TestStoreFile extends HBaseTestCase { conf, cacheConf); writeStoreFile(writer); checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf, - StoreFile.BloomType.NONE, null)); + StoreFile.BloomType.NONE)); } private void writeStoreFile(final StoreFile.Writer writer) throws IOException { @@ -134,7 +130,7 @@ public class TestStoreFile extends HBaseTestCase { conf, cacheConf); writeStoreFile(writer); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); StoreFile.Reader reader = hsf.createReader(); // 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 @@ -146,7 +142,7 @@ public class TestStoreFile extends HBaseTestCase { // Make a reference Path refPath = StoreFile.split(fs, dir, hsf, midRow, Range.top); StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); // Now confirm that I can read from the reference and that it only gets // keys from top half of the file. HFileScanner s = refHsf.createReader().getScanner(false, false); @@ -182,10 +178,10 @@ public class TestStoreFile extends HBaseTestCase { Path bottomPath = StoreFile.split(this.fs, bottomDir, f, midRow, Range.bottom); // Make readers on top and bottom. - StoreFile.Reader top = new StoreFile(this.fs, topPath, - conf, cacheConf).createReader(); - StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, - conf, cacheConf).createReader(); + StoreFile.Reader top = new StoreFile(this.fs, topPath, conf, cacheConf, + StoreFile.BloomType.NONE).createReader(); + StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, + StoreFile.BloomType.NONE).createReader(); ByteBuffer previous = null; LOG.info("Midkey: " + midKV.toString()); ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey); @@ -241,9 +237,9 @@ public class TestStoreFile extends HBaseTestCase { bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, Range.bottom); top = new StoreFile(this.fs, topPath, conf, cacheConf, - StoreFile.BloomType.NONE, null).createReader(); + StoreFile.BloomType.NONE).createReader(); bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, - StoreFile.BloomType.NONE, null).createReader(); + StoreFile.BloomType.NONE).createReader(); bottomScanner = bottom.getScanner(false, false); int count = 0; while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || @@ -286,9 +282,9 @@ public class TestStoreFile extends HBaseTestCase { bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey, Range.bottom); top = new StoreFile(this.fs, topPath, conf, cacheConf, - StoreFile.BloomType.NONE, null).createReader(); + StoreFile.BloomType.NONE).createReader(); bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf, - StoreFile.BloomType.NONE, null).createReader(); + StoreFile.BloomType.NONE).createReader(); first = true; bottomScanner = bottom.getScanner(false, false); while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || @@ -344,7 +340,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, null); + StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf); reader.loadFileInfo(); reader.loadBloomfilter(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); @@ -383,10 +379,10 @@ public class TestStoreFile extends HBaseTestCase { // write the file Path f = new Path(ROOT_DIR, getName()); - StoreFile.Writer writer = - new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, - HFile.DEFAULT_COMPRESSION_ALGORITHM, null, conf, cacheConf, - KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000); + StoreFile.Writer writer = new StoreFile.Writer(fs, f, + StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, + conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000); + bloomWriteRead(writer, fs); } @@ -403,8 +399,7 @@ public class TestStoreFile extends HBaseTestCase { StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, - 2000); + conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000); // add delete family long now = System.currentTimeMillis(); @@ -416,7 +411,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, null); + StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf); reader.loadFileInfo(); reader.loadBloomfilter(); @@ -471,7 +466,7 @@ public class TestStoreFile extends HBaseTestCase { StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - null, conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]); + conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]); long now = System.currentTimeMillis(); for (int i = 0; i < rowCount*2; i += 2) { // rows @@ -488,7 +483,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, null); + StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf); reader.loadFileInfo(); reader.loadBloomfilter(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); @@ -541,8 +536,7 @@ public class TestStoreFile extends HBaseTestCase { // this should not create a bloom because the max keys is too small StoreFile.Writer writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, - 2000); + conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000); assertFalse(writer.hasGeneralBloom()); writer.close(); fs.delete(f, true); @@ -565,7 +559,7 @@ public class TestStoreFile extends HBaseTestCase { // because Java can't create a contiguous array > MAX_INT writer = new StoreFile.Writer(fs, f, StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM, - null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, + conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE); assertFalse(writer.hasGeneralBloom()); writer.close(); @@ -670,7 +664,7 @@ public class TestStoreFile extends HBaseTestCase { writer.close(); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); StoreFile.Reader reader = hsf.createReader(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); TreeSet columns = new TreeSet(); @@ -714,7 +708,7 @@ public class TestStoreFile extends HBaseTestCase { Path pathCowOff = new Path(baseDir, "123456789"); StoreFile.Writer writer = writeStoreFile(conf, cacheConf, pathCowOff, 3); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); LOG.debug(hsf.getPath().toString()); // Read this file, we should see 3 misses @@ -736,7 +730,7 @@ public class TestStoreFile extends HBaseTestCase { Path pathCowOn = new Path(baseDir, "123456788"); writer = writeStoreFile(conf, cacheConf, pathCowOn, 3); hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); // Read this file, we should see 3 hits reader = hsf.createReader(); @@ -752,13 +746,13 @@ public class TestStoreFile extends HBaseTestCase { // Let's read back the two files to ensure the blocks exactly match hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); StoreFile.Reader readerOne = hsf.createReader(); readerOne.loadFileInfo(); StoreFileScanner scannerOne = readerOne.getStoreFileScanner(true, true); scannerOne.seek(KeyValue.LOWESTKEY); hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); StoreFile.Reader readerTwo = hsf.createReader(); readerTwo.loadFileInfo(); StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true); @@ -789,7 +783,7 @@ public class TestStoreFile extends HBaseTestCase { conf.setBoolean("hbase.rs.evictblocksonclose", true); cacheConf = new CacheConfig(conf); hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); reader = hsf.createReader(); reader.close(cacheConf.shouldEvictOnClose()); @@ -803,7 +797,7 @@ public class TestStoreFile extends HBaseTestCase { conf.setBoolean("hbase.rs.evictblocksonclose", false); cacheConf = new CacheConfig(conf); hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, - StoreFile.BloomType.NONE, null); + StoreFile.BloomType.NONE); reader = hsf.createReader(); reader.close(cacheConf.shouldEvictOnClose()); @@ -830,8 +824,7 @@ public class TestStoreFile extends HBaseTestCase { int blockSize = totalSize / numBlocks; StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize, HFile.DEFAULT_COMPRESSION_ALGORITHM, - null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, - 2000); + conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000); // We'll write N-1 KVs to ensure we don't write an extra block kvs.remove(kvs.size()-1); for (KeyValue kv : kvs) { @@ -842,43 +835,6 @@ public class TestStoreFile extends HBaseTestCase { return writer; } - /** - * Check if data block encoding information is saved correctly in HFile's - * file info. - */ - public void testDataBlockEncodingMetaData() throws IOException { - Path dir = new Path(new Path(this.testDir, "regionname"), "familyname"); - Path path = new Path(dir, "1234567890"); - - DataBlockEncodings.Algorithm dataBlockEncoderAlgo = - DataBlockEncodings.Algorithm.FAST_DIFF; - HFileDataBlockEncoder dataBlockEncoder = - new HFileDataBlockEncoderImpl( - dataBlockEncoderAlgo, - DataBlockEncodings.Algorithm.NONE, - false); - cacheConf = new CacheConfig(conf); - StoreFile.Writer writer = new StoreFile.Writer(fs, - path, HFile.DEFAULT_BLOCKSIZE, - HFile.DEFAULT_COMPRESSION_ALGORITHM, - dataBlockEncoder, - conf, - cacheConf, - KeyValue.COMPARATOR, - StoreFile.BloomType.NONE, - 2000); - writer.close(); - - StoreFile storeFile = new StoreFile(fs, writer.getPath(), conf, - cacheConf, BloomType.NONE, dataBlockEncoder); - StoreFile.Reader reader = storeFile.createReader(); - - Map fileInfo = reader.loadFileInfo(); - byte[] value = fileInfo.get(StoreFile.DATA_BLOCK_ENCODING); - - assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value); - } - @org.junit.Rule public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); diff --git a/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java index 29bf1d67089..2e175da70bd 100644 --- a/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java +++ b/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.PerformanceEvaluation; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncodings; import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -73,12 +72,6 @@ public class LoadTestTool extends AbstractHBaseTool { private static final String OPT_BLOOM = "bloom"; private static final String OPT_COMPRESSION = "compression"; - private static final String OPT_DATA_BLOCK_ENCODING = "data_block_encoding"; - private static final String OPT_DATA_BLOCK_ENCODING_CACHE_ONLY = - "data_block_encoding_cache_only"; - private static final String OPT_ENCODED_DATA_BLOCK_SEEK = - "encoded_data_block_seek"; - private static final String OPT_KEY_WINDOW = "key_window"; private static final String OPT_WRITE = "write"; private static final String OPT_MAX_READ_ERRORS = "max_read_errors"; @@ -89,8 +82,6 @@ public class LoadTestTool extends AbstractHBaseTool { private static final String OPT_TABLE_NAME = "tn"; private static final String OPT_ZK_QUORUM = "zk"; - private static final long DEFAULT_START_KEY = 0; - /** This will be removed as we factor out the dependency on command line */ private CommandLine cmd; @@ -117,7 +108,7 @@ public class LoadTestTool extends AbstractHBaseTool { public void createTables() throws IOException { HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableName, COLUMN_FAMILY); - applyColumnFamilyOptions(tableName, COLUMN_FAMILIES); + applyBloomFilterAndCompression(tableName, COLUMN_FAMILIES); } private String[] splitColonSeparated(String option, @@ -138,10 +129,9 @@ public class LoadTestTool extends AbstractHBaseTool { } /** - * Apply column family options such as Bloom filters, compression, and data - * block encoding. + * Apply the given Bloom filter type to all column families we care about. */ - private void applyColumnFamilyOptions(byte[] tableName, + private void applyBloomFilterAndCompression(byte[] tableName, byte[][] columnFamilies) throws IOException { String bloomStr = cmd.getOptionValue(OPT_BLOOM); StoreFile.BloomType bloomType = bloomStr == null ? null : @@ -151,16 +141,8 @@ public class LoadTestTool extends AbstractHBaseTool { Compression.Algorithm compressAlgo = compressStr == null ? null : Compression.Algorithm.valueOf(compressStr); - String dataBlockEncodingStr = cmd.getOptionValue(OPT_DATA_BLOCK_ENCODING); - DataBlockEncodings.Algorithm dataBlockEncodingAlgo = - dataBlockEncodingStr == null ? null : - DataBlockEncodings.Algorithm.valueOf(dataBlockEncodingStr); - - if (bloomStr == null && compressStr == null - && dataBlockEncodingStr == null) { - // No reason to disable/enable the table. + if (bloomStr == null && compressStr == null) return; - } HBaseAdmin admin = new HBaseAdmin(conf); HTableDescriptor tableDesc = admin.getTableDescriptor(tableName); @@ -168,21 +150,10 @@ public class LoadTestTool extends AbstractHBaseTool { admin.disableTable(tableName); for (byte[] cf : columnFamilies) { HColumnDescriptor columnDesc = tableDesc.getFamily(cf); - if (bloomStr != null) { + if (bloomStr != null) columnDesc.setBloomFilterType(bloomType); - } - if (compressStr != null) { + if (compressStr != null) columnDesc.setCompressionType(compressAlgo); - } - if (dataBlockEncodingAlgo != null) { - columnDesc.setDataBlockEncodingOnDisk( - cmd.hasOption(OPT_DATA_BLOCK_ENCODING_CACHE_ONLY) ? - DataBlockEncodings.Algorithm.NONE : - dataBlockEncodingAlgo); - columnDesc.setDataBlockEncodingInCache(dataBlockEncodingAlgo); - columnDesc.setEncodedDataBlockSeek( - cmd.hasOption(OPT_ENCODED_DATA_BLOCK_SEEK)); - } admin.modifyColumn(tableName, columnDesc); } LOG.info("Enabling table " + Bytes.toString(tableName)); @@ -198,29 +169,17 @@ public class LoadTestTool extends AbstractHBaseTool { addOptWithArg(OPT_READ, OPT_USAGE_READ); addOptWithArg(OPT_BLOOM, OPT_USAGE_BLOOM); addOptWithArg(OPT_COMPRESSION, OPT_USAGE_COMPRESSION); - addOptWithArg(OPT_DATA_BLOCK_ENCODING, "Encoding algorithm (e.g. prefix " - + "compression) to use for data blocks in the test column family, " - + "one of " + - Arrays.toString(DataBlockEncodings.Algorithm.values()) + "."); addOptWithArg(OPT_MAX_READ_ERRORS, "The maximum number of read errors " + "to tolerate before terminating all reader threads. The default is " + MultiThreadedReader.DEFAULT_MAX_ERRORS + "."); addOptWithArg(OPT_KEY_WINDOW, "The 'key window' to maintain between " + "reads and writes for concurrent write/read workload. The default " + "is " + MultiThreadedReader.DEFAULT_KEY_WINDOW + "."); - addOptNoArg(OPT_MULTIPUT, "Whether to use multi-puts as opposed to " + "separate puts for every column in a row"); - addOptNoArg(OPT_DATA_BLOCK_ENCODING_CACHE_ONLY, "If using a data block " + - "encoding, this flag will only enable encoding in cache but not on " + - "disk."); - addOptNoArg(OPT_ENCODED_DATA_BLOCK_SEEK, "If using a data block " + - "encoding, this will enable doing seek operations on encoded blocks."); addRequiredOptWithArg(OPT_NUM_KEYS, "The number of keys to read/write"); - addOptWithArg(OPT_START_KEY, "The first key to read/write " + - "(a 0-based index). The default value is " + - DEFAULT_START_KEY + "."); + addRequiredOptWithArg(OPT_START_KEY, "The first key to read/write"); } @Override @@ -229,8 +188,8 @@ public class LoadTestTool extends AbstractHBaseTool { tableName = Bytes.toBytes(cmd.getOptionValue(OPT_TABLE_NAME, DEFAULT_TABLE_NAME)); - startKey = parseLong(cmd.getOptionValue(OPT_START_KEY, - String.valueOf(DEFAULT_START_KEY)), 0, Long.MAX_VALUE); + startKey = parseLong(cmd.getOptionValue(OPT_START_KEY), 0, + Long.MAX_VALUE); long numKeys = parseLong(cmd.getOptionValue(OPT_NUM_KEYS), 1, Long.MAX_VALUE - startKey); endKey = startKey + numKeys; @@ -289,7 +248,7 @@ public class LoadTestTool extends AbstractHBaseTool { System.out.println("Reader threads: " + numReaderThreads); } - System.out.println("Key range: [" + startKey + ".." + (endKey - 1) + "]"); + System.out.println("Key range: " + startKey + ".." + (endKey - 1)); } @Override diff --git a/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java deleted file mode 100644 index 40a59482407..00000000000 --- a/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java +++ /dev/null @@ -1,356 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.apache.hadoop.hbase.util; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collection; -import java.util.Collections; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import org.apache.hadoop.hbase.SmallTests; -import org.apache.hadoop.io.WritableUtils; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(SmallTests.class) -public class TestByteBufferUtils { - - private byte[] array; - - /** - * Create an array with sample data. - */ - @Before - public void setUp() { - array = new byte[8]; - for (int i = 0 ; i < array.length ; ++i) { - array[i] = (byte) ('a' + i); - } - } - - private static final int MAX_VLONG_LENGTH = 9; - private static final Collection testNumbers; - - private static void addNumber(Set a, long l) { - if (l != Long.MIN_VALUE) { - a.add(l - 1); - } - a.add(l); - if (l != Long.MAX_VALUE) { - a.add(l + 1); - } - for (long divisor = 3; divisor <= 10; ++divisor) { - for (long delta = -1; delta <= 1; ++delta) { - a.add(l / divisor + delta); - } - } - } - - static { - SortedSet a = new TreeSet(); - for (int i = 0; i <= 63; ++i) { - long v = (-1L) << i; - assertTrue(v < 0); - addNumber(a, v); - v = (1L << i) - 1; - assertTrue(v >= 0); - addNumber(a, v); - } - - testNumbers = Collections.unmodifiableSet(a); - System.err.println("Testing variable-length long serialization using: " - + testNumbers + " (count: " + testNumbers.size() + ")"); - assertEquals(1753, testNumbers.size()); - assertEquals(Long.MIN_VALUE, a.first().longValue()); - assertEquals(Long.MAX_VALUE, a.last().longValue()); - } - - @Test - public void testReadWriteVLong() { - for (long l : testNumbers) { - ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH); - ByteBufferUtils.writeVLong(b, l); - b.flip(); - assertEquals(l, ByteBufferUtils.readVLong(b)); - } - } - - @Test - public void testConsistencyWithHadoopVLong() throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dos = new DataOutputStream(baos); - for (long l : testNumbers) { - baos.reset(); - ByteBuffer b = ByteBuffer.allocate(MAX_VLONG_LENGTH); - ByteBufferUtils.writeVLong(b, l); - String bufStr = Bytes.toStringBinary(b.array(), - b.arrayOffset(), b.position()); - WritableUtils.writeVLong(dos, l); - String baosStr = Bytes.toStringBinary(baos.toByteArray()); - assertEquals(baosStr, bufStr); - } - } - - /** - * Test copying to stream from buffer. - */ - @Test - public void testCopyToStream() { - ByteBuffer buffer = ByteBuffer.wrap(array); - - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - - try { - ByteBufferUtils.copyToStream(bos, buffer, array.length); - } catch (IOException e) { - fail("IOException in testCopyToStream()"); - } - assertArrayEquals(bos.toByteArray(), array); - } - - /** - * Test copying to stream one byte. - * @throws IOException On test failure. - */ - @Test - public void testCopyToStreamByte() throws IOException { - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - - for (int i = 0 ; i < array.length ; ++i) { - ByteBufferUtils.copyToStream(bos, array[i]); - } - byte[] actual = bos.toByteArray(); - for (int i = 0 ; i < array.length ; ++i) { - assertEquals(array[i], actual[i]); - } - } - - /** - * Test copying to stream from buffer with offset. - * @throws IOException On test failure. - */ - @Test - public void testCopyToStreamWithOffset() throws IOException { - ByteBuffer buffer = ByteBuffer.wrap(array); - - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - - ByteBufferUtils.copyToStream(bos, buffer, array.length / 2, - array.length / 2); - - byte[] returnedArray = bos.toByteArray(); - for (int i = 0 ; i < array.length / 2 ; ++i) { - int pos = array.length / 2 + i; - assertEquals(returnedArray[i], array[pos]); - } - } - - /** - * Test copying data from stream. - * @throws IOException On test failure. - */ - @Test - public void testCopyFromStream() throws IOException { - ByteBuffer buffer = ByteBuffer.allocate(array.length); - ByteArrayInputStream bis = new ByteArrayInputStream(array); - DataInputStream dis = new DataInputStream(bis); - - ByteBufferUtils.copyFromStream(dis, buffer, array.length / 2); - ByteBufferUtils.copyFromStream(dis, buffer, - array.length - array.length / 2); - for (int i = 0 ; i < array.length ; ++i) { - assertEquals(array[i], buffer.get(i)); - } - } - - /** - * Test copying from buffer. - */ - @Test - public void testCopyFromBuffer() { - ByteBuffer srcBuffer = ByteBuffer.allocate(array.length); - ByteBuffer dstBuffer = ByteBuffer.allocate(array.length); - srcBuffer.put(array); - - ByteBufferUtils.copyFromBuffer(srcBuffer, dstBuffer, - array.length / 2, array.length / 4); - for (int i = 0 ; i < array.length / 4 ; ++i) { - assertEquals(srcBuffer.get(i + array.length / 2), - dstBuffer.get(i)); - } - } - - /** - * Test 7-bit encoding of integers. - * @throws IOException On test failure. - */ - @Test - public void testCompressedInt() throws IOException { - testCompressedInt(0); - testCompressedInt(Integer.MAX_VALUE); - testCompressedInt(Integer.MIN_VALUE); - - for (int i = 0 ; i < 3 ; i++) { - testCompressedInt((128 << i) - 1); - } - - for (int i = 0 ; i < 3 ; i++) { - testCompressedInt((128 << i)); - } - } - - /** - * Test fitting integer in less bytes. - */ - @Test - public void testFixedInt() { - testFixedInt(0, 1); - testFixedInt(Integer.MAX_VALUE, 4); - testFixedInt(Integer.MIN_VALUE, 4); - - for (int i = 0 ; i < 3 ; i++) { - testFixedInt((128 << i) - 1, i + 1); - } - - for (int i = 0 ; i < 3 ; i++) { - testFixedInt((128 << i), 2 + i); - } - } - - /** - * Test how much bytes we need to store integer. - */ - @Test - public void testIntFitsIn() { - assertEquals(1, ByteBufferUtils.intFitsIn(0)); - assertEquals(1, ByteBufferUtils.intFitsIn(1)); - assertEquals(2, ByteBufferUtils.intFitsIn(1 << 8)); - assertEquals(3, ByteBufferUtils.intFitsIn(1 << 16)); - assertEquals(4, ByteBufferUtils.intFitsIn(-1)); - assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MAX_VALUE)); - assertEquals(4, ByteBufferUtils.intFitsIn(Integer.MIN_VALUE)); - } - - /** - * Test how much bytes we need to store long. - */ - @Test - public void testLongFitsIn() { - assertEquals(1, ByteBufferUtils.longFitsIn(0)); - assertEquals(1, ByteBufferUtils.longFitsIn(1)); - assertEquals(3, ByteBufferUtils.longFitsIn(1l << 16)); - assertEquals(5, ByteBufferUtils.longFitsIn(1l << 32)); - assertEquals(8, ByteBufferUtils.longFitsIn(-1)); - assertEquals(8, ByteBufferUtils.longFitsIn(Long.MIN_VALUE)); - assertEquals(8, ByteBufferUtils.longFitsIn(Long.MAX_VALUE)); - } - - /** - * Test if we are comparing equal bytes. - */ - @Test - public void testArePartEqual() { - byte[] array = new byte[] { 1, 2, 3, 4, 5, 1, 2, 3, 4 }; - ByteBuffer buffer = ByteBuffer.wrap(array); - assertTrue(ByteBufferUtils.arePartsEqual(buffer, 0, 4, 5, 4)); - assertTrue(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 2)); - assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 2, 6, 3)); - assertFalse(ByteBufferUtils.arePartsEqual(buffer, 1, 3, 6, 2)); - assertFalse(ByteBufferUtils.arePartsEqual(buffer, 0, 3, 6, 3)); - } - - /** - * Test serializing int to bytes - */ - @Test - public void testPutInt() { - testPutInt(0); - testPutInt(Integer.MAX_VALUE); - - for (int i = 0 ; i < 3 ; i++) { - testPutInt((128 << i) - 1); - } - - for (int i = 0 ; i < 3 ; i++) { - testPutInt((128 << i)); - } - } - - // Utility methods invoked from test methods - - private void testCompressedInt(int value) throws IOException { - int parsedValue = 0; - - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ByteBufferUtils.putCompressedInt(bos, value); - - ByteArrayInputStream bis = new ByteArrayInputStream( - bos.toByteArray()); - parsedValue = ByteBufferUtils.readCompressedInt(bis); - - assertEquals(value, parsedValue); - } - - private void testFixedInt(int value, int width) { - int parsedValue = 0; - byte[] bValue = new byte[Bytes.SIZEOF_INT]; - Bytes.putInt(bValue, 0, value); - - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - bos.write(bValue, Bytes.SIZEOF_INT - width, width); - - ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - try { - parsedValue = ByteBufferUtils.readCompressedInt(bis, width); - } catch (IOException e) { - fail("While reading fixed int"); - } - - assertEquals(value, parsedValue); - } - - private void testPutInt(int value) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try { - ByteBufferUtils.putInt(baos, value); - } catch (IOException e) { - throw new RuntimeException("Bug in putIn()", e); - } - - ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - DataInputStream dis = new DataInputStream(bais); - try { - assertEquals(dis.readInt(), value); - } catch (IOException e) { - throw new RuntimeException("Bug in test!", e); - } - } -}