[jira] [HBASE-4218] HFile data block encoding framework and delta encoding

implementation (Jacek Midgal, Mikhail Bautin)

Summary:

Adding a framework that allows to "encode" keys in an HFile data block. We
support two modes of encoding: (1) both on disk and in cache, and (2) in cache
only. This is distinct from compression that is already being done in HBase,
e.g. GZ or LZO. When data block encoding is enabled, we store blocks in cache
in an uncompressed but encoded form. This allows to fit more blocks in cache
and reduce the number of disk reads.

The most common example of data block encoding is delta encoding, where we take
advantage of the fact that HFile keys are sorted and share a lot of common
prefixes, and only store the delta between each pair of consecutive keys.
Initial encoding algorithms implemented are DIFF, FAST_DIFF, and PREFIX.

This is based on the delta encoding patch developed by Jacek Midgal during his
2011 summer internship at Facebook. The original patch is available here:
https://reviews.apache.org/r/2308/diff/.

Test Plan: Unit tests. Distributed load test on a five-node cluster.

Reviewers: JIRA, tedyu, stack, nspiegelberg, Kannan

Reviewed By: Kannan

CC: tedyu, todd, mbautin, stack, Kannan, mcorgan, gqchen

Differential Revision: https://reviews.facebook.net/D447



git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1236031 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
mbautin 2012-01-26 02:58:57 +00:00
parent 0d0b5b4dcc
commit ec1c804fc1
93 changed files with 8817 additions and 1113 deletions

View File

@ -27,6 +27,7 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -53,10 +54,16 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
// 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
private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)8;
// Version 9 -- add data block encoding
private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 9;
// 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 ENCODE_ON_DISK =
"ENCODE_ON_DISK";
public static final String DATA_BLOCK_ENCODING =
"DATA_BLOCK_ENCODING";
public static final String BLOCKCACHE = "BLOCKCACHE";
/**
@ -79,6 +86,17 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
*/
public static final String DEFAULT_COMPRESSION =
Compression.Algorithm.NONE.getName();
/**
* Default value of the flag that enables data block encoding on disk, as
* opposed to encoding in cache only. We encode blocks everywhere by default,
* as long as {@link #DATA_BLOCK_ENCODING} is not NONE.
*/
public static final boolean DEFAULT_ENCODE_ON_DISK = true;
/** Default data block encoding algorithm. */
public static final String DEFAULT_DATA_BLOCK_ENCODING =
DataBlockEncoding.NONE.toString();
/**
* Default number of versions of a record to keep.
@ -143,6 +161,10 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
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(ENCODE_ON_DISK,
String.valueOf(DEFAULT_ENCODE_ON_DISK));
DEFAULT_VALUES.put(DATA_BLOCK_ENCODING,
String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
}
// Column family name
@ -241,7 +263,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
* cache
* @param blockCacheEnabled If true, MapFile blocks should be cached
* @param blocksize Block size to use when writing out storefiles. Use
* smaller blocksizes for faster random-access at expense of larger indices
* smaller block sizes for faster random-access at expense of larger indices
* (more memory consumption). Default is usually 64k.
* @param timeToLive Time-to-live of cell contents, in seconds
* (use HConstants.FOREVER for unlimited TTL)
@ -258,8 +280,9 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
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, inMemory, blockCacheEnabled, blocksize, timeToLive,
bloomFilter, scope);
compression, DEFAULT_ENCODE_ON_DISK, DEFAULT_DATA_BLOCK_ENCODING,
inMemory, blockCacheEnabled, blocksize, timeToLive, bloomFilter,
scope);
}
/**
@ -271,6 +294,9 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
* @param keepDeletedCells Whether to retain deleted cells until they expire
* up to maxVersions versions.
* @param compression Compression type
* @param encodeOnDisk whether to use the specified data block encoding
* on disk. If false, the encoding will be used in cache only.
* @param dataBlockEncoding data block encoding
* @param inMemory If true, column data should be kept in an HRegionServer's
* cache
* @param blockCacheEnabled If true, MapFile blocks should be cached
@ -289,7 +315,8 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
*/
public HColumnDescriptor(final byte[] familyName, final int minVersions,
final int maxVersions, final boolean keepDeletedCells,
final String compression, final boolean inMemory,
final String compression, final boolean encodeOnDisk,
final String dataBlockEncoding, final boolean inMemory,
final boolean blockCacheEnabled, final int blocksize,
final int timeToLive, final String bloomFilter, final int scope) {
isLegalFamilyName(familyName);
@ -319,6 +346,9 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
setTimeToLive(timeToLive);
setCompressionType(Compression.Algorithm.
valueOf(compression.toUpperCase()));
setEncodeOnDisk(encodeOnDisk);
setDataBlockEncoding(DataBlockEncoding.
valueOf(dataBlockEncoding.toUpperCase()));
setBloomFilterType(StoreFile.BloomType.
valueOf(bloomFilter.toUpperCase()));
setBlocksize(blocksize);
@ -496,6 +526,57 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
setValue(COMPRESSION, compressionType);
}
/** @return data block encoding algorithm used on disk */
public DataBlockEncoding getDataBlockEncodingOnDisk() {
String encodeOnDiskStr = getValue(ENCODE_ON_DISK);
boolean encodeOnDisk;
if (encodeOnDiskStr == null) {
encodeOnDisk = DEFAULT_ENCODE_ON_DISK;
} else {
encodeOnDisk = Boolean.valueOf(encodeOnDiskStr);
}
if (!encodeOnDisk) {
// No encoding on disk.
return DataBlockEncoding.NONE;
}
return getDataBlockEncoding();
}
/**
* Set the flag indicating that we only want to encode data block in cache
* but not on disk.
*/
public void setEncodeOnDisk(boolean encodeOnDisk) {
setValue(ENCODE_ON_DISK, String.valueOf(encodeOnDisk));
}
/**
* @return the data block encoding algorithm used in block cache and
* optionally on disk
*/
public DataBlockEncoding getDataBlockEncoding() {
String type = getValue(DATA_BLOCK_ENCODING);
if (type == null) {
type = DEFAULT_DATA_BLOCK_ENCODING;
}
return DataBlockEncoding.valueOf(type);
}
/**
* Set data block encoding algorithm used in block cache.
* @param type What kind of data block encoding will be used.
*/
public void setDataBlockEncoding(DataBlockEncoding type) {
String name;
if (type != null) {
name = type.toString();
} else {
name = DataBlockEncoding.NONE.toString();
}
setValue(DATA_BLOCK_ENCODING, name);
}
/**
* @return Compression type setting.
*/

View File

@ -237,6 +237,10 @@ public final class HConstants {
/** Default maximum file size */
public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
/** Conf key for the memstore size at which we flush the memstore */
public static final String HREGION_MEMSTORE_FLUSH_SIZE =
"hbase.hregion.memstore.flush.size";
/** Default size of a reservation block */
public static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5;

View File

@ -130,16 +130,27 @@ 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 =
Bytes.SIZEOF_LONG /* timestamp */ +
Bytes.SIZEOF_BYTE /*keytype*/;
public static final int TIMESTAMP_TYPE_SIZE = TIMESTAMP_SIZE + TYPE_SIZE;
// Size of the length shorts and bytes in key.
public static final int KEY_INFRASTRUCTURE_SIZE =
Bytes.SIZEOF_SHORT /*rowlength*/ +
Bytes.SIZEOF_BYTE /*columnfamilylength*/ +
TIMESTAMP_TYPE_SIZE;
public static final int KEY_INFRASTRUCTURE_SIZE = ROW_LENGTH_SIZE
+ FAMILY_LENGTH_SIZE + 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.
@ -701,10 +712,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<String,?> containing data from this key
*/
public Map<String, Object> toStringMap() {
@ -730,13 +741,21 @@ public class KeyValue implements Writable, HeapSize {
Bytes.toStringBinary(b, columnoffset + familylength,
columnlength - familylength);
long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE));
String timestampStr = humanReadableTimestamp(timestamp);
byte type = b[o + l - 1];
// return row + "/" + family +
// (family != null && family.length() > 0? COLUMN_FAMILY_DELIMITER: "") +
// qualifier + "/" + timestamp + "/" + Type.codeToType(type);
return row + "/" + family +
(family != null && family.length() > 0? ":" :"") +
qualifier + "/" + timestamp + "/" + Type.codeToType(type);
qualifier + "/" + timestampStr + "/" + Type.codeToType(type);
}
public static String humanReadableTimestamp(final long timestamp) {
if (timestamp == HConstants.LATEST_TIMESTAMP) {
return "LATEST_TIMESTAMP";
}
if (timestamp == HConstants.OLDEST_TIMESTAMP) {
return "OLDEST_TIMESTAMP";
}
return String.valueOf(timestamp);
}
//---------------------------------------------------------------------------
@ -780,7 +799,7 @@ public class KeyValue implements Writable, HeapSize {
* @return length of entire KeyValue, in bytes
*/
private static int getLength(byte [] bytes, int offset) {
return (2 * Bytes.SIZEOF_INT) +
return ROW_OFFSET +
Bytes.toInt(bytes, offset) +
Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
}
@ -1321,8 +1340,8 @@ public class KeyValue implements Writable, HeapSize {
// KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
// Rebuild as: <keylen:4><0:4><key:keylen>
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,
byte [] newBuffer = new byte[getKeyLength() + ROW_OFFSET + dataLen];
System.arraycopy(this.bytes, this.offset, newBuffer, 0,
Math.min(newBuffer.length,this.length));
Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
if (lenAsVal) {
@ -1393,7 +1412,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,
@ -1561,7 +1580,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();
@ -1595,7 +1614,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;
@ -1879,9 +1898,9 @@ public class KeyValue implements Writable, HeapSize {
*/
public static KeyValue createKeyValueFromKey(final byte [] b, final int o,
final int l) {
byte [] newb = new byte[b.length + ROW_OFFSET];
byte [] newb = new byte[l + ROW_OFFSET];
System.arraycopy(b, o, newb, ROW_OFFSET, l);
Bytes.putInt(newb, 0, b.length);
Bytes.putInt(newb, 0, l);
Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
return new KeyValue(newb);
}
@ -2003,10 +2022,24 @@ public class KeyValue implements Writable, HeapSize {
}
}
/**
* Avoids redundant comparisons for better performance.
*/
public static interface SamePrefixComparator<T> {
/**
* 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<byte []> {
public static class KeyComparator
implements RawComparator<byte []>, SamePrefixComparator<byte[]> {
volatile boolean ignoreTimestamp = false;
volatile boolean ignoreType = false;
@ -2016,45 +2049,123 @@ 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 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);
int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE -
(rcolumnoffset - roffset);
// 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);
}
// if row matches, and no column in the 'left' AND put type is 'minimum',
/**
* 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;
int lcolumnlength = llength - TIMESTAMP_TYPE_SIZE -
(lcolumnoffset - loffset);
int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE -
(rcolumnoffset - roffset);
// 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()) {
return 1; // left is bigger.
// left is "bigger", i.e. it appears later in the sorted order
return 1;
}
if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
return -1;
}
// TODO the family and qualifier should be compared separately
compare = Bytes.compareTo(left, lcolumnoffset, lcolumnlength, right,
rcolumnoffset, rcolumnlength);
if (compare != 0) {
return compare;
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));
}
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,
@ -2069,7 +2180,9 @@ 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
// of higher numbers sort before those of lesser numbers. Maximum (255)
// appears ahead of everything, and minimum (0) appears after
// everything.
return (0xff & rtype) - (0xff & ltype);
}
return 0;

View File

@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -60,9 +61,9 @@ public class HalfStoreFileReader extends StoreFile.Reader {
* @throws IOException
*/
public HalfStoreFileReader(final FileSystem fs, final Path p,
final CacheConfig cacheConf, final Reference r)
throws IOException {
super(fs, p, cacheConf);
final CacheConfig cacheConf, final Reference r,
DataBlockEncoding preferredEncodingInCache) throws IOException {
super(fs, p, cacheConf, preferredEncodingInCache);
// 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

View File

@ -0,0 +1,302 @@
/*
* 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.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;
}
}
/**
* Copy the state from the next one into this instance (the previous state
* placeholder). Used to save the previous state when we are advancing the
* seeker to the next key/value.
*/
protected void copyFromNext(SeekerState nextState) {
if (keyBuffer.length != nextState.keyBuffer.length) {
keyBuffer = nextState.keyBuffer.clone();
} else if (!isValid()) {
// Note: we can only call isValid before we override our state, so this
// comes before all the assignments at the end of this method.
System.arraycopy(nextState.keyBuffer, 0, keyBuffer, 0,
nextState.keyLength);
} else {
// don't copy the common prefix between this key and the previous one
System.arraycopy(nextState.keyBuffer, nextState.lastCommonPrefix,
keyBuffer, nextState.lastCommonPrefix, nextState.keyLength
- nextState.lastCommonPrefix);
}
valueOffset = nextState.valueOffset;
keyLength = nextState.keyLength;
valueLength = nextState.valueLength;
lastCommonPrefix = nextState.lastCommonPrefix;
nextKvOffset = nextState.nextKvOffset;
memstoreTS = nextState.memstoreTS;
}
}
protected abstract static class
BufferedEncodedSeeker<STATE extends SeekerState>
implements EncodedSeeker {
protected final RawComparator<byte[]> comparator;
protected final SamePrefixComparator<byte[]> samePrefixComparator;
protected ByteBuffer currentBuffer;
protected STATE current = createSeekerState(); // always valid
protected STATE previous = createSeekerState(); // may not be valid
@SuppressWarnings("unchecked")
public BufferedEncodedSeeker(RawComparator<byte[]> comparator) {
this.comparator = comparator;
if (comparator instanceof SamePrefixComparator) {
this.samePrefixComparator = (SamePrefixComparator<byte[]>) comparator;
} else {
this.samePrefixComparator = null;
}
}
@Override
public void setCurrentBuffer(ByteBuffer buffer) {
currentBuffer = buffer;
decodeFirst();
previous.invalidate();
}
@Override
public ByteBuffer getKeyDeepCopy() {
ByteBuffer keyBuffer = ByteBuffer.allocate(current.keyLength);
keyBuffer.put(current.keyBuffer, 0, current.keyLength);
return keyBuffer;
}
@Override
public ByteBuffer getValueShallowCopy() {
return ByteBuffer.wrap(currentBuffer.array(),
currentBuffer.arrayOffset() + current.valueOffset,
current.valueLength);
}
@Override
public ByteBuffer getKeyValueBuffer() {
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 getKeyValue() {
ByteBuffer kvBuf = getKeyValueBuffer();
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 seekToKeyInBlock(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) {
if (!previous.isValid()) {
// The caller (seekBefore) has to ensure that we are not at the
// first key in the block.
throw new IllegalStateException("Cannot seekBefore if " +
"positioned at the first key in the block: key=" +
Bytes.toStringBinary(key, offset, length));
}
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()) {
previous.copyFromNext(current);
decodeNext();
} else {
break;
}
} while (true);
// we hit the end of the block, not an 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.");
}
STATE tmp = previous;
previous = current;
current = tmp;
// move after last key value
currentBuffer.position(current.nextKvOffset);
previous.invalidate();
}
@SuppressWarnings("unchecked")
protected STATE createSeekerState() {
// This will fail for non-default seeker state if the subclass does not
// override this method.
return (STATE) new SeekerState();
}
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");
}
}
}
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");
}
}
}
}

View File

@ -0,0 +1,116 @@
/*
* 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;
private 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 to avoid moving the position.
* <p>
* This method overrides all the fields of this instance, except
* {@link #prevOffset}, which is usually manipulated directly by encoders
* and decoders.
* @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;
}
}

View File

@ -0,0 +1,95 @@
/*
* 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.moveBufferToStream(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.copyFromStreamToBuffer(buffer, source, 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<byte[]> comparator,
final boolean includesMemstoreTS) {
return new BufferedEncodedSeeker<SeekerState>(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();
}
};
}
}

View File

@ -0,0 +1,160 @@
/*
* 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:
* <ul>
* <li>the KeyValues are stored sorted by key</li>
* <li>we know the structure of KeyValue</li>
* <li>the values are always iterated forward from beginning of block</li>
* <li>knowledge of Key Value format</li>
* </ul>
* 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. Typically does not make
* a deep copy but returns a buffer wrapping a segment of the actual block's
* byte array. This is because the first key in block is usually stored
* unencoded.
* @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<byte[]> 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);
/**
* Does a deep copy of the key at the current position. A deep copy is
* necessary because buffers are reused in the decoder.
* @return key at current position
*/
public ByteBuffer getKeyDeepCopy();
/**
* Does a shallow copy of the value at the current position. A shallow
* copy is possible because the returned buffer refers to the backing array
* of the original encoded buffer.
* @return value at current position
*/
public ByteBuffer getValueShallowCopy();
/** @return key value at current position. */
public ByteBuffer getKeyValueBuffer();
/**
* @return the KeyValue object at the current position. Includes memstore
* timestamp.
*/
public KeyValue getKeyValue();
/** 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();
/**
* Moves the seeker position within the current block to:
* <ul>
* <li>the last key that that is less than or equal to the given key if
* <code>seekBefore</code> is false</li>
* <li>the last key that is strictly less than the given key if <code>
* seekBefore</code> is true. The caller is responsible for loading the
* previous block if the requested key turns out to be the first key of the
* current block.</li>
* </ul>
* @param key byte array containing the key
* @param offset key position the array
* @param length key length in bytes
* @param seekBefore find the key strictly less than the given key in case
* of an exact match. Does not matter in case of an inexact match.
* @return 0 on exact match, 1 on inexact match.
*/
public int seekToKeyInBlock(byte[] key, int offset, int length,
boolean seekBefore);
}
}

View File

@ -0,0 +1,180 @@
/*
* 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.util.Bytes;
/**
* Provide access to all data block encoding algorithms. All of the algorithms
* are required to have unique id which should <b>NEVER</b> 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 enum DataBlockEncoding {
/** Disable data block encoding. */
NONE(0, null),
// id 1 is reserved for the BITSET algorithm to be added later
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;
public static final int ID_SIZE = Bytes.SIZEOF_SHORT;
/** Maps data block encoding ids to enum instances. */
private static Map<Short, DataBlockEncoding> idToEncoding =
new HashMap<Short, DataBlockEncoding>();
static {
for (DataBlockEncoding algo : values()) {
if (idToEncoding.containsKey(algo.id)) {
throw new RuntimeException(String.format(
"Two data block encoder algorithms '%s' and '%s' have " +
"the same id %d",
idToEncoding.get(algo.id).toString(), algo.toString(),
(int) algo.id));
}
idToEncoding.put(algo.id, algo);
}
}
private DataBlockEncoding(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 != 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 the id 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;
}
/**
* 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<DataBlockEncoder> getAllEncoders() {
ArrayList<DataBlockEncoder> encoders = new ArrayList<DataBlockEncoder>();
for (DataBlockEncoding algo : 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 getDataBlockEncoderById(short encoderId) {
if (!idToEncoding.containsKey(encoderId)) {
throw new IllegalArgumentException(String.format(
"There is no data block encoder for given id '%d'",
(int) encoderId));
}
return idToEncoding.get(encoderId).getEncoder();
}
/**
* Find and return the name of data block encoder for the 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 idToEncoding.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 (!idToEncoding.containsKey(encoderId)) {
throw new IllegalArgumentException(String.format(
"There is no data block encoder for given id '%d'",
(int) encoderId));
}
DataBlockEncoding algorithm = idToEncoding.get(encoderId);
return algorithm.getClass().equals(encoder.getClass());
}
public static DataBlockEncoding getEncodingById(short dataBlockEncodingId) {
return idToEncoding.get(dataBlockEncodingId);
}
}

View File

@ -0,0 +1,549 @@
/*
* 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.moveBufferToStream(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 negativeTimestamp = timestamp < 0;
if (negativeTimestamp) {
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 (negativeTimestamp) {
flag |= FLAG_TIMESTAMP_SIGN;
}
}
}
out.write(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.moveBufferToStream(out, in, restRowLength);
ByteBufferUtils.skip(in, currentState.familyLength +
KeyValue.FAMILY_LENGTH_SIZE);
ByteBufferUtils.moveBufferToStream(out, in, currentState.qualifierLength);
} else {
ByteBufferUtils.moveBufferToStream(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) {
out.write(currentState.type);
}
ByteBufferUtils.skip(in, KeyValue.TIMESTAMP_TYPE_SIZE);
ByteBufferUtils.moveBufferToStream(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.copyFromBufferToBuffer(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.copyFromStreamToBuffer(buffer, source,
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.copyFromStreamToBuffer(buffer, source, 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.copyFromStreamToBuffer(buffer, source, 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.copyFromStreamToBuffer(buffer, source, 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();
}
protected static class DiffSeekerState extends SeekerState {
private int rowLengthWithSize;
private long timestamp;
@Override
protected void copyFromNext(SeekerState that) {
super.copyFromNext(that);
DiffSeekerState other = (DiffSeekerState) that;
rowLengthWithSize = other.rowLengthWithSize;
timestamp = other.timestamp;
}
}
@Override
public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
final boolean includesMemstoreTS) {
return new BufferedEncodedSeeker<DiffSeekerState>(comparator) {
private byte[] familyNameWithSize;
private static final int TIMESTAMP_WITH_TYPE_LENGTH =
Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE;
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);
current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) +
Bytes.SIZEOF_SHORT;
// copy the rest of row
currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT,
current.rowLengthWithSize - Bytes.SIZEOF_SHORT);
// copy the column family
System.arraycopy(familyNameWithSize, 0, current.keyBuffer,
current.rowLengthWithSize, familyNameWithSize.length);
// copy the qualifier
currentBuffer.get(current.keyBuffer,
current.rowLengthWithSize + familyNameWithSize.length,
current.keyLength - current.rowLengthWithSize -
familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH);
} else if (current.lastCommonPrefix < current.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,
current.rowLengthWithSize - current.lastCommonPrefix);
// after column family (qualifier)
currentBuffer.get(current.keyBuffer,
current.rowLengthWithSize + familyNameWithSize.length,
current.keyLength - current.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
current.timestamp = timestampOrDiff;
} else { // it is diff
current.timestamp = current.timestamp - timestampOrDiff;
}
Bytes.putLong(current.keyBuffer, pos, current.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);
}
@Override
protected DiffSeekerState createSeekerState() {
return new DiffSeekerState();
}
};
}
}

View File

@ -0,0 +1,221 @@
/*
* 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<KeyValue> getIterator() {
final int uncompressedSize = uncompressedOutputStream.size();
final ByteArrayInputStream bais = new ByteArrayInputStream(
getCompressedData());
final DataInputStream dis = new DataInputStream(bais);
return new Iterator<KeyValue>() {
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();
}
}

View File

@ -0,0 +1,28 @@
/*
* 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);
}
}

View File

@ -0,0 +1,546 @@
/*
* 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 (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 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;
}
/**
* Copies the first key/value from the given stream, and initializes
* decompression state based on it. Assumes that we have already read key
* and value lengths. Does not set {@link #qualifierLength} (not used by
* decompression) or {@link #prevOffset} (set by the calle afterwards).
*/
private void decompressFirstKV(ByteBuffer out, DataInputStream in)
throws IOException {
int kvPos = out.position();
out.putInt(keyLength);
out.putInt(valueLength);
prevTimestampOffset = out.position() + keyLength -
KeyValue.TIMESTAMP_TYPE_SIZE;
ByteBufferUtils.copyFromStreamToBuffer(out, in, keyLength + valueLength);
rowLength = out.getShort(kvPos + KeyValue.ROW_OFFSET);
familyLength = out.get(kvPos + KeyValue.ROW_OFFSET +
KeyValue.ROW_LENGTH_SIZE + rowLength);
type = out.get(prevTimestampOffset + KeyValue.TIMESTAMP_SIZE);
}
}
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
out.write(flag);
ByteBufferUtils.putCompressedInt(out, keyLength);
ByteBufferUtils.putCompressedInt(out, valueLength);
ByteBufferUtils.putCompressedInt(out, 0);
currentState.readKey(in, keyLength, valueLength);
ByteBufferUtils.moveBufferToStream(out, in, keyLength + valueLength);
} else {
// find a common prefix and skip it
int commonPrefix = ByteBufferUtils.findCommonPrefix(in, in.position(),
previousState.prevOffset + KeyValue.ROW_OFFSET,
Math.min(keyLength, previousState.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 commonTimestampPrefix = findCommonTimestampPrefix(
currentState, previousState);
flag |= commonTimestampPrefix << SHIFT_TIMESTAMP_LENGTH;
// Check if current and previous values are the same. Compare value
// length first as an optimization.
if (valueLength == previousState.valueLength) {
int previousValueOffset = previousState.prevOffset
+ previousState.keyLength + KeyValue.ROW_OFFSET;
if (ByteBufferUtils.arePartsEqual(in,
previousValueOffset, previousState.valueLength,
valueOffset, valueLength)) {
flag |= FLAG_SAME_VALUE;
}
}
out.write(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) {
// Previous and current rows are different. Copy the differing part of
// the row, skip the column family, and copy the qualifier.
ByteBufferUtils.moveBufferToStream(out, in,
currentState.rowLength + KeyValue.ROW_LENGTH_SIZE - commonPrefix);
ByteBufferUtils.skip(in, currentState.familyLength +
KeyValue.FAMILY_LENGTH_SIZE);
ByteBufferUtils.moveBufferToStream(out, in,
currentState.qualifierLength);
} else {
// The common part includes the whole row. As the column family is the
// same across the whole file, it will automatically be included in the
// common prefix, so we need not special-case it here.
int restKeyLength = keyLength - commonPrefix -
KeyValue.TIMESTAMP_TYPE_SIZE;
ByteBufferUtils.moveBufferToStream(out, in, restKeyLength);
}
ByteBufferUtils.skip(in, commonTimestampPrefix);
ByteBufferUtils.moveBufferToStream(out, in,
KeyValue.TIMESTAMP_SIZE - commonTimestampPrefix);
// Write the type if it is not the same as before.
if ((flag & FLAG_SAME_TYPE) == 0) {
out.write(currentState.type);
}
// Write the value if it is not the same as before.
if ((flag & FLAG_SAME_VALUE) == 0) {
ByteBufferUtils.copyBufferToStream(out, in, valueOffset, valueLength);
}
// Skip key type and value in the input buffer.
ByteBufferUtils.skip(in, KeyValue.TYPE_SIZE + currentState.valueLength);
}
}
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 out, 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(out, state.keyLength + state.valueLength +
KeyValue.ROW_OFFSET);
int kvPos = out.position();
if (!state.isFirst()) {
// copy the prefix
int common;
int prevOffset;
if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) {
out.putInt(state.keyLength);
out.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 {
out.putInt(state.keyLength);
prevOffset = state.prevOffset + KeyValue.KEY_LENGTH_SIZE;
common = commonLength + KeyValue.KEY_LENGTH_SIZE;
}
}
ByteBufferUtils.copyFromBufferToBuffer(out, out, 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.copyFromStreamToBuffer(out, source,
KeyValue.ROW_LENGTH_SIZE - commonLength);
rowWithSizeLength = out.getShort(out.position() -
KeyValue.ROW_LENGTH_SIZE) + KeyValue.ROW_LENGTH_SIZE;
rowRestLength = rowWithSizeLength - KeyValue.ROW_LENGTH_SIZE;
} else {
// already in kvBuffer, just read it
rowWithSizeLength = out.getShort(kvPos + KeyValue.ROW_OFFSET) +
KeyValue.ROW_LENGTH_SIZE;
rowRestLength = rowWithSizeLength - commonLength;
}
// copy the rest of row
ByteBufferUtils.copyFromStreamToBuffer(out, source, rowRestLength);
// copy the column family
ByteBufferUtils.copyFromBufferToBuffer(out, out,
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.copyFromStreamToBuffer(out, source, keyRestLength);
// copy timestamp
int prefixTimestamp =
(flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH;
ByteBufferUtils.copyFromBufferToBuffer(out, out,
state.prevTimestampOffset, prefixTimestamp);
state.prevTimestampOffset = out.position() - prefixTimestamp;
ByteBufferUtils.copyFromStreamToBuffer(out, source,
KeyValue.TIMESTAMP_SIZE - prefixTimestamp);
// copy the type and value
if ((flag & FLAG_SAME_TYPE) != 0) {
out.put(state.type);
if ((flag & FLAG_SAME_VALUE) != 0) {
ByteBufferUtils.copyFromBufferToBuffer(out, out, state.prevOffset +
KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength);
} else {
ByteBufferUtils.copyFromStreamToBuffer(out, source,
state.valueLength);
}
} else {
if ((flag & FLAG_SAME_VALUE) != 0) {
ByteBufferUtils.copyFromStreamToBuffer(out, source,
KeyValue.TYPE_SIZE);
ByteBufferUtils.copyFromBufferToBuffer(out, out, state.prevOffset +
KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength);
} else {
ByteBufferUtils.copyFromStreamToBuffer(out, source,
state.valueLength + KeyValue.TYPE_SIZE);
}
state.type = out.get(state.prevTimestampOffset +
KeyValue.TIMESTAMP_SIZE);
}
} else { // this is the first element
state.decompressFirstKV(out, source);
}
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();
}
protected static class FastDiffSeekerState extends SeekerState {
private byte[] prevTimestampAndType =
new byte[KeyValue.TIMESTAMP_TYPE_SIZE];
private int rowLengthWithSize;
private int familyLengthWithSize;
@Override
protected void copyFromNext(SeekerState that) {
super.copyFromNext(that);
FastDiffSeekerState other = (FastDiffSeekerState) that;
System.arraycopy(other.prevTimestampAndType, 0,
prevTimestampAndType, 0,
KeyValue.TIMESTAMP_TYPE_SIZE);
rowLengthWithSize = other.rowLengthWithSize;
familyLengthWithSize = other.familyLengthWithSize;
}
}
@Override
public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
final boolean includesMemstoreTS) {
return new BufferedEncodedSeeker<FastDiffSeekerState>(comparator) {
private void decode(boolean isFirst) {
byte flag = currentBuffer.get();
if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
if (!isFirst) {
System.arraycopy(current.keyBuffer,
current.keyLength - current.prevTimestampAndType.length,
current.prevTimestampAndType, 0,
current.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 - current.prevTimestampAndType.length);
current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) +
Bytes.SIZEOF_SHORT;
current.familyLengthWithSize =
current.keyBuffer[current.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 = current.rowLengthWithSize;
currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
Bytes.SIZEOF_SHORT - current.lastCommonPrefix);
current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) +
Bytes.SIZEOF_SHORT;
// move the column family
System.arraycopy(current.keyBuffer, oldRowLengthWithSize,
current.keyBuffer, current.rowLengthWithSize,
current.familyLengthWithSize);
// copy the rest of row
currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT,
current.rowLengthWithSize - Bytes.SIZEOF_SHORT);
// copy the qualifier
currentBuffer.get(current.keyBuffer, current.rowLengthWithSize
+ current.familyLengthWithSize, current.keyLength
- current.rowLengthWithSize - current.familyLengthWithSize
- current.prevTimestampAndType.length);
} else if (current.lastCommonPrefix < current.rowLengthWithSize) {
// We have to copy part of row and qualifier, but the column family
// is in the right place.
// before column family (rest of row)
currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
current.rowLengthWithSize - current.lastCommonPrefix);
// after column family (qualifier)
currentBuffer.get(current.keyBuffer, current.rowLengthWithSize
+ current.familyLengthWithSize, current.keyLength
- current.rowLengthWithSize - current.familyLengthWithSize
- current.prevTimestampAndType.length);
} else {
// copy just the ending
currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
current.keyLength - current.prevTimestampAndType.length
- current.lastCommonPrefix);
}
// timestamp
int pos = current.keyLength - current.prevTimestampAndType.length;
int commonTimestampPrefix = (flag & MASK_TIMESTAMP_LENGTH) >>>
SHIFT_TIMESTAMP_LENGTH;
if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
System.arraycopy(current.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] =
current.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);
}
@Override
protected FastDiffSeekerState createSeekerState() {
return new FastDiffSeekerState();
}
};
}
}

View File

@ -0,0 +1,196 @@
/*
* 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 prevKeyOffset, DataOutputStream out,
ByteBuffer in, int prevKeyLength) throws IOException {
int keyLength = in.getInt();
int valueLength = in.getInt();
if (prevKeyOffset == -1) {
// copy the key, there is no common prefix with none
ByteBufferUtils.putCompressedInt(out, keyLength);
ByteBufferUtils.putCompressedInt(out, valueLength);
ByteBufferUtils.putCompressedInt(out, 0);
ByteBufferUtils.moveBufferToStream(out, in, keyLength + valueLength);
} else {
// find a common prefix and skip it
int common = ByteBufferUtils.findCommonPrefix(
in, prevKeyOffset + 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.moveBufferToStream(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.copyFromBufferToBuffer(buffer, buffer, prevKeyOffset,
commonLength);
} else {
keyOffset = buffer.position();
}
// copy rest of the key and value
int len = keyLength - commonLength + valueLength;
ByteBufferUtils.copyFromStreamToBuffer(buffer, source, 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<byte[]> comparator,
final boolean includesMemstoreTS) {
return new BufferedEncodedSeeker<SeekerState>(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();
}
};
}
}

View File

@ -1,4 +1,4 @@
/*
/*
* Copyright 2011 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
@ -26,8 +26,8 @@ import java.util.concurrent.atomic.AtomicLong;
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.DataBlockEncoding;
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;
@ -60,6 +60,13 @@ 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 HFileDataBlockEncoder dataBlockEncoder =
NoOpDataBlockEncoder.INSTANCE;
/** Last key in the file. Filled in when we read in the file info */
protected byte [] lastKey = null;
@ -275,8 +282,11 @@ public abstract class AbstractHFileReader extends SchemaConfigured
protected int blockFetches;
public Scanner(final boolean cacheBlocks,
protected final HFile.Reader reader;
public Scanner(final HFile.Reader reader, final boolean cacheBlocks,
final boolean pread, final boolean isCompaction) {
this.reader = reader;
this.cacheBlocks = cacheBlocks;
this.pread = pread;
this.isCompaction = isCompaction;
@ -296,6 +306,26 @@ 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 */
@ -307,4 +337,9 @@ public abstract class AbstractHFileReader extends SchemaConfigured
return path;
}
@Override
public DataBlockEncoding getEncodingOnDisk() {
return dataBlockEncoder.getEncodingOnDisk();
}
}

View File

@ -83,6 +83,12 @@ 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.
* {@link NoOpDataBlockEncoder#INSTANCE} if there is no encoding.
*/
protected final HFileDataBlockEncoder blockEncoder;
/** First key in a block. */
protected byte[] firstKeyInBlock = null;
@ -102,7 +108,9 @@ public abstract class AbstractHFileWriter extends SchemaConfigured
public AbstractHFileWriter(CacheConfig cacheConf,
FSDataOutputStream outputStream, Path path, int blockSize,
Compression.Algorithm compressAlgo, KeyComparator comparator) {
Compression.Algorithm compressAlgo,
HFileDataBlockEncoder dataBlockEncoder,
KeyComparator comparator) {
super(null, path);
this.outputStream = outputStream;
this.path = path;
@ -110,6 +118,8 @@ public abstract class AbstractHFileWriter extends SchemaConfigured
this.blockSize = blockSize;
this.compressAlgo = compressAlgo == null
? HFile.DEFAULT_COMPRESSION_ALGORITHM : compressAlgo;
this.blockEncoder = dataBlockEncoder != null
? dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
this.comparator = comparator != null ? comparator
: Bytes.BYTES_RAWCOMPARATOR;

View File

@ -18,14 +18,28 @@
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
/**
* Cache Key for use with implementations of {@link BlockCache}
*/
public class BlockCacheKey implements HeapSize {
private String hfileName;
private long offset;
private final String hfileName;
private final long offset;
private final DataBlockEncoding encoding;
public BlockCacheKey(String file, long offset, DataBlockEncoding encoding,
BlockType blockType) {
this.hfileName = file;
this.offset = offset;
// We add encoding to the cache key only for data blocks. If the block type
// is unknown (this should never be the case in production), we just use
// the provided encoding, because it might be a data block.
this.encoding = (blockType == null || blockType.isData()) ? encoding :
DataBlockEncoding.NONE;
}
/**
* Construct a new BlockCacheKey
@ -33,13 +47,13 @@ public class BlockCacheKey implements HeapSize {
* @param offset Offset of the block into the file
*/
public BlockCacheKey(String file, long offset) {
this.hfileName = file;
this.offset = offset;
this(file, offset, DataBlockEncoding.NONE, null);
}
@Override
public int hashCode() {
return hfileName.hashCode() * 127 + (int) (offset ^ (offset >>> 32));
return hfileName.hashCode() * 127 + (int) (offset ^ (offset >>> 32)) +
encoding.ordinal() * 17;
}
@Override
@ -56,14 +70,18 @@ public class BlockCacheKey implements HeapSize {
@Override
public String toString() {
return hfileName + "_" + offset;
return hfileName + "_" + offset
+ (encoding == DataBlockEncoding.NONE ? "" : "_" + encoding);
}
// Strings have two bytes per character due to default
// Java unicode encoding (hence the times 2).
/**
* Strings have two bytes per character due to default Java Unicode encoding
* (hence length times 2).
*/
@Override
public long heapSize() {
return 2 * hfileName.length() + Bytes.SIZEOF_LONG;
return ClassSize.align(ClassSize.OBJECT + 2 * hfileName.length() +
Bytes.SIZEOF_LONG + 2 * ClassSize.REFERENCE);
}
// can't avoid this unfortunately
@ -73,4 +91,8 @@ public class BlockCacheKey implements HeapSize {
public String getHfileName() {
return hfileName;
}
public DataBlockEncoding getDataBlockEncoding() {
return encoding;
}
}

View File

@ -39,6 +39,14 @@ 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),
@ -103,6 +111,17 @@ public enum BlockType {
assert magic.length == MAGIC_LENGTH;
}
/**
* Use this instead of {@link #ordinal()}. They work exactly the same, except
* DATA and ENCODED_DATA get the same id using this method (overridden for
* {@link #ENCODED_DATA}).
* @return block type id from 0 to the number of block types - 1
*/
public int getId() {
// Default implementation, can be overridden for individual enum members.
return ordinal();
}
public void writeToStream(OutputStream out) throws IOException {
out.write(magic);
}
@ -189,4 +208,11 @@ public enum BlockType {
}
}
/**
* @return whether this block type is encoded or unencoded data block
*/
public final boolean isData() {
return this == DATA || this == ENCODED_DATA;
}
}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
@ -151,13 +152,14 @@ public class HFile {
public final static int MIN_NUM_HFILE_PATH_LEVELS = 5;
// For measuring latency of "sequential" reads and writes
static volatile AtomicInteger readOps = new AtomicInteger();
static volatile AtomicLong readTimeNano = new AtomicLong();
static volatile AtomicInteger writeOps = new AtomicInteger();
static volatile AtomicLong writeTimeNano = new AtomicLong();
static final AtomicInteger readOps = new AtomicInteger();
static final AtomicLong readTimeNano = new AtomicLong();
static final AtomicInteger writeOps = new AtomicInteger();
static final AtomicLong writeTimeNano = new AtomicLong();
// For measuring latency of pread
static volatile AtomicInteger preadOps = new AtomicInteger();
static volatile AtomicLong preadTimeNano = new AtomicLong();
static final AtomicInteger preadOps = new AtomicInteger();
static final AtomicLong preadTimeNano = new AtomicLong();
// for test purpose
public static volatile AtomicLong dataBlockReadCnt = new AtomicLong(0);
@ -243,6 +245,7 @@ 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,
@ -300,7 +303,8 @@ public class HFile {
/** An abstraction used by the block index */
public interface CachingBlockReader {
HFileBlock readBlock(long offset, long onDiskBlockSize,
boolean cacheBlock, final boolean pread, final boolean isCompaction)
boolean cacheBlock, final boolean pread, final boolean isCompaction,
BlockType expectedBlockType)
throws IOException;
}
@ -368,11 +372,14 @@ public class HFile {
/** Close method with optional evictOnClose */
void close(boolean evictOnClose) throws IOException;
DataBlockEncoding getEncodingOnDisk();
}
private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
long size, boolean closeIStream, CacheConfig cacheConf)
throws IOException {
long size, boolean closeIStream, CacheConfig cacheConf,
DataBlockEncoding preferredEncodingInCache)
throws IOException {
FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis, size);
switch (trailer.getVersion()) {
case 1:
@ -380,23 +387,34 @@ public class HFile {
cacheConf);
case 2:
return new HFileReaderV2(path, trailer, fsdis, size, closeIStream,
cacheConf);
cacheConf, preferredEncodingInCache);
default:
throw new IOException("Cannot instantiate reader for HFile version " +
trailer.getVersion());
}
}
public static Reader createReader(FileSystem fs, Path path,
CacheConfig cacheConf) throws IOException {
public static Reader createReaderWithEncoding(
FileSystem fs, Path path, CacheConfig cacheConf,
DataBlockEncoding preferredEncodingInCache) throws IOException {
final boolean closeIStream = true;
return pickReaderVersion(path, fs.open(path),
fs.getFileStatus(path).getLen(), true, cacheConf);
fs.getFileStatus(path).getLen(), closeIStream, cacheConf,
preferredEncodingInCache);
}
public static Reader createReader(Path path, FSDataInputStream fsdis,
long size, CacheConfig cacheConf)
public static Reader createReader(
FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
return createReaderWithEncoding(fs, path, cacheConf,
DataBlockEncoding.NONE);
}
public static Reader createReaderFromStream(Path path,
FSDataInputStream fsdis, long size, CacheConfig cacheConf)
throws IOException {
return pickReaderVersion(path, fsdis, size, false, cacheConf);
final boolean closeIStream = false;
return pickReaderVersion(path, fsdis, size, closeIStream, cacheConf,
DataBlockEncoding.NONE);
}
/*
@ -501,10 +519,6 @@ public class HFile {
System.exit(prettyPrinter.run(args));
}
public static BlockCacheKey getBlockCacheKey(String hfileName, long offset) {
return new BlockCacheKey(hfileName, offset);
}
/**
* Checks the given {@link HFile} format version, and throws an exception if
* invalid. Note that if the version number comes from an input file and has

View File

@ -17,6 +17,9 @@
*/
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;
@ -30,13 +33,14 @@ 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.encoding.DataBlockEncoding;
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.CompoundBloomFilter;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
@ -45,9 +49,6 @@ 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.
* <ul>
@ -75,10 +76,20 @@ import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.NONE;
*/
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;
/**
* 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
+ DataBlockEncoding.ID_SIZE;
/** Just an array of bytes of the right size. */
public static final byte[] DUMMY_HEADER = new byte[HEADER_SIZE];
@ -107,10 +118,11 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
};
private BlockType blockType;
private final int onDiskSizeWithoutHeader;
private 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
@ -146,7 +158,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 fillHeader, long offset, boolean includesMemstoreTS) {
this.blockType = blockType;
this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
@ -155,6 +167,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
if (fillHeader)
overwriteHeader();
this.offset = offset;
this.includesMemstoreTS = includesMemstoreTS;
}
/**
@ -177,6 +190,15 @@ 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
*/
@ -421,7 +443,9 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
// On-disk size, uncompressed size, and next block's on-disk size
3 * Bytes.SIZEOF_INT +
// This and previous block offset
2 * Bytes.SIZEOF_LONG
2 * Bytes.SIZEOF_LONG +
// "Include memstore timestamp" flag
Bytes.SIZEOF_BOOLEAN
);
if (buf != null) {
@ -509,29 +533,30 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
/** Compression algorithm for all blocks this instance writes. */
private final Compression.Algorithm compressAlgo;
/**
* 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;
/** Data block encoder used for data blocks */
private final HFileDataBlockEncoder dataBlockEncoder;
/**
* The stream we use to accumulate uncompressed block data for
* cache-on-write. Null when cache-on-write is turned off.
* 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.
*/
private ByteArrayOutputStream baosInMemory;
/** Compressor, which is also reused between consecutive blocks. */
private Compressor compressor;
/** Current block type. Set in {@link #startWriting(BlockType)}. */
/**
* Current block type. Set in {@link #startWriting(BlockType)}. Could be
* changed in {@link #encodeDataBlockForDisk()} from {@link BlockType#DATA}
* to {@link BlockType#ENCODED_DATA}.
*/
private BlockType blockType;
/**
* A stream that we write uncompressed bytes to, which compresses them and
* writes them to {@link #baosOnDisk}.
* writes them to {@link #baosInMemory}.
*/
private DataOutputStream userDataStream;
@ -542,14 +567,8 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
private byte[] onDiskBytesWithHeader;
/**
* 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
* 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
* {@link #uncompressedSizeWithoutHeader} + {@link HFileBlock#HEADER_SIZE}.
*/
private byte[] uncompressedBytesWithHeader;
@ -566,30 +585,31 @@ 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;
/**
* @param compressionAlgorithm
* compression algorithm to use
*/
public Writer(Compression.Algorithm compressionAlgorithm) {
compressAlgo = compressionAlgorithm == null ? NONE
: compressionAlgorithm;
/** Whether we are including memstore timestamp after every key/value */
private boolean includesMemstoreTS;
baosOnDisk = new ByteArrayOutputStream();
/**
* @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 : NoOpDataBlockEncoder.INSTANCE;
baosInMemory = 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;
}
/**
@ -598,44 +618,24 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
* @return the stream the user can write their data into
* @throws IOException
*/
public DataOutputStream startWriting(BlockType newBlockType,
boolean cacheOnWrite) throws IOException {
public DataOutputStream startWriting(BlockType newBlockType)
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.ordinal()] = startOffset;
prevOffsetByType[blockType.getId()] = startOffset;
}
this.cacheOnWrite = cacheOnWrite;
startOffset = -1;
blockType = newBlockType;
baosOnDisk.reset();
baosOnDisk.write(DUMMY_HEADER);
baosInMemory.reset();
baosInMemory.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);
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;
}
@ -662,45 +662,108 @@ 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. Block writer
* state transitions must be managed by the caller.
* uncompressed stream for caching on write, if applicable. Sets block
* write state to "block ready".
*/
private void finishBlock() throws IOException {
userDataStream.flush();
uncompressedSizeWithoutHeader = userDataStream.size();
onDiskBytesWithHeader = baosOnDisk.toByteArray();
prevOffset = prevOffsetByType[blockType.ordinal()];
putHeader(onDiskBytesWithHeader, 0);
// This does an array copy, so it is safe to cache this byte array.
uncompressedBytesWithHeader = baosInMemory.toByteArray();
prevOffset = prevOffsetByType[blockType.getId()];
if (cacheOnWrite && compressAlgo != NONE) {
uncompressedBytesWithHeader = baosInMemory.toByteArray();
// 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 (uncompressedSizeWithoutHeader !=
uncompressedBytesWithHeader.length - HEADER_SIZE) {
throw new IOException("Uncompressed size mismatch: "
+ uncompressedSizeWithoutHeader + " vs. "
+ (uncompressedBytesWithHeader.length - HEADER_SIZE));
}
doCompression();
putHeader(uncompressedBytesWithHeader, 0, onDiskBytesWithHeader.length,
uncompressedBytesWithHeader.length);
}
// Write the header into the beginning of the uncompressed byte array.
putHeader(uncompressedBytesWithHeader, 0);
/**
* 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;
}
}
/** Put the header into the given byte array at the given offset. */
private void putHeader(byte[] dest, int offset) {
/**
* 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<ByteBuffer, BlockType> encodingResult =
dataBlockEncoder.beforeWriteToDisk(rawKeyValues,
includesMemstoreTS);
BlockType encodedBlockType = encodingResult.getSecond();
if (encodedBlockType == BlockType.ENCODED_DATA) {
uncompressedBytesWithHeader = encodingResult.getFirst().array();
blockType = BlockType.ENCODED_DATA;
} 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() !=
uncompressedBytesWithHeader.length - HEADER_SIZE) {
throw new IOException("Uncompressed size mismatch: "
+ userDataStream.size() + " vs. "
+ (uncompressedBytesWithHeader.length - HEADER_SIZE));
}
}
}
/**
* 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) {
offset = blockType.put(dest, offset);
offset = Bytes.putInt(dest, offset, onDiskBytesWithHeader.length
- HEADER_SIZE);
offset = Bytes.putInt(dest, offset, uncompressedSizeWithoutHeader);
offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
Bytes.putLong(dest, offset, prevOffset);
}
@ -793,7 +856,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
*/
public int getUncompressedSizeWithoutHeader() {
expectState(State.BLOCK_READY);
return uncompressedSizeWithoutHeader;
return uncompressedBytesWithHeader.length - HEADER_SIZE;
}
/**
@ -801,7 +864,7 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
*/
public int getUncompressedSizeWithHeader() {
expectState(State.BLOCK_READY);
return uncompressedSizeWithoutHeader + HEADER_SIZE;
return uncompressedBytesWithHeader.length;
}
/** @return true if a block is being written */
@ -832,15 +895,6 @@ 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;
}
@ -874,14 +928,15 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
*/
public void writeBlock(BlockWritable bw, FSDataOutputStream out)
throws IOException {
bw.writeToBlock(startWriting(bw.getBlockType(), false));
bw.writeToBlock(startWriting(bw.getBlockType()));
writeHeaderAndData(out);
}
public HFileBlock getBlockForCaching() {
return new HFileBlock(blockType, onDiskBytesWithHeader.length
- HEADER_SIZE, uncompressedSizeWithoutHeader, prevOffset,
getUncompressedBufferWithHeader(), false, startOffset);
return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
getUncompressedSizeWithoutHeader(), prevOffset,
getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
includesMemstoreTS);
}
}
@ -1214,7 +1269,8 @@ 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, true, offset);
uncompressedSizeWithMagic - MAGIC_LENGTH, -1L, buf, FILL_HEADER,
offset, MemStore.NO_PERSISTENT_TS);
return b;
}
}
@ -1232,6 +1288,13 @@ 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;
/** Data block encoding used to read from file */
protected HFileDataBlockEncoder dataBlockEncoder =
NoOpDataBlockEncoder.INSTANCE;
private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
new ThreadLocal<PrefetchedHeader>() {
@Override
@ -1438,6 +1501,8 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
}
}
}
b.includesMemstoreTS = includesMemstoreTS;
b.offset = offset;
return b;
}
@ -1451,6 +1516,13 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
prefetchedHeader.header, 0, HEADER_SIZE);
}
void setIncludesMemstoreTS(boolean enabled) {
includesMemstoreTS = enabled;
}
void setDataBlockEncoder(HFileDataBlockEncoder encoder) {
this.dataBlockEncoder = encoder;
}
}
@Override
@ -1518,5 +1590,16 @@ public class HFileBlock extends SchemaConfigured implements Cacheable {
return true;
}
public boolean doesIncludeMemstoreTS() {
return includesMemstoreTS;
}
public DataBlockEncoding getDataBlockEncoding() {
if (blockType == BlockType.ENCODED_DATA) {
return DataBlockEncoding.getEncodingById(getDataBlockEncodingId());
}
return DataBlockEncoding.NONE;
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
import org.apache.hadoop.hbase.util.Bytes;
@ -200,8 +201,18 @@ public class HFileBlockIndex {
// Call HFile's caching block reader API. We always cache index
// blocks, otherwise we might get terrible performance.
boolean shouldCache = cacheBlocks || (lookupLevel < searchTreeLevel);
block = cachingBlockReader.readBlock(currentOffset, currentOnDiskSize,
shouldCache, pread, isCompaction);
BlockType expectedBlockType;
if (lookupLevel < searchTreeLevel - 1) {
expectedBlockType = BlockType.INTERMEDIATE_INDEX;
} else if (lookupLevel == searchTreeLevel - 1) {
expectedBlockType = BlockType.LEAF_INDEX;
} else {
// this also accounts for ENCODED_DATA
expectedBlockType = BlockType.DATA;
}
block = cachingBlockReader.readBlock(currentOffset,
currentOnDiskSize, shouldCache, pread, isCompaction,
expectedBlockType);
}
if (block == null) {
@ -210,7 +221,8 @@ public class HFileBlockIndex {
}
// Found a data block, break the loop and check our level in the tree.
if (block.getBlockType().equals(BlockType.DATA)) {
if (block.getBlockType().equals(BlockType.DATA) ||
block.getBlockType().equals(BlockType.ENCODED_DATA)) {
break;
}
@ -267,7 +279,8 @@ public class HFileBlockIndex {
// Caching, using pread, assuming this is not a compaction.
HFileBlock midLeafBlock = cachingBlockReader.readBlock(
midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false);
midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false,
BlockType.LEAF_INDEX);
ByteBuffer b = midLeafBlock.getBufferWithoutHeader();
int numDataBlocks = b.getInt();
@ -733,8 +746,8 @@ public class HFileBlockIndex {
long rootLevelIndexPos = out.getPos();
{
DataOutput blockStream = blockWriter.startWriting(BlockType.ROOT_INDEX,
false);
DataOutput blockStream =
blockWriter.startWriting(BlockType.ROOT_INDEX);
rootChunk.writeRoot(blockStream);
if (midKeyMetadata != null)
blockStream.write(midKeyMetadata);
@ -829,7 +842,7 @@ public class HFileBlockIndex {
BlockIndexChunk parent, BlockIndexChunk curChunk) throws IOException {
long beginOffset = out.getPos();
DataOutputStream dos = blockWriter.startWriting(
BlockType.INTERMEDIATE_INDEX, cacheOnWrite());
BlockType.INTERMEDIATE_INDEX);
curChunk.writeNonRoot(dos);
byte[] curFirstKey = curChunk.getBlockKey(0);
blockWriter.writeHeaderAndData(out);
@ -837,8 +850,9 @@ public class HFileBlockIndex {
if (blockCache != null) {
HFileBlock blockForCaching = blockWriter.getBlockForCaching();
passSchemaMetricsTo(blockForCaching);
blockCache.cacheBlock(HFile.getBlockCacheKey(nameForCaching,
beginOffset), blockForCaching);
blockCache.cacheBlock(new BlockCacheKey(nameForCaching,
beginOffset, DataBlockEncoding.NONE,
blockForCaching.getBlockType()), blockForCaching);
}
// Add intermediate index block size

View File

@ -0,0 +1,85 @@
/*
* 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.io.encoding.DataBlockEncoding;
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 or the given block is not a data block (encoded or not), methods
* should just return the unmodified block.
*/
public interface HFileDataBlockEncoder {
/**
* Converts a block from the on-disk format to the in-cache format. Called in
* the following cases:
* <ul>
* <li>After an encoded or unencoded data block is read from disk, but before
* it is put into the cache.</li>
* <li>To convert brand-new blocks to the in-cache format when doing
* cache-on-write.</li>
* </ul>
* @param block a block in an on-disk format (read from HFile or freshly
* generated).
* @return non null block which is coded according to the settings.
*/
public HFileBlock diskToCacheFormat(HFileBlock block,
boolean isCompaction);
/**
* 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<ByteBuffer, BlockType> beforeWriteToDisk(
ByteBuffer in, boolean includesMemstoreTS);
/**
* Decides whether we should use a scanner over encoded blocks.
* @param isCompaction whether we are in a 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;
/** @return the on-disk data block encoding */
public DataBlockEncoding getEncodingOnDisk();
/** @return the preferred in-cache data block encoding for normal reads */
public DataBlockEncoding getEncodingInCache();
/**
* @return the effective in-cache data block encoding, taking into account
* whether we are doing a compaction.
*/
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);
}

View File

@ -0,0 +1,212 @@
/*
* 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.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.base.Preconditions;
/**
* Do different kinds of data block encoding according to column family
* options.
*/
public class HFileDataBlockEncoderImpl implements HFileDataBlockEncoder {
private final DataBlockEncoding onDisk;
private final DataBlockEncoding inCache;
public HFileDataBlockEncoderImpl(DataBlockEncoding encoding) {
this(encoding, encoding);
}
/**
* Do data block encoding with specified options.
* @param onDisk What kind of data block encoding will be used before writing
* HFileBlock to disk. This must be either the same as inCache or
* {@link DataBlockEncoding#NONE}.
* @param inCache What kind of data block encoding will be used in block
* cache.
*/
public HFileDataBlockEncoderImpl(DataBlockEncoding onDisk,
DataBlockEncoding inCache) {
this.onDisk = onDisk != null ?
onDisk : DataBlockEncoding.NONE;
this.inCache = inCache != null ?
inCache : DataBlockEncoding.NONE;
Preconditions.checkArgument(onDisk == DataBlockEncoding.NONE ||
onDisk == inCache, "on-disk encoding (" + onDisk + ") must be " +
"either the same as in-cache encoding (" + inCache + ") or " +
DataBlockEncoding.NONE);
}
public static HFileDataBlockEncoder createFromFileInfo(
FileInfo fileInfo, DataBlockEncoding preferredEncodingInCache)
throws IOException {
byte[] dataBlockEncodingType =
fileInfo.get(StoreFile.DATA_BLOCK_ENCODING);
if (dataBlockEncodingType == null) {
return NoOpDataBlockEncoder.INSTANCE;
}
String dataBlockEncodingStr = Bytes.toString(dataBlockEncodingType);
DataBlockEncoding onDisk;
try {
onDisk = DataBlockEncoding.valueOf(dataBlockEncodingStr);
} catch (IllegalArgumentException ex) {
throw new IOException("Invalid data block encoding type in file info: " +
dataBlockEncodingStr, ex);
}
DataBlockEncoding inCache;
if (onDisk == DataBlockEncoding.NONE) {
// This is an "in-cache-only" encoding or fully-unencoded scenario.
// Either way, we use the given encoding (possibly NONE) specified by
// the column family in cache.
inCache = preferredEncodingInCache;
} else {
// Leave blocks in cache encoded the same way as they are on disk.
// If we switch encoding type for the CF or the in-cache-only encoding
// flag, old files will keep their encoding both on disk and in cache,
// but new files will be generated with the new encoding.
inCache = onDisk;
}
return new HFileDataBlockEncoderImpl(onDisk, inCache);
}
@Override
public void saveMetadata(StoreFile.Writer storeFileWriter)
throws IOException {
storeFileWriter.appendFileInfo(StoreFile.DATA_BLOCK_ENCODING,
onDisk.getNameInBytes());
}
@Override
public DataBlockEncoding getEncodingOnDisk() {
return onDisk;
}
@Override
public DataBlockEncoding getEncodingInCache() {
return inCache;
}
@Override
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
if (!useEncodedScanner(isCompaction)) {
return DataBlockEncoding.NONE;
}
return inCache;
}
@Override
public HFileBlock diskToCacheFormat(HFileBlock block, boolean isCompaction) {
if (block.getBlockType() == BlockType.DATA) {
if (!useEncodedScanner(isCompaction)) {
// Unencoded block, and we don't want to encode in cache.
return block;
}
// Encode the unencoded block with the in-cache encoding.
return encodeDataBlock(block, inCache, block.doesIncludeMemstoreTS());
}
if (block.getBlockType() == BlockType.ENCODED_DATA) {
if (block.getDataBlockEncodingId() == onDisk.getId()) {
// The block is already in the desired in-cache encoding.
return block;
}
// We don't want to re-encode a block in a different encoding. The HFile
// reader should have been instantiated in such a way that we would not
// have to do this.
throw new AssertionError("Expected on-disk data block encoding " +
onDisk + ", got " + block.getDataBlockEncoding());
}
return block;
}
/**
* Precondition: a non-encoded buffer.
* Postcondition: on-disk encoding.
*/
@Override
public Pair<ByteBuffer, BlockType> beforeWriteToDisk(ByteBuffer in,
boolean includesMemstoreTS) {
if (onDisk == DataBlockEncoding.NONE) {
// there is no need to encode the block before writing it to disk
return new Pair<ByteBuffer, BlockType>(in, BlockType.DATA);
}
ByteBuffer encodedBuffer = encodeBufferToHFileBlockBuffer(in,
onDisk, includesMemstoreTS);
return new Pair<ByteBuffer, BlockType>(encodedBuffer,
BlockType.ENCODED_DATA);
}
@Override
public boolean useEncodedScanner(boolean isCompaction) {
if (isCompaction && onDisk == DataBlockEncoding.NONE) {
return false;
}
return inCache != DataBlockEncoding.NONE;
}
private ByteBuffer encodeBufferToHFileBlockBuffer(ByteBuffer in,
DataBlockEncoding 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,
DataBlockEncoding algo, boolean includesMemstoreTS) {
ByteBuffer compressedBuffer = encodeBufferToHFileBlockBuffer(
block.getBufferWithoutHeader(), algo, includesMemstoreTS);
int sizeWithoutHeader = compressedBuffer.limit() - HFileBlock.HEADER_SIZE;
HFileBlock encodedBlock = new HFileBlock(BlockType.ENCODED_DATA,
block.getOnDiskSizeWithoutHeader(),
sizeWithoutHeader, block.getPrevBlockOffset(),
compressedBuffer, HFileBlock.FILL_HEADER, block.getOffset(),
includesMemstoreTS);
block.passSchemaMetricsTo(encodedBlock);
return encodedBlock;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(onDisk=" + onDisk + ", inCache=" +
inCache + ")";
}
}

View File

@ -42,6 +42,7 @@ 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;
@ -170,6 +171,7 @@ 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;

View File

@ -30,9 +30,9 @@ 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.DataBlockEncoding;
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.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.util.Bytes;
@ -42,7 +42,9 @@ import org.apache.hadoop.io.RawComparator;
import com.google.common.base.Preconditions;
/**
* {@link HFile} reader for version 1.
* {@link HFile} reader for version 1. Does not support data block encoding,
* even in cache only, i.e. HFile v1 blocks are always brought into cache
* unencoded.
*/
public class HFileReaderV1 extends AbstractHFileReader {
private static final Log LOG = LogFactory.getLog(HFileReaderV1.class);
@ -212,7 +214,8 @@ public class HFileReaderV1 extends AbstractHFileReader {
long startTimeNs = System.nanoTime();
BlockCacheKey cacheKey = HFile.getBlockCacheKey(name, offset);
BlockCacheKey cacheKey = new BlockCacheKey(name, offset,
DataBlockEncoding.NONE, BlockType.META);
BlockCategory effectiveCategory = BlockCategory.META;
if (metaBlockName.equals(HFileWriterV1.BLOOM_FILTER_META_KEY) ||
@ -280,7 +283,7 @@ public class HFileReaderV1 extends AbstractHFileReader {
}
long offset = dataBlockIndexReader.getRootBlockOffset(block);
BlockCacheKey cacheKey = HFile.getBlockCacheKey(name, offset);
BlockCacheKey cacheKey = new BlockCacheKey(name, offset);
// For any given block from any given file, synchronize reads for said
// block.
@ -297,8 +300,8 @@ public class HFileReaderV1 extends AbstractHFileReader {
cacheConf.shouldCacheDataOnRead());
if (cachedBlock != null) {
cacheHits.incrementAndGet();
getSchemaMetrics().updateOnCacheHit(cachedBlock.getBlockType().getCategory(),
isCompaction);
getSchemaMetrics().updateOnCacheHit(
cachedBlock.getBlockType().getCategory(), isCompaction);
return cachedBlock.getBufferWithoutHeader();
}
// Carry on, please load.
@ -322,7 +325,6 @@ public class HFileReaderV1 extends AbstractHFileReader {
- offset, dataBlockIndexReader.getRootBlockDataSize(block), pread);
passSchemaMetricsTo(hfileBlock);
hfileBlock.expectType(BlockType.DATA);
ByteBuffer buf = hfileBlock.getBufferWithoutHeader();
long delta = System.nanoTime() - startTimeNs;
if (pread) {
@ -341,8 +343,7 @@ public class HFileReaderV1 extends AbstractHFileReader {
cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock,
cacheConf.isInMemory());
}
return buf;
return hfileBlock.getBufferWithoutHeader();
}
}
@ -382,9 +383,12 @@ public class HFileReaderV1 extends AbstractHFileReader {
if (evictOnClose && cacheConf.isBlockCacheEnabled()) {
int numEvicted = 0;
for (int i = 0; i < dataBlockIndexReader.getRootBlockCount(); i++) {
if (cacheConf.getBlockCache().evictBlock(HFile.getBlockCacheKey(name,
dataBlockIndexReader.getRootBlockOffset(i))))
if (cacheConf.getBlockCache().evictBlock(
new BlockCacheKey(name,
dataBlockIndexReader.getRootBlockOffset(i),
DataBlockEncoding.NONE, BlockType.DATA))) {
numEvicted++;
}
}
LOG.debug("On close of file " + name + " evicted " + numEvicted
+ " block(s) of " + dataBlockIndexReader.getRootBlockCount()
@ -396,16 +400,106 @@ public class HFileReaderV1 extends AbstractHFileReader {
}
}
protected abstract static class AbstractScannerV1
extends AbstractHFileReader.Scanner {
protected int currBlock;
/**
* This masks a field with the same name in the superclass and saves us the
* runtime overhead of casting from abstract reader to reader V1.
*/
protected HFileReaderV1 reader;
public AbstractScannerV1(HFileReaderV1 reader, boolean cacheBlocks,
final boolean pread, final boolean isCompaction) {
super(reader, cacheBlocks, pread, isCompaction);
this.reader = (HFileReaderV1) 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 = 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, 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 = reader.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 = 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;
}
}
/**
* Implementation of {@link HFileScanner} interface.
*/
protected static class ScannerV1 extends AbstractHFileReader.Scanner {
private final HFileReaderV1 reader;
private int currBlock;
protected static class ScannerV1 extends AbstractScannerV1 {
private HFileReaderV1 reader;
public ScannerV1(HFileReaderV1 reader, boolean cacheBlocks,
final boolean pread, final boolean isCompaction) {
super(cacheBlocks, pread, isCompaction);
super(reader, cacheBlocks, pread, isCompaction);
this.reader = reader;
}
@ -486,57 +580,7 @@ public class HFileReaderV1 extends AbstractHFileReader {
}
@Override
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,
protected int blockSeek(byte[] key, int offset, int length,
boolean seekBefore) {
int klen, vlen;
int lastLen = 0;
@ -577,37 +621,6 @@ 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(),
@ -620,11 +633,6 @@ 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()) {
@ -645,7 +653,8 @@ public class HFileReaderV1 extends AbstractHFileReader {
return true;
}
private void loadBlock(int bloc, boolean rewind) throws IOException {
@Override
protected void loadBlock(int bloc, boolean rewind) throws IOException {
if (blockBuffer == null) {
blockBuffer = reader.readBlockBuffer(bloc, cacheBlocks, pread,
isCompaction);
@ -674,7 +683,8 @@ public class HFileReaderV1 extends AbstractHFileReader {
@Override
public HFileBlock readBlock(long offset, long onDiskBlockSize,
boolean cacheBlock, boolean pread, boolean isCompaction) {
boolean cacheBlock, boolean pread, boolean isCompaction,
BlockType expectedBlockType) {
throw new UnsupportedOperationException();
}

View File

@ -30,6 +30,8 @@ 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.DataBlockEncoding;
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;
@ -80,17 +82,20 @@ public class HFileReaderV2 extends AbstractHFileReader {
* @param size Length of the stream.
* @param closeIStream Whether to close the stream.
* @param cacheConf Cache configuration.
* @throws IOException
* @param preferredEncodingInCache the encoding to use in cache in case we
* have a choice. If the file is already encoded on disk, we will
* still use its on-disk encoding in cache.
*/
public HFileReaderV2(Path path, FixedFileTrailer trailer,
final FSDataInputStream fsdis, final long size,
final boolean closeIStream, final CacheConfig cacheConf)
throws IOException {
final boolean closeIStream, final CacheConfig cacheConf,
DataBlockEncoding preferredEncodingInCache)
throws IOException {
super(path, trailer, fsdis, size, closeIStream, cacheConf);
trailer.expectVersion(2);
fsBlockReader = new HFileBlock.FSReaderV2(fsdis, compressAlgo,
fileSize);
HFileBlock.FSReaderV2 fsBlockReaderV2 = new HFileBlock.FSReaderV2(fsdis,
compressAlgo, fileSize);
this.fsBlockReader = fsBlockReaderV2; // upcast
// Comparator class name is stored in the trailer in version 2.
comparator = trailer.createComparator();
@ -101,7 +106,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// Parse load-on-open data.
HFileBlock.BlockIterator blockIter = fsBlockReader.blockRange(
HFileBlock.BlockIterator blockIter = fsBlockReaderV2.blockRange(
trailer.getLoadOnOpenDataOffset(),
fileSize - trailer.getTrailerSize());
@ -122,9 +127,17 @@ public class HFileReaderV2 extends AbstractHFileReader {
lastKey = fileInfo.get(FileInfo.LASTKEY);
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);
byte [] keyValueFormatVersion =
fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION);
includesMemstoreTS = keyValueFormatVersion != null &&
Bytes.toInt(keyValueFormatVersion) ==
HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE;
fsBlockReaderV2.setIncludesMemstoreTS(includesMemstoreTS);
// Read data block encoding algorithm name from file info.
dataBlockEncoder = HFileDataBlockEncoderImpl.createFromFileInfo(fileInfo,
preferredEncodingInCache);
fsBlockReaderV2.setDataBlockEncoder(dataBlockEncoder);
// Store all other load-on-open blocks for further consumption.
HFileBlock b;
@ -145,9 +158,15 @@ 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 (dataBlockEncoder.useEncodedScanner(isCompaction)) {
return new EncodedScannerV2(this, cacheBlocks, pread, isCompaction,
includesMemstoreTS);
}
return new ScannerV2(this, cacheBlocks, pread, isCompaction);
}
@ -183,7 +202,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
// Check cache for block. If found return.
long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
BlockCacheKey cacheKey = HFile.getBlockCacheKey(name, metaBlockOffset);
BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset,
DataBlockEncoding.NONE, BlockType.META);
cacheBlock &= cacheConf.shouldCacheDataOnRead();
if (cacheConf.isBlockCacheEnabled()) {
@ -220,19 +240,23 @@ public class HFileReaderV2 extends AbstractHFileReader {
/**
* Read in a file block.
*
* @param dataBlockOffset offset to read.
* @param onDiskBlockSize size of the block
* @param cacheBlock
* @param pread Use positional read instead of seek+read (positional is better
* doing random reads whereas seek+read is better scanning).
* @param pread Use positional read instead of seek+read (positional is
* better doing random reads whereas seek+read is better scanning).
* @param isCompaction is this block being read as part of a compaction
* @param expectedBlockType the block type we are expecting to read with this
* read operation, or null to read whatever block type is available
* and avoid checking (that might reduce caching efficiency of
* encoded data blocks)
* @return Block wrapped in a ByteBuffer.
* @throws IOException
*/
@Override
public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
final boolean cacheBlock, boolean pread, final boolean isCompaction)
final boolean cacheBlock, boolean pread, final boolean isCompaction,
BlockType expectedBlockType)
throws IOException {
if (dataBlockIndexReader == null) {
throw new IOException("Block index not loaded");
@ -249,15 +273,18 @@ public class HFileReaderV2 extends AbstractHFileReader {
// the other choice is to duplicate work (which the cache would prevent you
// from doing).
BlockCacheKey cacheKey = HFile.getBlockCacheKey(name, dataBlockOffset);
BlockCacheKey cacheKey =
new BlockCacheKey(name, dataBlockOffset,
dataBlockEncoder.getEffectiveEncodingInCache(isCompaction),
expectedBlockType);
IdLock.Entry lockEntry = offsetLock.getLockEntry(dataBlockOffset);
try {
blockLoads.incrementAndGet();
// Check cache for block. If found return.
if (cacheConf.isBlockCacheEnabled()) {
HFileBlock cachedBlock =
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
HFileBlock cachedBlock = (HFileBlock)
cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
if (cachedBlock != null) {
BlockCategory blockCategory =
cachedBlock.getBlockType().getCategory();
@ -265,8 +292,21 @@ public class HFileReaderV2 extends AbstractHFileReader {
getSchemaMetrics().updateOnCacheHit(blockCategory, isCompaction);
if (cachedBlock.getBlockType() == BlockType.DATA)
if (cachedBlock.getBlockType() == BlockType.DATA) {
HFile.dataBlockReadCnt.incrementAndGet();
}
validateBlockType(cachedBlock, expectedBlockType);
// Validate encoding type for encoded blocks. We include encoding
// type in the cache key, and we expect it to match on a cache hit.
if (cachedBlock.getBlockType() == BlockType.ENCODED_DATA &&
cachedBlock.getDataBlockEncoding() !=
dataBlockEncoder.getEncodingInCache()) {
throw new IOException("Cached block under key " + cacheKey + " " +
"has wrong encoding: " + cachedBlock.getDataBlockEncoding() +
" (expected: " + dataBlockEncoder.getEncodingInCache() + ")");
}
return cachedBlock;
}
// Carry on, please load.
@ -276,6 +316,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
long startTimeNs = System.nanoTime();
HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset,
onDiskBlockSize, -1, pread);
hfileBlock = dataBlockEncoder.diskToCacheFormat(hfileBlock,
isCompaction);
validateBlockType(hfileBlock, expectedBlockType);
passSchemaMetricsTo(hfileBlock);
BlockCategory blockCategory = hfileBlock.getBlockType().getCategory();
@ -306,6 +349,33 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
}
/**
* Compares the actual type of a block retrieved from cache or disk with its
* expected type and throws an exception in case of a mismatch. Expected
* block type of {@link BlockType#DATA} is considered to match the actual
* block type [@link {@link BlockType#ENCODED_DATA} as well.
* @param block a block retrieved from cache or disk
* @param expectedBlockType the expected block type, or null to skip the
* check
*/
private void validateBlockType(HFileBlock block,
BlockType expectedBlockType) throws IOException {
if (expectedBlockType == null) {
return;
}
BlockType actualBlockType = block.getBlockType();
if (actualBlockType == BlockType.ENCODED_DATA &&
expectedBlockType == BlockType.DATA) {
// We consider DATA to match ENCODED_DATA for the purpose of this
// verification.
return;
}
if (actualBlockType != expectedBlockType) {
throw new IOException("Expected block type " + expectedBlockType + ", " +
"but got " + actualBlockType + ": " + block);
}
}
/**
* @return Last key in the file. May be null if file has no entries. Note that
* this is not the last row key, but rather the byte form of the last
@ -345,31 +415,163 @@ 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, BlockType.DATA);
// 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);
}
// We are reading the next block without block type validation, because
// it might turn out to be a non-data block.
curBlock = reader.readBlock(curBlock.getOffset()
+ curBlock.getOnDiskSizeWithHeader(),
curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
isCompaction, null);
} while (!(curBlock.getBlockType().equals(BlockType.DATA) ||
curBlock.getBlockType().equals(BlockType.ENCODED_DATA)));
return curBlock;
}
}
/**
* Implementation of {@link HFileScanner} interface.
*/
protected static class ScannerV2 extends AbstractHFileReader.Scanner {
private HFileBlock block;
protected static class ScannerV2 extends AbstractScannerV2 {
private HFileReaderV2 reader;
public ScannerV2(HFileReaderV2 r, boolean cacheBlocks,
final boolean pread, final boolean isCompaction) {
super(cacheBlocks, pread, isCompaction);
super(r, cacheBlocks, pread, isCompaction);
this.reader = r;
}
@Override
public HFileReaderV2 getReader() {
return reader;
}
@Override
public KeyValue getKeyValue() {
if (!isSeeked())
return null;
KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position());
KeyValue ret = new KeyValue(blockBuffer.array(),
blockBuffer.arrayOffset() + blockBuffer.position());
if (this.reader.shouldIncludeMemstoreTS()) {
ret.setMemstoreTS(currMemstoreTS);
}
@ -451,36 +653,6 @@ 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.
*
@ -508,7 +680,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
isCompaction);
isCompaction, BlockType.DATA);
if (block.getOffset() < 0) {
throw new IOException("Invalid block offset: " + block.getOffset());
}
@ -517,70 +689,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
@Override
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,
protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind,
byte[] key, int offset, int length, boolean seekBefore)
throws IOException {
if (block == null || block.getOffset() != seekToBlock.getOffset()) {
@ -599,6 +708,16 @@ 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, got " + block.getBlockType() + "; " +
"fileName=" + reader.name + ", " +
"dataBlockEncoder=" + reader.dataBlockEncoder + ", " +
"isCompaction=" + isCompaction);
}
blockBuffer = block.getBufferWithoutHeader();
readKeyValueLen();
blockFetches++;
@ -611,12 +730,14 @@ public class HFileReaderV2 extends AbstractHFileReader {
blockBuffer.reset();
if (this.reader.shouldIncludeMemstoreTS()) {
try {
int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position()
+ KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen;
currMemstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset);
int memstoreTSOffset = blockBuffer.arrayOffset()
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen
+ currValueLen;
currMemstoreTS = Bytes.readVLong(blockBuffer.array(),
memstoreTSOffset);
currMemstoreTSLen = WritableUtils.getVIntSize(currMemstoreTS);
} catch (Exception e) {
throw new RuntimeException("Error reading memstoreTS. " + e);
throw new RuntimeException("Error reading memstore timestamp", e);
}
}
@ -631,7 +752,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
/**
* Within a loaded block, seek looking for the first key that is smaller
* Within a loaded block, seek looking for the last 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
@ -656,12 +777,13 @@ public class HFileReaderV2 extends AbstractHFileReader {
blockBuffer.reset();
if (this.reader.shouldIncludeMemstoreTS()) {
try {
int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position()
+ KEY_VALUE_LEN_SIZE + klen + vlen;
memstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset);
int memstoreTSOffset = blockBuffer.arrayOffset()
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE + klen + vlen;
memstoreTS = Bytes.readVLong(blockBuffer.array(),
memstoreTSOffset);
memstoreTSLen = WritableUtils.getVIntSize(memstoreTS);
} catch (Exception e) {
throw new RuntimeException("Error reading memstoreTS. " + e);
throw new RuntimeException("Error reading memstore timestamp", e);
}
}
@ -713,11 +835,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
@Override
public boolean seekBefore(byte[] key) throws IOException {
return seekBefore(key, 0, key.length);
}
private ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) {
protected ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) {
ByteBuffer buffer = curBlock.getBufferWithoutHeader();
// It is safe to manipulate this buffer because we own the buffer object.
buffer.rewind();
@ -729,40 +847,6 @@ 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(),
@ -776,7 +860,162 @@ public class HFileReaderV2 extends AbstractHFileReader {
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen,
currValueLen);
}
}
/**
* ScannerV2 that operates 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 ||
!DataBlockEncoding.isCorrectEncoder(dataBlockEncoder,
dataBlockEncoderId)) {
DataBlockEncoder encoder =
DataBlockEncoding.getDataBlockEncoderById(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 +
DataBlockEncoding.ID_SIZE,
origBlock.limit() - HFileBlock.HEADER_SIZE -
DataBlockEncoding.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, BlockType.DATA);
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.getKeyDeepCopy();
}
@Override
public ByteBuffer getValue() {
assertValidSeek();
return seeker.getValueShallowCopy();
}
@Override
public KeyValue getKeyValue() {
if (block == null) {
return null;
}
return seeker.getKeyValue();
}
@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.seekToKeyInBlock(key, offset, length, seekBefore);
}
}
/**

View File

@ -35,8 +35,11 @@ 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.encoding.DataBlockEncoding;
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;
@ -44,7 +47,7 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.compress.Compressor;
/**
* Writes version 1 HFiles. Mainly used for testing backwards-compatibilty.
* Writes version 1 HFiles. Mainly used for testing backwards-compatibility.
*/
public class HFileWriterV1 extends AbstractHFileWriter {
@ -91,16 +94,17 @@ public class HFileWriterV1 extends AbstractHFileWriter {
@Override
public Writer createWriter(FileSystem fs, Path path, int blockSize,
Compression.Algorithm compressAlgo, final KeyComparator comparator)
Algorithm compressAlgo, HFileDataBlockEncoder dataBlockEncoder,
KeyComparator comparator)
throws IOException {
return new HFileWriterV1(conf, cacheConf, fs, path, blockSize,
compressAlgo, comparator);
compressAlgo, dataBlockEncoder, comparator);
}
@Override
public Writer createWriter(FileSystem fs, Path path, int blockSize,
String compressAlgoName,
final KeyComparator comparator) throws IOException {
String compressAlgoName, KeyComparator comparator)
throws IOException {
return new HFileWriterV1(conf, cacheConf, fs, path, blockSize,
compressAlgoName, comparator);
}
@ -117,7 +121,8 @@ 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, c);
return new HFileWriterV1(cacheConf, ostream, blockSize, compress,
NoOpDataBlockEncoder.INSTANCE, c);
}
}
@ -127,7 +132,7 @@ public class HFileWriterV1 extends AbstractHFileWriter {
throws IOException {
this(conf, cacheConf, fs, path, HFile.DEFAULT_BLOCKSIZE,
HFile.DEFAULT_COMPRESSION_ALGORITHM,
null);
NoOpDataBlockEncoder.INSTANCE, null);
}
/**
@ -138,15 +143,18 @@ public class HFileWriterV1 extends AbstractHFileWriter {
Path path, int blockSize, String compressAlgoName,
final KeyComparator comparator) throws IOException {
this(conf, cacheConf, fs, path, blockSize,
compressionByName(compressAlgoName), comparator);
compressionByName(compressAlgoName), NoOpDataBlockEncoder.INSTANCE,
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,
public HFileWriterV1(Configuration conf, CacheConfig cacheConf,
FileSystem fs, Path path,
int blockSize, Compression.Algorithm compress,
HFileDataBlockEncoder blockEncoder,
final KeyComparator comparator) throws IOException {
super(cacheConf, createOutputStream(conf, fs, path), path,
blockSize, compress, comparator);
blockSize, compress, blockEncoder, comparator);
SchemaMetrics.configureGlobally(conf);
}
@ -157,15 +165,17 @@ public class HFileWriterV1 extends AbstractHFileWriter {
throws IOException {
this(cacheConf, outputStream, blockSize,
Compression.getCompressionAlgorithmByName(compressAlgoName),
comparator);
NoOpDataBlockEncoder.INSTANCE, comparator);
}
/** Constructor that takes a stream. */
public HFileWriterV1(CacheConfig cacheConf,
final FSDataOutputStream outputStream, final int blockSize,
final Compression.Algorithm compress, final KeyComparator comparator)
final Compression.Algorithm compress,
HFileDataBlockEncoder blockEncoder, final KeyComparator comparator)
throws IOException {
super(cacheConf, outputStream, null, blockSize, compress, comparator);
super(cacheConf, outputStream, null, blockSize, compress,
blockEncoder, comparator);
}
/**
@ -202,13 +212,17 @@ public class HFileWriterV1 extends AbstractHFileWriter {
if (cacheConf.shouldCacheDataOnWrite()) {
baosDos.flush();
// we do not do data block encoding on disk for HFile v1
byte[] bytes = baos.toByteArray();
HFileBlock cBlock = new HFileBlock(BlockType.DATA,
HFileBlock block = new HFileBlock(BlockType.DATA,
(int) (outputStream.getPos() - blockBegin), bytes.length, -1,
ByteBuffer.wrap(bytes, 0, bytes.length), true, blockBegin);
passSchemaMetricsTo(cBlock);
ByteBuffer.wrap(bytes, 0, bytes.length), HFileBlock.FILL_HEADER,
blockBegin, MemStore.NO_PERSISTENT_TS);
block = blockEncoder.diskToCacheFormat(block, false);
passSchemaMetricsTo(block);
cacheConf.getBlockCache().cacheBlock(
HFile.getBlockCacheKey(name, blockBegin), cBlock);
new BlockCacheKey(name, blockBegin, DataBlockEncoding.NONE,
block.getBlockType()), block);
baosDos.close();
}
blockNumber++;

View File

@ -49,9 +49,13 @@ 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;
@ -92,10 +96,10 @@ public class HFileWriterV2 extends AbstractHFileWriter {
@Override
public Writer createWriter(FileSystem fs, Path path, int blockSize,
Compression.Algorithm compress,
Compression.Algorithm compress, HFileDataBlockEncoder blockEncoder,
final KeyComparator comparator) throws IOException {
return new HFileWriterV2(conf, cacheConf, fs, path, blockSize,
compress, comparator);
compress, blockEncoder, comparator);
}
@Override
@ -128,7 +132,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);
HFile.DEFAULT_COMPRESSION_ALGORITHM, null, null);
}
/**
@ -139,15 +143,16 @@ public class HFileWriterV2 extends AbstractHFileWriter {
Path path, int blockSize, String compressAlgoName,
final KeyComparator comparator) throws IOException {
this(conf, cacheConf, fs, path, blockSize,
compressionByName(compressAlgoName), comparator);
compressionByName(compressAlgoName), null, 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, comparator);
blockSize, compressAlgo, blockEncoder, comparator);
SchemaMetrics.configureGlobally(conf);
finishInit(conf);
}
@ -167,7 +172,8 @@ 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, comparator);
super(cacheConf, outputStream, null, blockSize, compress, null,
comparator);
finishInit(conf);
}
@ -177,7 +183,8 @@ public class HFileWriterV2 extends AbstractHFileWriter {
throw new IllegalStateException("finishInit called twice");
// HFile filesystem-level (non-caching) block writer
fsBlockWriter = new HFileBlock.Writer(compressAlgo);
fsBlockWriter = new HFileBlock.Writer(compressAlgo, blockEncoder,
includeMemstoreTS);
// Data block index writer
boolean cacheIndexesOnWrite = cacheConf.shouldCacheIndexesOnWrite();
@ -225,8 +232,9 @@ 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();
@ -253,7 +261,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
long offset = outputStream.getPos();
boolean cacheThisBlock = ibw.cacheOnWrite();
ibw.writeInlineBlock(fsBlockWriter.startWriting(
ibw.getInlineBlockType(), cacheThisBlock));
ibw.getInlineBlockType()));
fsBlockWriter.writeHeaderAndData(outputStream);
ibw.blockWritten(offset, fsBlockWriter.getOnDiskSizeWithHeader(),
fsBlockWriter.getUncompressedSizeWithoutHeader());
@ -272,11 +280,15 @@ public class HFileWriterV2 extends AbstractHFileWriter {
* the cache key.
*/
private void doCacheOnWrite(long offset) {
// Cache this block on write.
HFileBlock cacheFormatBlock = fsBlockWriter.getBlockForCaching();
// We don't cache-on-write data blocks on compaction, so assume this is not
// a compaction.
final boolean isCompaction = false;
HFileBlock cacheFormatBlock = blockEncoder.diskToCacheFormat(
fsBlockWriter.getBlockForCaching(), isCompaction);
passSchemaMetricsTo(cacheFormatBlock);
cacheConf.getBlockCache().cacheBlock(
HFile.getBlockCacheKey(name, offset), cacheFormatBlock);
new BlockCacheKey(name, offset, blockEncoder.getEncodingInCache(),
cacheFormatBlock.getBlockType()), cacheFormatBlock);
}
/**
@ -286,8 +298,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
*/
private void newBlock() throws IOException {
// This is where the next block begins.
fsBlockWriter.startWriting(BlockType.DATA,
cacheConf.shouldCacheDataOnWrite());
fsBlockWriter.startWriting(BlockType.DATA);
firstKeyInBlock = null;
}
@ -419,8 +430,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
// store the beginning offset
long offset = outputStream.getPos();
// write the metadata content
DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META,
cacheConf.shouldCacheDataOnWrite());
DataOutputStream dos = fsBlockWriter.startWriting(BlockType.META);
metaData.get(i).write(dos);
fsBlockWriter.writeHeaderAndData(outputStream);
@ -446,7 +456,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
// Meta block index.
metaBlockIndexWriter.writeSingleLevelIndex(fsBlockWriter.startWriting(
BlockType.ROOT_INDEX, false), "meta");
BlockType.ROOT_INDEX), "meta");
fsBlockWriter.writeHeaderAndData(outputStream);
totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
@ -456,8 +466,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
}
// File info
writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO,
false));
writeFileInfo(trailer, fsBlockWriter.startWriting(BlockType.FILE_INFO));
fsBlockWriter.writeHeaderAndData(outputStream);
totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -753,4 +754,16 @@ public class LruBlockCache implements BlockCache, HeapSize {
}
return counts;
}
public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
Map<DataBlockEncoding, Integer> counts =
new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
for (BlockCacheKey cacheKey : map.keySet()) {
DataBlockEncoding encoding = cacheKey.getDataBlockEncoding();
Integer count = counts.get(encoding);
counts.put(encoding, (count == null ? 0 : count) + 1);
}
return counts;
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.io.encoding.DataBlockEncoding;
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 {
public static final NoOpDataBlockEncoder INSTANCE =
new NoOpDataBlockEncoder();
/** Cannot be instantiated. Use {@link #INSTANCE} instead. */
private NoOpDataBlockEncoder() {
}
@Override
public HFileBlock diskToCacheFormat(HFileBlock block, boolean isCompaction) {
if (block.getBlockType() == BlockType.ENCODED_DATA) {
throw new IllegalStateException("Unexpected encoded block");
}
return block;
}
@Override
public Pair<ByteBuffer, BlockType> beforeWriteToDisk(
ByteBuffer in, boolean includesMemstoreTS) {
return new Pair<ByteBuffer, BlockType>(in, BlockType.DATA);
}
@Override
public boolean useEncodedScanner(boolean isCompaction) {
return false;
}
@Override
public void saveMetadata(StoreFile.Writer storeFileWriter) {
}
@Override
public DataBlockEncoding getEncodingOnDisk() {
return DataBlockEncoding.NONE;
}
@Override
public DataBlockEncoding getEncodingInCache() {
return DataBlockEncoding.NONE;
}
@Override
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
return DataBlockEncoding.NONE;
}
@Override
public String toString() {
return getClass().getSimpleName();
}
}

View File

@ -63,8 +63,11 @@ import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.Reference.Range;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
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;
@ -528,9 +531,12 @@ 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.getDataBlockEncoding());
try {
halfReader = new HalfStoreFileReader(fs, inFile, cacheConf,
reference);
reference, DataBlockEncoding.NONE);
Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
int blocksize = familyDescriptor.getBlocksize();
@ -538,7 +544,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
halfWriter = new StoreFile.Writer(
fs, outFile, blocksize, compression, conf, cacheConf,
fs, outFile, blocksize, compression, dataBlockEncoder,
conf, cacheConf,
KeyValue.COMPARATOR, bloomFilterType, 0);
HFileScanner scanner = halfReader.getScanner(false, false, false);
scanner.seekTo();
@ -638,7 +645,6 @@ 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;

View File

@ -79,7 +79,7 @@ public class CompactSplitThread implements CompactionRequestor {
"hbase.regionserver.thread.compaction.throttle", 0);
} else {
// we have a complicated default. see HBASE-3877
long flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
long flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
long splitSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE,
HConstants.DEFAULT_MAX_FILE_SIZE);

View File

@ -490,7 +490,7 @@ public class HRegion implements HeapSize { // , Writable{
long flushSize = this.htableDescriptor.getMemStoreFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
}
this.memstoreFlushSize = flushSize;

View File

@ -879,6 +879,9 @@ public class MemStore implements HeapSize {
ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
(2 * ClassSize.CONCURRENT_SKIPLISTMAP));
/** Used for readability when we don't store memstore timestamp in HFile */
public static final boolean NO_PERSISTENT_TS = false;
/*
* Calculate how the MemStore size has changed. Includes overhead of the
* backing Map.

View File

@ -54,9 +54,12 @@ 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;
import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.StoreScanner.ScanType;
import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
@ -150,6 +153,7 @@ 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;
@ -187,6 +191,11 @@ 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.getDataBlockEncoding());
this.comparator = info.getComparator();
// getTimeToLive returns ttl in seconds. Convert to milliseconds.
this.ttl = family.getTimeToLive();
@ -276,6 +285,21 @@ 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
*/
void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
this.dataBlockEncoder = blockEncoder;
}
/**
* Creates an unsorted list of StoreFile loaded in parallel
@ -314,7 +338,7 @@ public class Store extends SchemaConfigured implements HeapSize {
completionService.submit(new Callable<StoreFile>() {
public StoreFile call() throws IOException {
StoreFile storeFile = new StoreFile(fs, p, conf, cacheConf,
family.getBloomFilterType());
family.getBloomFilterType(), dataBlockEncoder);
passSchemaMetricsTo(storeFile);
storeFile.createReader();
return storeFile;
@ -488,8 +512,9 @@ 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.family.getBloomFilterType(), this.dataBlockEncoder);
passSchemaMetricsTo(sf);
sf.createReader();
LOG.info("Moved hfile " + srcPath + " into store directory " +
@ -624,7 +649,6 @@ 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;
@ -720,8 +744,9 @@ 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.family.getBloomFilterType(), this.dataBlockEncoder);
passSchemaMetricsTo(sf);
StoreFile.Reader r = sf.createReader();
this.storeSize += r.length();
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
@ -768,8 +793,8 @@ public class Store extends SchemaConfigured implements HeapSize {
writerCacheConf = cacheConf;
}
StoreFile.Writer w = StoreFile.createWriter(fs, region.getTmpDir(),
blocksize, compression, comparator, conf, writerCacheConf,
family.getBloomFilterType(), maxKeyCount);
blocksize, compression, dataBlockEncoder, comparator, conf,
writerCacheConf, family.getBloomFilterType(), maxKeyCount);
// The store file writer's path does not include the CF name, so we need
// to configure the HFile writer directly.
SchemaConfigured sc = (SchemaConfigured) w.writer;
@ -1377,7 +1402,8 @@ public class Store extends SchemaConfigured implements HeapSize {
LOG.debug("Compacting " + file +
", keycount=" + keyCount +
", bloomtype=" + r.getBloomFilterType().toString() +
", size=" + StringUtils.humanReadableInt(r.length()) );
", size=" + StringUtils.humanReadableInt(r.length()) +
", encoding=" + r.getHFileReader().getEncodingOnDisk());
}
}
// For major compactions calculate the earliest put timestamp
@ -1494,7 +1520,8 @@ public class Store extends SchemaConfigured implements HeapSize {
StoreFile storeFile = null;
try {
storeFile = new StoreFile(this.fs, path, this.conf,
this.cacheConf, this.family.getBloomFilterType());
this.cacheConf, this.family.getBloomFilterType(),
NoOpDataBlockEncoder.INSTANCE);
passSchemaMetricsTo(storeFile);
storeFile.createReader();
} catch (IOException e) {
@ -1546,7 +1573,7 @@ public class Store extends SchemaConfigured implements HeapSize {
" to " + destPath);
}
result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
this.family.getBloomFilterType());
this.family.getBloomFilterType(), this.dataBlockEncoder);
passSchemaMetricsTo(result);
result.createReader();
}
@ -1641,7 +1668,7 @@ public class Store extends SchemaConfigured implements HeapSize {
/**
* Find the key that matches <i>row</i> exactly, or the one that immediately
* preceeds it. WARNING: Only use this method on a table where writes occur
* precedes it. WARNING: Only use this method on a table where writes occur
* with strictly increasing timestamps. This method assumes this pattern of
* writes in order to make it reasonably performant. Also our search is
* dependent on the axiom that deletes are for cells that are in the container
@ -2134,8 +2161,8 @@ public class Store extends SchemaConfigured implements HeapSize {
}
public static final long FIXED_OVERHEAD =
ClassSize.align(new SchemaConfigured().heapSize()
+ (18 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG)
ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
+ (19 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG)
+ (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.Compression;
@ -56,6 +57,8 @@ 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;
@ -130,6 +133,10 @@ 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;
@ -148,7 +155,10 @@ public class StoreFile extends SchemaConfigured {
// Block cache configuration and reference.
private final CacheConfig cacheConf;
// HDFS blocks distribuion information
// What kind of data block encoding will be used
private final HFileDataBlockEncoder dataBlockEncoder;
// HDFS blocks distribution information
private HDFSBlocksDistribution hdfsBlocksDistribution;
// Keys for metadata stored in backing HFile.
@ -220,17 +230,22 @@ 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 BloomType cfBloomType,
final HFileDataBlockEncoder dataBlockEncoder)
throws IOException {
this.fs = fs;
this.path = p;
this.cacheConf = cacheConf;
this.dataBlockEncoder =
dataBlockEncoder == null ? NoOpDataBlockEncoder.INSTANCE
: dataBlockEncoder;
if (isReference(p)) {
this.reference = Reference.read(fs, p);
this.referencePath = getReferredToFile(this.path);
@ -493,9 +508,11 @@ public class StoreFile extends SchemaConfigured {
}
if (isReference()) {
this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
this.cacheConf, this.reference);
this.cacheConf, this.reference,
dataBlockEncoder.getEncodingInCache());
} else {
this.reader = new Reader(this.fs, this.path, this.cacheConf);
this.reader = new Reader(this.fs, this.path, this.cacheConf,
dataBlockEncoder.getEncodingInCache());
}
if (isSchemaConfigured()) {
@ -514,7 +531,7 @@ public class StoreFile extends SchemaConfigured {
// By convention, if halfhfile, top half has a sequence number > bottom
// half. Thats why we add one in below. Its done for case the two halves
// are ever merged back together --rare. Without it, on open of store,
// since store files are distingushed by sequence id, the one half would
// since store files are distinguished by sequence id, the one half would
// subsume the other.
this.sequenceid = Bytes.toLong(b);
if (isReference()) {
@ -598,11 +615,11 @@ public class StoreFile extends SchemaConfigured {
}
/**
* @param evictOnClose
* @param evictOnClose whether to evict blocks belonging to this file
* @throws IOException
*/
public synchronized void closeReader(boolean evictOnClose)
throws IOException {
throws IOException {
if (this.reader != null) {
this.reader.close(evictOnClose);
this.reader = null;
@ -677,8 +694,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, null, conf, cacheConf,
BloomType.NONE, 0);
return createWriter(fs, dir, blocksize, null, NoOpDataBlockEncoder.INSTANCE,
null, conf, cacheConf, BloomType.NONE, 0);
}
/**
@ -688,8 +705,9 @@ public class StoreFile extends SchemaConfigured {
* @param dir Path to family directory. Makes the directory if doesn't exist.
* Creates a file with a unique name in this directory.
* @param blocksize
* @param algorithm Pass null to get default.
* @param c Pass null to get default.
* @param compressAlgo Compression algorithm. Pass null to get default.
* @param dataBlockEncoder Pass null to disable data block encoding.
* @param comparator Key-value comparator. Pass null to get default.
* @param conf HBase system configuration. used with bloom filters
* @param cacheConf Cache configuration and reference.
* @param bloomType column family setting for bloom filters
@ -698,14 +716,11 @@ public class StoreFile extends SchemaConfigured {
* @throws IOException
*/
public static StoreFile.Writer createWriter(final FileSystem fs,
final Path dir,
final int blocksize,
final Compression.Algorithm algorithm,
final KeyValue.KVComparator c,
final Configuration conf,
final CacheConfig cacheConf,
BloomType bloomType,
long maxKeyCount)
final Path dir, final int blocksize,
Compression.Algorithm compressAlgo,
final HFileDataBlockEncoder dataBlockEncoder,
KeyValue.KVComparator comparator, final Configuration conf,
final CacheConfig cacheConf, BloomType bloomType, long maxKeyCount)
throws IOException {
if (!fs.exists(dir)) {
@ -716,10 +731,14 @@ public class StoreFile extends SchemaConfigured {
bloomType = BloomType.NONE;
}
return new Writer(fs, path, blocksize,
algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
conf, cacheConf, c == null ? KeyValue.COMPARATOR: c, bloomType,
maxKeyCount);
if (compressAlgo == null) {
compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
}
if (comparator == null) {
comparator = KeyValue.COMPARATOR;
}
return new Writer(fs, path, blocksize, compressAlgo, dataBlockEncoder,
conf, cacheConf, comparator, bloomType, maxKeyCount);
}
/**
@ -814,6 +833,8 @@ 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
@ -838,13 +859,16 @@ public class StoreFile extends SchemaConfigured {
* @throws IOException problem writing to FS
*/
public Writer(FileSystem fs, Path path, int blocksize,
Compression.Algorithm compress, final Configuration conf,
Compression.Algorithm compress,
HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
CacheConfig cacheConf,
final KVComparator comparator, BloomType bloomType, long maxKeys)
throws IOException {
this.dataBlockEncoder = dataBlockEncoder != null ?
dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
writer = HFile.getWriterFactory(conf, cacheConf).createWriter(
fs, path, blocksize,
compress, comparator.getRawComparator());
compress, this.dataBlockEncoder, comparator.getRawComparator());
this.kvComparator = comparator;
@ -940,7 +964,8 @@ public class StoreFile extends SchemaConfigured {
newKey = false;
break;
default:
throw new IOException("Invalid Bloom filter type: " + bloomType);
throw new IOException("Invalid Bloom filter type: " + bloomType +
" (ROW or ROWCOL expected)");
}
}
if (newKey) {
@ -1081,6 +1106,9 @@ public class StoreFile extends SchemaConfigured {
}
public void close() throws IOException {
// Save data block encoder metadata in the file info.
dataBlockEncoder.saveMetadata(this);
boolean hasGeneralBloom = this.closeGeneralBloomFilter();
boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
@ -1119,10 +1147,11 @@ public class StoreFile extends SchemaConfigured {
private byte[] lastBloomKey;
private long deleteFamilyCnt = -1;
public Reader(FileSystem fs, Path path, CacheConfig cacheConf)
throws IOException {
public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
DataBlockEncoding preferredEncodingInCache) throws IOException {
super(path);
reader = HFile.createReader(fs, path, cacheConf);
reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
preferredEncodingInCache);
bloomFilterType = BloomType.NONE;
}
@ -1262,7 +1291,7 @@ public class StoreFile extends SchemaConfigured {
default:
return true;
}
}
}
public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
@ -1312,7 +1341,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 " +

View File

@ -154,8 +154,8 @@ class StoreFileScanner implements KeyValueScanner {
} finally {
realSeekDone = true;
}
} catch(IOException ioe) {
throw new IOException("Could not seek " + this, ioe);
} catch (IOException ioe) {
throw new IOException("Could not seek " + this + " to key " + key, ioe);
}
}
@ -175,7 +175,8 @@ class StoreFileScanner implements KeyValueScanner {
realSeekDone = true;
}
} catch (IOException ioe) {
throw new IOException("Could not seek " + this, ioe);
throw new IOException("Could not reseek " + this + " to key " + key,
ioe);
}
}

View File

@ -138,8 +138,8 @@ class StoreScanner extends NonLazyKeyValueScanner
* @param store who we scan
* @param scan the spec
* @param scanners ancillary scanners
* @param smallestReadPoint the readPoint that we should use for tracking versions
* @param retainDeletesInOutput should we retain deletes after compaction?
* @param smallestReadPoint the readPoint that we should use for tracking
* versions
*/
StoreScanner(Store store, Scan scan,
List<? extends KeyValueScanner> scanners, ScanType scanType,

View File

@ -82,6 +82,15 @@ 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).

View File

@ -0,0 +1,439 @@
/*
* 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 the data to the output stream and update position in buffer.
* @param out the stream to write bytes to
* @param in the buffer to read bytes from
* @param length the number of bytes to copy
*/
public static void moveBufferToStream(OutputStream out, ByteBuffer in,
int length) throws IOException {
copyBufferToStream(out, in, in.position(), length);
skip(in, length);
}
/**
* Copy data from a buffer to an output stream. Does not update the position
* in the buffer.
* @param out the stream to write bytes to
* @param in the buffer to read bytes from
* @param offset the offset in the buffer (from the buffer's array offset)
* to start copying bytes from
* @param length the number of bytes to copy
*/
public static void copyBufferToStream(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));
}
}
}
public static int putLong(OutputStream out, final long value,
final int fitInBytes) throws IOException {
long tmpValue = value;
for (int i = 0; i < fitInBytes; ++i) {
out.write((byte) (tmpValue & 0xff));
tmpValue >>>= 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 the integer that has been read
* @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 in, final int fitInBytes)
throws IOException {
long tmpLong = 0;
for (int i = 0; i < fitInBytes; ++i) {
tmpLong |= (in.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 in, final int fitInBytes) {
long tmpLength = 0;
for (int i = 0; i < fitInBytes; ++i) {
tmpLength |= (in.get() & 0xffl) << (8l * i);
}
return tmpLength;
}
/**
* Asserts that there is at least the given amount of unfilled space
* remaining in the given buffer.
* @param out typically, the buffer we are writing to
* @param length the required space in the buffer
* @throws EncoderBufferTooSmallException If there are no enough bytes.
*/
public static void ensureSpace(ByteBuffer out, int length)
throws EncoderBufferTooSmallException {
if (out.position() + length > out.limit()) {
throw new EncoderBufferTooSmallException(
"Buffer position=" + out.position() +
", buffer limit=" + out.limit() +
", length to be written=" + length);
}
}
/**
* Copy the given number of bytes from the given stream and put it at the
* current position of the given buffer, updating the position in the buffer.
* @param out the buffer to write data to
* @param in the stream to read data from
* @param length the number of bytes to read/write
*/
public static void copyFromStreamToBuffer(ByteBuffer out,
DataInputStream in, int length) throws IOException {
if (out.hasArray()) {
in.readFully(out.array(), out.position() + out.arrayOffset(),
length);
skip(out, length);
} else {
for (int i = 0; i < length; ++i) {
out.put(in.readByte());
}
}
}
/**
* Copy from one buffer to another from given offset
* @param out destination buffer
* @param in source buffer
* @param sourceOffset offset in the source buffer
* @param length how many bytes to copy
*/
public static void copyFromBufferToBuffer(ByteBuffer out,
ByteBuffer in, int sourceOffset, int length) {
if (in.hasArray() && out.hasArray()) {
System.arraycopy(in.array(), sourceOffset + in.arrayOffset(),
out.array(), out.position() +
out.arrayOffset(), length);
skip(out, length);
} else {
for (int i = 0; i < length; ++i) {
out.put(in.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);
}
}

View File

@ -24,6 +24,7 @@ import java.io.DataInput;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
@ -95,7 +96,8 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
try {
// We cache the block and use a positional read.
bloomBlock = reader.readBlock(index.getRootBlockOffset(block),
index.getRootBlockDataSize(block), true, true, false);
index.getRootBlockDataSize(block), true, true, false,
BlockType.BLOOM_CHUNK);
} catch (IOException ex) {
// The Bloom filter is broken, turn it off.
throw new IllegalArgumentException(

View File

@ -532,6 +532,8 @@ 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.setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::DATA_BLOCK_ENCODING)
family.setEncodeOnDisk(JBoolean.valueOf(arg[org.apache.hadoop.hbase.HColumnDescriptor::ENCODE_ON_DISK])) if arg.include?(org.apache.hadoop.hbase.HColumnDescriptor::ENCODE_ON_DISK)
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)

View File

@ -1,330 +0,0 @@
/**
* Copyright 2010 The Apache Software Foundation
*
* 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;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Ignore;
import org.junit.Test;
import com.google.common.collect.Lists;
/**
* Test case that uses multiple threads to read and write multifamily rows
* into a table, verifying that reads never see partially-complete writes.
*
* This can run as a junit test, or with a main() function which runs against
* a real cluster (eg for testing with failures, region movement, etc)
*/
public class BROKE_TODO_FIX_TestAcidGuarantees {
protected static final Log LOG = LogFactory.getLog(BROKE_TODO_FIX_TestAcidGuarantees.class);
public static final byte [] TABLE_NAME = Bytes.toBytes("TestAcidGuarantees");
public static final byte [] FAMILY_A = Bytes.toBytes("A");
public static final byte [] FAMILY_B = Bytes.toBytes("B");
public static final byte [] FAMILY_C = Bytes.toBytes("C");
public static final byte [] QUALIFIER_NAME = Bytes.toBytes("data");
public static final byte[][] FAMILIES = new byte[][] {
FAMILY_A, FAMILY_B, FAMILY_C };
private HBaseTestingUtility util;
public static int NUM_COLS_TO_CHECK = 50;
private void createTableIfMissing()
throws IOException {
try {
util.createTable(TABLE_NAME, FAMILIES);
} catch (TableExistsException tee) {
}
}
public BROKE_TODO_FIX_TestAcidGuarantees() {
// Set small flush size for minicluster so we exercise reseeking scanners
Configuration conf = HBaseConfiguration.create();
conf.set("hbase.hregion.memstore.flush.size", String.valueOf(128*1024));
util = new HBaseTestingUtility(conf);
}
/**
* Thread that does random full-row writes into a table.
*/
public static class AtomicityWriter extends RepeatingTestThread {
Random rand = new Random();
byte data[] = new byte[10];
byte targetRows[][];
byte targetFamilies[][];
HTable table;
AtomicLong numWritten = new AtomicLong();
public AtomicityWriter(TestContext ctx, byte targetRows[][],
byte targetFamilies[][]) throws IOException {
super(ctx);
this.targetRows = targetRows;
this.targetFamilies = targetFamilies;
table = new HTable(ctx.getConf(), TABLE_NAME);
}
public void doAnAction() throws Exception {
// Pick a random row to write into
byte[] targetRow = targetRows[rand.nextInt(targetRows.length)];
Put p = new Put(targetRow);
rand.nextBytes(data);
for (byte[] family : targetFamilies) {
for (int i = 0; i < NUM_COLS_TO_CHECK; i++) {
byte qualifier[] = Bytes.toBytes("col" + i);
p.add(family, qualifier, data);
}
}
table.put(p);
numWritten.getAndIncrement();
}
}
/**
* Thread that does single-row reads in a table, looking for partially
* completed rows.
*/
public static class AtomicGetReader extends RepeatingTestThread {
byte targetRow[];
byte targetFamilies[][];
HTable table;
int numVerified = 0;
AtomicLong numRead = new AtomicLong();
public AtomicGetReader(TestContext ctx, byte targetRow[],
byte targetFamilies[][]) throws IOException {
super(ctx);
this.targetRow = targetRow;
this.targetFamilies = targetFamilies;
table = new HTable(ctx.getConf(), TABLE_NAME);
}
public void doAnAction() throws Exception {
Get g = new Get(targetRow);
Result res = table.get(g);
byte[] gotValue = null;
if (res.getRow() == null) {
// Trying to verify but we didn't find the row - the writing
// thread probably just hasn't started writing yet, so we can
// ignore this action
return;
}
for (byte[] family : targetFamilies) {
for (int i = 0; i < NUM_COLS_TO_CHECK; i++) {
byte qualifier[] = Bytes.toBytes("col" + i);
byte thisValue[] = res.getValue(family, qualifier);
if (gotValue != null && !Bytes.equals(gotValue, thisValue)) {
gotFailure(gotValue, res);
}
numVerified++;
gotValue = thisValue;
}
}
numRead.getAndIncrement();
}
private void gotFailure(byte[] expected, Result res) {
StringBuilder msg = new StringBuilder();
msg.append("Failed after ").append(numVerified).append("!");
msg.append("Expected=").append(Bytes.toStringBinary(expected));
msg.append("Got:\n");
for (KeyValue kv : res.list()) {
msg.append(kv.toString());
msg.append(" val= ");
msg.append(Bytes.toStringBinary(kv.getValue()));
msg.append("\n");
}
throw new RuntimeException(msg.toString());
}
}
/**
* Thread that does full scans of the table looking for any partially completed
* rows.
*/
public static class AtomicScanReader extends RepeatingTestThread {
byte targetFamilies[][];
HTable table;
AtomicLong numScans = new AtomicLong();
AtomicLong numRowsScanned = new AtomicLong();
public AtomicScanReader(TestContext ctx,
byte targetFamilies[][]) throws IOException {
super(ctx);
this.targetFamilies = targetFamilies;
table = new HTable(ctx.getConf(), TABLE_NAME);
}
public void doAnAction() throws Exception {
Scan s = new Scan();
for (byte[] family : targetFamilies) {
s.addFamily(family);
}
ResultScanner scanner = table.getScanner(s);
for (Result res : scanner) {
byte[] gotValue = null;
for (byte[] family : targetFamilies) {
for (int i = 0; i < NUM_COLS_TO_CHECK; i++) {
byte qualifier[] = Bytes.toBytes("col" + i);
byte thisValue[] = res.getValue(family, qualifier);
if (gotValue != null && !Bytes.equals(gotValue, thisValue)) {
gotFailure(gotValue, res);
}
gotValue = thisValue;
}
}
numRowsScanned.getAndIncrement();
}
numScans.getAndIncrement();
}
private void gotFailure(byte[] expected, Result res) {
StringBuilder msg = new StringBuilder();
msg.append("Failed after ").append(numRowsScanned).append("!");
msg.append("Expected=").append(Bytes.toStringBinary(expected));
msg.append("Got:\n");
for (KeyValue kv : res.list()) {
msg.append(kv.toString());
msg.append(" val= ");
msg.append(Bytes.toStringBinary(kv.getValue()));
msg.append("\n");
}
throw new RuntimeException(msg.toString());
}
}
public void runTestAtomicity(long millisToRun,
int numWriters,
int numGetters,
int numScanners,
int numUniqueRows) throws Exception {
createTableIfMissing();
TestContext ctx = new TestContext(util.getConfiguration());
byte rows[][] = new byte[numUniqueRows][];
for (int i = 0; i < numUniqueRows; i++) {
rows[i] = Bytes.toBytes("test_row_" + i);
}
List<AtomicityWriter> writers = Lists.newArrayList();
for (int i = 0; i < numWriters; i++) {
AtomicityWriter writer = new AtomicityWriter(
ctx, rows, FAMILIES);
writers.add(writer);
ctx.addThread(writer);
}
List<AtomicGetReader> getters = Lists.newArrayList();
for (int i = 0; i < numGetters; i++) {
AtomicGetReader getter = new AtomicGetReader(
ctx, rows[i % numUniqueRows], FAMILIES);
getters.add(getter);
ctx.addThread(getter);
}
List<AtomicScanReader> scanners = Lists.newArrayList();
for (int i = 0; i < numScanners; i++) {
AtomicScanReader scanner = new AtomicScanReader(ctx, FAMILIES);
scanners.add(scanner);
ctx.addThread(scanner);
}
ctx.startThreads();
ctx.waitFor(millisToRun);
ctx.stop();
LOG.info("Finished test. Writers:");
for (AtomicityWriter writer : writers) {
LOG.info(" wrote " + writer.numWritten.get());
}
LOG.info("Readers:");
for (AtomicGetReader reader : getters) {
LOG.info(" read " + reader.numRead.get());
}
LOG.info("Scanners:");
for (AtomicScanReader scanner : scanners) {
LOG.info(" scanned " + scanner.numScans.get());
LOG.info(" verified " + scanner.numRowsScanned.get() + " rows");
}
}
@Test
public void testGetAtomicity() throws Exception {
util.startMiniCluster(1);
try {
runTestAtomicity(20000, 5, 5, 0, 3);
} finally {
util.shutdownMiniCluster();
}
}
@Test
@Ignore("Currently not passing - see HBASE-2670")
public void testScanAtomicity() throws Exception {
util.startMiniCluster(1);
try {
runTestAtomicity(20000, 5, 0, 5, 3);
} finally {
util.shutdownMiniCluster();
}
}
@Test
@Ignore("Currently not passing - see HBASE-2670")
public void testMixedAtomicity() throws Exception {
util.startMiniCluster(1);
try {
runTestAtomicity(20000, 5, 2, 2, 3);
} finally {
util.shutdownMiniCluster();
}
}
public static void main(String args[]) throws Exception {
Configuration c = HBaseConfiguration.create();
BROKE_TODO_FIX_TestAcidGuarantees test = new BROKE_TODO_FIX_TestAcidGuarantees();
test.setConf(c);
test.runTestAtomicity(5*60*1000, 5, 2, 2, 3);
}
private void setConf(Configuration c) {
util = new HBaseTestingUtility(c);
}
}

View File

@ -221,18 +221,30 @@ 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, false, false,
HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl,
HColumnDescriptor.DEFAULT_BLOOMFILTER,
HConstants.REPLICATION_SCOPE_LOCAL));
keepDeleted,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
false, false,
HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl,
HColumnDescriptor.DEFAULT_BLOOMFILTER,
HConstants.REPLICATION_SCOPE_LOCAL));
htd.addFamily(new HColumnDescriptor(fam2, minVersions, versions,
keepDeleted, HColumnDescriptor.DEFAULT_COMPRESSION, false, false,
keepDeleted,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
false, false,
HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl,
HColumnDescriptor.DEFAULT_BLOOMFILTER,
HConstants.REPLICATION_SCOPE_LOCAL));
htd.addFamily(new HColumnDescriptor(fam3, minVersions, versions,
keepDeleted, HColumnDescriptor.DEFAULT_COMPRESSION, false, false,
HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl,
keepDeleted,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
false, false,
HColumnDescriptor.DEFAULT_BLOCKSIZE, ttl,
HColumnDescriptor.DEFAULT_BLOOMFILTER,
HConstants.REPLICATION_SCOPE_LOCAL));
return htd;

View File

@ -56,7 +56,9 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -1879,9 +1881,13 @@ public class HBaseTestingUtility {
* @return the number of regions the table was split into
*/
public static int createPreSplitLoadTestTable(Configuration conf,
byte[] tableName, byte[] columnFamily) throws IOException {
byte[] tableName, byte[] columnFamily, Algorithm compression,
DataBlockEncoding dataBlockEncoding) throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(columnFamily));
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
hcd.setDataBlockEncoding(dataBlockEncoding);
hcd.setCompressionType(compression);
desc.addFamily(hcd);
int totalNumberOfRegions = 0;
try {
@ -1924,15 +1930,18 @@ public class HBaseTestingUtility {
public HRegion createTestRegion(String tableName, String cfName,
Compression.Algorithm comprAlgo, BloomType bloomType, int maxVersions,
boolean blockCacheEnabled, int blockSize) throws IOException {
int blockSize, DataBlockEncoding encoding, boolean encodeOnDisk)
throws IOException {
HColumnDescriptor hcd =
new HColumnDescriptor(Bytes.toBytes(cfName), maxVersions,
comprAlgo.getName(),
HColumnDescriptor.DEFAULT_IN_MEMORY,
blockCacheEnabled,
HColumnDescriptor.DEFAULT_BLOCKCACHE,
HColumnDescriptor.DEFAULT_TTL,
bloomType.toString());
hcd.setBlocksize(HFile.DEFAULT_BLOCKSIZE);
hcd.setDataBlockEncoding(encoding);
hcd.setEncodeOnDisk(encodeOnDisk);
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(hcd);
HRegionInfo info =

View File

@ -191,7 +191,8 @@ public class HFilePerformanceEvaluation {
void setUp() throws Exception {
writer =
HFile.getWriterFactoryNoCache(conf).createWriter(this.fs,
this.mf, RFILE_BLOCKSIZE, (Compression.Algorithm) null, null);
this.mf, RFILE_BLOCKSIZE, (Compression.Algorithm) null, null,
null);
}
@Override
@ -365,4 +366,4 @@ public class HFilePerformanceEvaluation {
public static void main(String[] args) throws Exception {
new HFilePerformanceEvaluation().runBenchmarks();
}
}
}

View File

@ -76,7 +76,7 @@ public class TestAcidGuarantees {
public TestAcidGuarantees() {
// Set small flush size for minicluster so we exercise reseeking scanners
Configuration conf = HBaseConfiguration.create();
conf.set("hbase.hregion.memstore.flush.size", String.valueOf(128*1024));
conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(128*1024));
util = new HBaseTestingUtility(conf);
}

View File

@ -373,6 +373,25 @@ public class TestKeyValue extends TestCase {
}
}
public void testCreateKeyValueFromKey() {
KeyValue kv = new KeyValue(Bytes.toBytes("myRow"), Bytes.toBytes("myCF"),
Bytes.toBytes("myQualifier"), 12345L, Bytes.toBytes("myValue"));
int initialPadding = 10;
int endingPadding = 20;
int keyLen = kv.getKeyLength();
byte[] tmpArr = new byte[initialPadding + endingPadding + keyLen];
System.arraycopy(kv.getBuffer(), kv.getKeyOffset(), tmpArr,
initialPadding, keyLen);
KeyValue kvFromKey = KeyValue.createKeyValueFromKey(tmpArr, initialPadding,
keyLen);
assertEquals(keyLen, kvFromKey.getKeyLength());
assertEquals(KeyValue.ROW_OFFSET + keyLen, kvFromKey.getBuffer().length);
System.err.println("kv=" + kv);
System.err.println("kvFromKey=" + kvFromKey);
assertEquals(kvFromKey.toString(),
kv.toString().replaceAll("=[0-9]+$", "=0"));
}
@org.junit.Rule
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

View File

@ -1407,7 +1407,7 @@ public class TestAdmin {
"hbase.hregion.memstore.optionalflushcount", 2);
// We flush the cache after every 8192 bytes
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.flush.size",
TEST_UTIL.getConfiguration().setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
8192);
// Increase the amount of time between client retries

View File

@ -157,6 +157,8 @@ public class TestFromClientSide {
HColumnDescriptor.DEFAULT_VERSIONS,
true,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
HColumnDescriptor.DEFAULT_IN_MEMORY,
HColumnDescriptor.DEFAULT_BLOCKCACHE,
HColumnDescriptor.DEFAULT_BLOCKSIZE,

View File

@ -31,6 +31,7 @@ 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.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
@ -97,8 +98,8 @@ public class TestHalfStoreFileReader {
private void doTestOfScanAndReseek(Path p, FileSystem fs, Reference bottom,
CacheConfig cacheConf)
throws IOException {
final HalfStoreFileReader halfreader =
new HalfStoreFileReader(fs, p, cacheConf, bottom);
final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,
cacheConf, bottom, DataBlockEncoding.NONE);
halfreader.loadFileInfo();
final HFileScanner scanner = halfreader.getScanner(false, false);
@ -135,7 +136,7 @@ public class TestHalfStoreFileReader {
List<KeyValue> genSomeKeys() {
List<KeyValue> ret = new ArrayList<KeyValue>(SIZE);
for (int i = 0 ; i < SIZE; i++) {
for (int i = 0; i < SIZE; i++) {
KeyValue kv =
new KeyValue(
_b(String.format("row_%04d", i)),

View File

@ -40,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -295,6 +296,12 @@ 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
cl = Store.class;
actual = Store.FIXED_OVERHEAD;
@ -313,16 +320,23 @@ public class TestHeapSize extends TestCase {
assertEquals(expected, actual);
}
// Block cache key overhead
cl = BlockCacheKey.class;
// Passing zero length file name, because estimateBase does not handle
// deep overhead.
actual = new BlockCacheKey("", 0).heapSize();
expected = ClassSize.estimateBase(cl, false);
if (expected != actual) {
ClassSize.estimateBase(cl, true);
assertEquals(expected, actual);
}
// Currently NOT testing Deep Overheads of many of these classes.
// Deep overheads cover a vast majority of stuff, but will not be 100%
// accurate because it's unclear when we're referencing stuff that's already
// 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

View File

@ -0,0 +1,290 @@
/*
* 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<byte[]> generateRows() {
// generate prefixes
List<byte[]> prefixes = new ArrayList<byte[]>();
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<byte[]> rows = new ArrayList<byte[]>();
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<KeyValue> generateTestKeyValues(int howMany) {
List<KeyValue> result = new ArrayList<KeyValue>();
List<byte[]> rows = generateRows();
Map<Integer, List<byte[]>> rowsToQualifier =
new HashMap<Integer, List<byte[]>>();
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<byte[]>());
}
rowsToQualifier.get(rowId).add(qualifier);
} else if (qualifierChance > chanceForSameQualifier) {
// similar qualifier
List<byte[]> 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<byte[]> 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<KeyValue> 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;
}
}

View File

@ -0,0 +1,43 @@
/*
* 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]);
}
}
}
}

View File

@ -0,0 +1,262 @@
/*
* 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 java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests changing data block encoding settings of a column family.
*/
@Category(LargeTests.class)
public class TestChangingEncoding {
private static final Log LOG = LogFactory.getLog(TestChangingEncoding.class);
static final String CF = "EncodingTestCF";
static final byte[] CF_BYTES = Bytes.toBytes(CF);
private static final int NUM_ROWS_PER_BATCH = 100;
private static final int NUM_COLS_PER_ROW = 20;
private static final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private static final Configuration conf = TEST_UTIL.getConfiguration();
private static final int TIMEOUT_MS = 120000;
private HBaseAdmin admin;
private HColumnDescriptor hcd;
private String tableName;
private static final List<DataBlockEncoding> ENCODINGS_TO_ITERATE =
createEncodingsToIterate();
private static final List<DataBlockEncoding> createEncodingsToIterate() {
List<DataBlockEncoding> encodings = new ArrayList<DataBlockEncoding>(
Arrays.asList(DataBlockEncoding.values()));
encodings.add(DataBlockEncoding.NONE);
return Collections.unmodifiableList(encodings);
}
/** A zero-based index of the current batch of test data being written */
private int numBatchesWritten;
private void prepareTest(String testId) throws IOException {
tableName = "test_table_" + testId;
HTableDescriptor htd = new HTableDescriptor(tableName);
hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd);
admin.createTable(htd);
numBatchesWritten = 0;
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Use a small flush size to create more HFiles.
conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
TEST_UTIL.startMiniCluster();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setUp() throws Exception {
admin = new HBaseAdmin(conf);
}
@After
public void tearDown() throws IOException {
admin.close();
}
private static byte[] getRowKey(int batchId, int i) {
return Bytes.toBytes("batch" + batchId + "_row" + i);
}
private static byte[] getQualifier(int j) {
return Bytes.toBytes("col" + j);
}
private static byte[] getValue(int batchId, int i, int j) {
return Bytes.toBytes("value_for_" + Bytes.toString(getRowKey(batchId, i))
+ "_col" + j);
}
static void writeTestDataBatch(Configuration conf, String tableName,
int batchId) throws Exception {
LOG.debug("Writing test data batch " + batchId);
HTable table = new HTable(conf, tableName);
for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
Put put = new Put(getRowKey(batchId, i));
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
put.add(CF_BYTES, getQualifier(j),
getValue(batchId, i, j));
table.put(put);
}
}
table.close();
}
static void verifyTestDataBatch(Configuration conf, String tableName,
int batchId) throws Exception {
LOG.debug("Verifying test data batch " + batchId);
HTable table = new HTable(conf, tableName);
for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
Get get = new Get(getRowKey(batchId, i));
Result result = table.get(get);
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
KeyValue kv = result.getColumnLatest(CF_BYTES, getQualifier(j));
assertEquals(Bytes.toStringBinary(getValue(batchId, i, j)),
Bytes.toStringBinary(kv.getValue()));
}
}
table.close();
}
private void writeSomeNewData() throws Exception {
writeTestDataBatch(conf, tableName, numBatchesWritten);
++numBatchesWritten;
}
private void verifyAllData() throws Exception {
for (int i = 0; i < numBatchesWritten; ++i) {
verifyTestDataBatch(conf, tableName, i);
}
}
private void setEncodingConf(DataBlockEncoding encoding,
boolean encodeOnDisk) throws IOException {
LOG.debug("Setting CF encoding to " + encoding + " (ordinal="
+ encoding.ordinal() + "), encodeOnDisk=" + encodeOnDisk);
admin.disableTable(tableName);
hcd.setDataBlockEncoding(encoding);
hcd.setEncodeOnDisk(encodeOnDisk);
admin.modifyColumn(tableName, hcd);
admin.enableTable(tableName);
}
@Test(timeout=TIMEOUT_MS)
public void testChangingEncoding() throws Exception {
prepareTest("ChangingEncoding");
for (boolean encodeOnDisk : new boolean[]{false, true}) {
for (DataBlockEncoding encoding : ENCODINGS_TO_ITERATE) {
setEncodingConf(encoding, encodeOnDisk);
writeSomeNewData();
verifyAllData();
}
}
}
@Test(timeout=TIMEOUT_MS)
public void testChangingEncodingWithCompaction() throws Exception {
prepareTest("ChangingEncodingWithCompaction");
for (boolean encodeOnDisk : new boolean[]{false, true}) {
for (DataBlockEncoding encoding : ENCODINGS_TO_ITERATE) {
setEncodingConf(encoding, encodeOnDisk);
writeSomeNewData();
verifyAllData();
compactAndWait();
verifyAllData();
}
}
}
@Test(timeout=TIMEOUT_MS)
public void testFlippingEncodeOnDisk() throws Exception {
prepareTest("FlippingEncodeOnDisk");
// The focus of this test case is to flip the "encoding on disk" flag,
// so we only try a couple of encodings.
DataBlockEncoding[] encodings = new DataBlockEncoding[] {
DataBlockEncoding.NONE, DataBlockEncoding.FAST_DIFF };
for (DataBlockEncoding encoding : encodings) {
boolean[] flagValues;
if (encoding == DataBlockEncoding.NONE) {
// encodeOnDisk does not matter when not using encoding.
flagValues =
new boolean[] { HColumnDescriptor.DEFAULT_ENCODE_ON_DISK };
} else {
flagValues = new boolean[] { false, true, false, true };
}
for (boolean encodeOnDisk : flagValues) {
setEncodingConf(encoding, encodeOnDisk);
writeSomeNewData();
verifyAllData();
compactAndWait();
verifyAllData();
}
}
}
private void compactAndWait() throws IOException, InterruptedException {
LOG.debug("Compacting table " + tableName);
admin.majorCompact(tableName);
Threads.sleepWithoutInterrupt(500);
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
Threads.sleep(50);
}
LOG.debug("Compaction queue size reached 0, continuing");
}
@Test
public void testCrazyRandomChanges() throws Exception {
prepareTest("RandomChanges");
Random rand = new Random(2934298742974297L);
for (int i = 0; i < 20; ++i) {
int encodingOrdinal = rand.nextInt(DataBlockEncoding.values().length);
DataBlockEncoding encoding = DataBlockEncoding.values()[encodingOrdinal];
setEncodingConf(encoding, rand.nextBoolean());
writeSomeNewData();
verifyAllData();
}
}
}

View File

@ -0,0 +1,346 @@
/*
* 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<Object[]> 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,
Bytes.toStringBinary(dataset), Bytes.toStringBinary(actualDataset));
}
/**
* Test data block encoding of empty KeyValue.
* @throws IOException On test failure.
*/
@Test
public void testEmptyKeyValues() throws IOException {
List<KeyValue> kvList = new ArrayList<KeyValue>();
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<KeyValue> kvList = new ArrayList<KeyValue>();
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<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV);
ByteBuffer originalBuffer =
RedundantKVGenerator.convertKvToByteBuffer(sampleKv,
includesMemstoreTS);
List<DataBlockEncoder> dataBlockEncoders =
DataBlockEncoding.getAllEncoders();
// create all seekers
List<DataBlockEncoder.EncodedSeeker> encodedSeekers =
new ArrayList<DataBlockEncoder.EncodedSeeker>();
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<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV);
ByteBuffer originalBuffer =
RedundantKVGenerator.convertKvToByteBuffer(sampleKv,
includesMemstoreTS);
List<DataBlockEncoder> dataBlockEncoders =
DataBlockEncoding.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.getKeyValueBuffer();
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<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV);
ByteBuffer originalBuffer =
RedundantKVGenerator.convertKvToByteBuffer(sampleKv,
includesMemstoreTS);
List<DataBlockEncoder> dataBlockEncoders =
DataBlockEncoding.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<DataBlockEncoder.EncodedSeeker> encodedSeekers, boolean seekBefore,
KeyValue keyValue) {
ByteBuffer expectedKeyValue = null;
ByteBuffer expectedKey = null;
ByteBuffer expectedValue = null;
for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
seeker.seekToKeyInBlock(keyValue.getBuffer(),
keyValue.getKeyOffset(), keyValue.getKeyLength(), seekBefore);
seeker.rewind();
ByteBuffer actualKeyValue = seeker.getKeyValueBuffer();
ByteBuffer actualKey = seeker.getKeyDeepCopy();
ByteBuffer actualValue = seeker.getValueShallowCopy();
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<DataBlockEncoder> dataBlockEncoders =
DataBlockEncoding.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);
}
}
}

View File

@ -0,0 +1,156 @@
/*
* 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.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.SmallTests;
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.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LoadTestKVGenerator;
import org.apache.hadoop.hbase.util.MultiThreadedWriter;
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;
/**
* Tests encoded seekers by loading and reading values.
*/
@Category(SmallTests.class)
@RunWith(Parameterized.class)
public class TestEncodedSeekers {
private static final String TABLE_NAME = "encodedSeekersTable";
private static final String CF_NAME = "encodedSeekersCF";
private static final byte[] CF_BYTES = Bytes.toBytes(CF_NAME);
private static final int MAX_VERSIONS = 5;
private static final int MIN_VALUE_SIZE = 30;
private static final int MAX_VALUE_SIZE = 60;
private static final int NUM_ROWS = 1000;
private static final int NUM_COLS_PER_ROW = 20;
private static final int NUM_HFILES = 4;
private static final int NUM_ROWS_PER_FLUSH = NUM_ROWS / NUM_HFILES;
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
private final DataBlockEncoding encoding;
private final boolean encodeOnDisk;
/** Enable when debugging */
private static final boolean VERBOSE = false;
@Parameters
public static Collection<Object[]> parameters() {
List<Object[]> paramList = new ArrayList<Object[]>();
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
for (boolean encodeOnDisk : new boolean[]{false, true}) {
paramList.add(new Object[] { encoding, encodeOnDisk });
}
}
return paramList;
}
public TestEncodedSeekers(DataBlockEncoding encoding, boolean encodeOnDisk) {
this.encoding = encoding;
this.encodeOnDisk = encodeOnDisk;
}
@Test
public void testEncodedSeeker() throws IOException {
System.err.println("Testing encoded seekers for encoding " + encoding);
LruBlockCache cache = (LruBlockCache)
new CacheConfig(testUtil.getConfiguration()).getBlockCache();
cache.clearCache();
HRegion region = testUtil.createTestRegion(TABLE_NAME, CF_NAME,
Algorithm.NONE, BloomType.NONE, MAX_VERSIONS, HFile.DEFAULT_BLOCKSIZE,
encoding, encodeOnDisk);
LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(
MIN_VALUE_SIZE, MAX_VALUE_SIZE);
// Write
for (int i = 0; i < NUM_ROWS; ++i) {
byte[] key = MultiThreadedWriter.longToByteArrayKey(i);
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
Put put = new Put(key);
String colAsStr = String.valueOf(j);
byte[] value = dataGenerator.generateRandomSizeValue(i, colAsStr);
put.add(CF_BYTES, Bytes.toBytes(colAsStr), value);
region.put(put);
}
if (i % NUM_ROWS_PER_FLUSH == 0) {
region.flushcache();
}
}
for (int doneCompaction = 0; doneCompaction <= 1; ++doneCompaction) {
// Read
for (int i = 0; i < NUM_ROWS; ++i) {
final byte[] rowKey = MultiThreadedWriter.longToByteArrayKey(i);
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
if (VERBOSE) {
System.err.println("Reading row " + i + ", column " + j);
}
final String qualStr = String.valueOf(j);
final byte[] qualBytes = Bytes.toBytes(qualStr);
Get get = new Get(rowKey);
get.addColumn(CF_BYTES, qualBytes);
Result result = region.get(get, null);
assertEquals(1, result.size());
assertTrue(LoadTestKVGenerator.verify(Bytes.toString(rowKey), qualStr,
result.getValue(CF_BYTES, qualBytes)));
}
}
if (doneCompaction == 0) {
// Compact, then read again at the next loop iteration.
region.compactStores();
}
}
Map<DataBlockEncoding, Integer> encodingCounts =
cache.getEncodingCountsForTest();
// Ensure that compactions don't pollute the cache with unencoded blocks
// in case of in-cache-only encoding.
System.err.println("encodingCounts=" + encodingCounts);
assertEquals(1, encodingCounts.size());
DataBlockEncoding encodingInCache =
encodingCounts.keySet().iterator().next();
assertEquals(encoding, encodingInCache);
assertTrue(encodingCounts.get(encodingInCache) > 0);
}
}

View File

@ -0,0 +1,91 @@
/*
* 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.util.Arrays;
import java.util.Collection;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.TestMiniClusterLoadSequential;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runners.Parameterized.Parameters;
/**
* Uses the load tester
*/
@Category(MediumTests.class)
public class TestLoadAndSwitchEncodeOnDisk extends
TestMiniClusterLoadSequential {
/** We do not alternate the multi-put flag in this test. */
private static final boolean USE_MULTI_PUT = true;
/** Un-parameterize the test */
@Parameters
public static Collection<Object[]> parameters() {
return Arrays.asList(new Object[][]{ new Object[0] });
}
public TestLoadAndSwitchEncodeOnDisk() {
super(USE_MULTI_PUT, DataBlockEncoding.PREFIX);
conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
}
protected int numKeys() {
return 3000;
}
@Test(timeout=TIMEOUT_MS)
public void loadTest() throws Exception {
HBaseAdmin admin = new HBaseAdmin(conf);
compression = Compression.Algorithm.GZ; // used for table setup
super.loadTest();
HColumnDescriptor hcd = getColumnDesc(admin);
System.err.println("\nDisabling encode-on-disk. Old column descriptor: " +
hcd + "\n");
admin.disableTable(TABLE);
hcd.setEncodeOnDisk(false);
admin.modifyColumn(TABLE, hcd);
System.err.println("\nRe-enabling table\n");
admin.enableTable(TABLE);
System.err.println("\nNew column descriptor: " +
getColumnDesc(admin) + "\n");
System.err.println("\nCompacting the table\n");
admin.majorCompact(TABLE);
// Wait until compaction completes
Threads.sleepWithoutInterrupt(5000);
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
Threads.sleep(50);
}
System.err.println("\nDone with the test, shutting down the cluster\n");
}
}

View File

@ -0,0 +1,115 @@
/*
* 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.apache.hadoop.hbase.io.encoding.TestChangingEncoding.CF;
import static org.apache.hadoop.hbase.io.encoding.TestChangingEncoding.CF_BYTES;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestUpgradeFromHFileV1ToEncoding {
private static final Log LOG =
LogFactory.getLog(TestUpgradeFromHFileV1ToEncoding.class);
private static final String TABLE = "UpgradeTable";
private static final byte[] TABLE_BYTES = Bytes.toBytes(TABLE);
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final Configuration conf = TEST_UTIL.getConfiguration();
private static final int NUM_HFILE_V1_BATCHES = 10;
private static final int NUM_HFILE_V2_BATCHES = 20;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Use a small flush size to create more HFiles.
conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
conf.setInt(HFile.FORMAT_VERSION_KEY, 1); // Use HFile v1 initially
TEST_UTIL.startMiniCluster();
LOG.debug("Started an HFile v1 cluster");
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testUpgrade() throws Exception {
int numBatches = 0;
HTableDescriptor htd = new HTableDescriptor(TABLE);
HColumnDescriptor hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd);
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(htd);
admin.close();
for (int i = 0; i < NUM_HFILE_V1_BATCHES; ++i) {
TestChangingEncoding.writeTestDataBatch(conf, TABLE, numBatches++);
}
TEST_UTIL.shutdownMiniHBaseCluster();
conf.setInt(HFile.FORMAT_VERSION_KEY, 2);
TEST_UTIL.startMiniHBaseCluster(1, 1);
LOG.debug("Started an HFile v2 cluster");
admin = new HBaseAdmin(conf);
htd = admin.getTableDescriptor(TABLE_BYTES);
hcd = htd.getFamily(CF_BYTES);
hcd.setDataBlockEncoding(DataBlockEncoding.PREFIX);
admin.disableTable(TABLE);
admin.modifyColumn(TABLE, hcd);
admin.enableTable(TABLE);
admin.close();
for (int i = 0; i < NUM_HFILE_V2_BATCHES; ++i) {
TestChangingEncoding.writeTestDataBatch(conf, TABLE, numBatches++);
}
LOG.debug("Verifying all 'batches', both HFile v1 and encoded HFile v2");
verifyBatches(numBatches);
LOG.debug("Doing a manual compaction");
admin.compact(TABLE);
Thread.sleep(TimeUnit.SECONDS.toMillis(10));
LOG.debug("Verify all the data again");
verifyBatches(numBatches);
}
private void verifyBatches(int numBatches) throws Exception {
for (int i = 0; i < numBatches; ++i) {
TestChangingEncoding.verifyTestDataBatch(conf, TABLE, i);
}
}
}

View File

@ -41,9 +41,15 @@ import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
public class CacheTestUtils {
/*Just checks if heapsize grows when something is cached, and gets smaller when the same object is evicted*/
private static final boolean includesMemstoreTS = true;
public static void testHeapSizeChanges(final BlockCache toBeTested, final int blockSize){
/**
* 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) {
HFileBlockPair[] blocks = generateHFileBlocks(blockSize, 1);
long heapSize = ((HeapSize) toBeTested).heapSize();
toBeTested.cacheBlock(blocks[0].blockName, blocks[0].block);
@ -316,7 +322,8 @@ public class CacheTestUtils {
HFileBlock generated = new HFileBlock(BlockType.DATA,
onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
prevBlockOffset, cachedBuffer, false, blockSize);
prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
blockSize, includesMemstoreTS);
String strKey;
/* No conflicting keys */

View File

@ -38,10 +38,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
@ -74,10 +74,13 @@ 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 testName;
private String testDescription;
private final CacheOnWriteType cowType;
private final Compression.Algorithm compress;
private final BlockEncoderTestType encoderType;
private final HFileDataBlockEncoder encoder;
private static final int DATA_BLOCK_SIZE = 2048;
private static final int NUM_KV = 25000;
@ -90,49 +93,87 @@ public class TestCacheOnWrite {
KeyValue.Type.values().length - 2;
private static enum CacheOnWriteType {
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);
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);
private final String confKey;
private final BlockType inlineBlockType;
private final BlockType blockType1;
private final BlockType blockType2;
private CacheOnWriteType(BlockType inlineBlockType, String confKey) {
this.inlineBlockType = 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;
this.confKey = confKey;
}
public boolean shouldBeCached(BlockType blockType) {
return blockType == inlineBlockType
|| blockType == BlockType.INTERMEDIATE_INDEX
&& inlineBlockType == BlockType.LEAF_INDEX;
return blockType == blockType1 || blockType == blockType2;
}
public void modifyConf(Configuration conf) {
for (CacheOnWriteType cowType : CacheOnWriteType.values())
for (CacheOnWriteType cowType : CacheOnWriteType.values()) {
conf.setBoolean(cowType.confKey, cowType == this);
}
}
}
private static final DataBlockEncoding ENCODING_ALGO =
DataBlockEncoding.PREFIX;
/** Provides fancy names for three combinations of two booleans */
private static enum BlockEncoderTestType {
NO_BLOCK_ENCODING(false, false),
BLOCK_ENCODING_IN_CACHE_ONLY(false, true),
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() {
return new HFileDataBlockEncoderImpl(
encodeOnDisk ? ENCODING_ALGO : DataBlockEncoding.NONE,
encodeInCache ? ENCODING_ALGO : DataBlockEncoding.NONE);
}
}
public TestCacheOnWrite(CacheOnWriteType cowType,
Compression.Algorithm compress) {
Compression.Algorithm compress, BlockEncoderTestType encoderType) {
this.cowType = cowType;
this.compress = compress;
testName = "[cacheOnWrite=" + cowType + ", compress=" + compress + "]";
System.out.println(testName);
this.encoderType = encoderType;
this.encoder = encoderType.getEncoder();
testDescription = "[cacheOnWrite=" + cowType + ", compress=" + compress +
", encoderType=" + encoderType + "]";
System.out.println(testDescription);
}
@Parameters
public static Collection<Object[]> getParameters() {
List<Object[]> cowTypes = new ArrayList<Object[]>();
for (CacheOnWriteType cowType : CacheOnWriteType.values())
for (CacheOnWriteType cowType : CacheOnWriteType.values()) {
for (Compression.Algorithm compress :
HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
cowTypes.add(new Object[] { cowType, compress });
for (BlockEncoderTestType encoderType :
BlockEncoderTestType.values()) {
cowTypes.add(new Object[] { cowType, compress, encoderType });
}
}
}
return cowTypes;
}
@ -153,7 +194,6 @@ public class TestCacheOnWrite {
fs = FileSystem.get(conf);
cacheConf = new CacheConfig(conf);
blockCache = cacheConf.getBlockCache();
System.out.println("setUp()");
}
@After
@ -169,29 +209,43 @@ public class TestCacheOnWrite {
}
private void readStoreFile() throws IOException {
HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs,
storeFilePath, cacheConf);
HFileReaderV2 reader = (HFileReaderV2) HFile.createReaderWithEncoding(fs,
storeFilePath, cacheConf, encoder.getEncodingInCache());
LOG.info("HFile information: " + reader);
HFileScanner scanner = reader.getScanner(false, false);
assertTrue(testName, scanner.seekTo());
final boolean cacheBlocks = false;
final boolean pread = false;
HFileScanner scanner = reader.getScanner(cacheBlocks, pread);
assertTrue(testDescription, scanner.seekTo());
long offset = 0;
HFileBlock prevBlock = null;
EnumMap<BlockType, Integer> blockCountByType =
new EnumMap<BlockType, Integer>(BlockType.class);
DataBlockEncoding encodingInCache =
encoderType.getEncoder().getEncodingInCache();
while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
long onDiskSize = -1;
if (prevBlock != null) {
onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
}
// Flags: don't cache the block, use pread, this is not a compaction.
// Also, pass null for expected block type to avoid checking it.
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
false);
BlockCacheKey blockCacheKey = HFile.getBlockCacheKey(reader.getName(), offset);
false, null);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
offset, encodingInCache, block.getBlockType());
boolean isCached = blockCache.getBlock(blockCacheKey, true) != null;
boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
assertEquals(testName + " " + block, shouldBeCached, isCached);
if (shouldBeCached != isCached) {
throw new AssertionError(
"shouldBeCached: " + shouldBeCached+ "\n" +
"isCached: " + isCached + "\n" +
"Test description: " + testDescription + "\n" +
"block: " + block + "\n" +
"encodingInCache: " + encodingInCache + "\n" +
"blockCacheKey: " + blockCacheKey);
}
prevBlock = block;
offset += block.getOnDiskSizeWithHeader();
BlockType bt = block.getBlockType();
@ -201,8 +255,10 @@ public class TestCacheOnWrite {
LOG.info("Block count by type: " + blockCountByType);
String countByType = blockCountByType.toString();
assertEquals(
"{DATA=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}",
BlockType cachedDataBlockType =
encoderType.encodeInCache ? BlockType.ENCODED_DATA : BlockType.DATA;
assertEquals("{" + cachedDataBlockType
+ "=1379, LEAF_INDEX=173, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}",
countByType);
reader.close();
@ -228,7 +284,7 @@ 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, KeyValue.COMPARATOR, conf,
DATA_BLOCK_SIZE, compress, encoder, KeyValue.COMPARATOR, conf,
cacheConf, BLOOM_TYPE, NUM_KV);
final int rowLen = 32;
@ -260,8 +316,9 @@ public class TestCacheOnWrite {
final byte[] cfBytes = Bytes.toBytes(cf);
final int maxVersions = 3;
HRegion region = TEST_UTIL.createTestRegion(table, cf, compress,
BLOOM_TYPE, maxVersions, HColumnDescriptor.DEFAULT_BLOCKCACHE,
HFile.DEFAULT_BLOCKSIZE);
BLOOM_TYPE, maxVersions, HFile.DEFAULT_BLOCKSIZE,
encoder.getEncodingInCache(),
encoder.getEncodingOnDisk() != DataBlockEncoding.NONE);
int rowIdx = 0;
long ts = EnvironmentEdgeManager.currentTimeMillis();
for (int iFile = 0; iFile < 5; ++iFile) {

View File

@ -158,7 +158,7 @@ public class TestHFile extends HBaseTestCase {
writeRecords(writer);
fout.close();
FSDataInputStream fin = fs.open(ncTFile);
Reader reader = HFile.createReader(ncTFile, fs.open(ncTFile),
Reader reader = HFile.createReaderFromStream(ncTFile, fs.open(ncTFile),
fs.getFileStatus(ncTFile).getLen(), cacheConf);
System.out.println(cacheConf.toString());
// Load up the index.
@ -236,7 +236,7 @@ public class TestHFile extends HBaseTestCase {
writer.close();
fout.close();
FSDataInputStream fin = fs.open(mFile);
Reader reader = HFile.createReader(mFile, fs.open(mFile),
Reader reader = HFile.createReaderFromStream(mFile, fs.open(mFile),
this.fs.getFileStatus(mFile).getLen(), cacheConf);
reader.loadFileInfo();
// No data -- this should return false.

View File

@ -27,6 +27,8 @@ 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;
@ -45,16 +47,24 @@ 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.DataBlockEncoding;
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;
@ -69,14 +79,29 @@ 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 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<Object[]> parameters() {
return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
}
@Before
public void setUp() throws IOException {
fs = FileSystem.get(TEST_UTIL.getConfiguration());
@ -88,6 +113,72 @@ public class TestHFileBlock {
dos.writeInt(i / 100);
}
private int writeTestKeyValues(OutputStream dos, int seed)
throws IOException {
List<KeyValue> keyValues = new ArrayList<KeyValue>();
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();
@ -105,8 +196,9 @@ public class TestHFileBlock {
private byte[] createTestV2Block(Compression.Algorithm algo)
throws IOException {
final BlockType blockType = BlockType.DATA;
HFileBlock.Writer hbw = new HFileBlock.Writer(algo);
DataOutputStream dos = hbw.startWriting(blockType, false);
HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
includesMemstoreTS);
DataOutputStream dos = hbw.startWriting(blockType);
writeTestBlockContents(dos);
byte[] headerAndData = hbw.getHeaderAndData();
assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
@ -194,10 +286,11 @@ 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);
HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
includesMemstoreTS);
long totalSize = 0;
for (int blockId = 0; blockId < 2; ++blockId) {
DataOutputStream dos = hbw.startWriting(BlockType.DATA, false);
DataOutputStream dos = hbw.startWriting(BlockType.DATA);
for (int i = 0; i < 1234; ++i)
dos.writeInt(i);
hbw.writeHeaderAndData(os);
@ -240,6 +333,136 @@ public class TestHFileBlock {
}
}
/**
* Test encoding/decoding data blocks.
* @throws IOException a bug or a problem with temporary files.
*/
@Test
public void testDataBlockEncoding() throws IOException {
final int numBlocks = 5;
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
for (boolean pread : new boolean[] { false, true }) {
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
+ algo + "_" + encoding.toString());
FSDataOutputStream os = fs.create(path);
HFileDataBlockEncoder dataBlockEncoder =
new HFileDataBlockEncoderImpl(encoding);
HFileBlock.Writer hbw = new HFileBlock.Writer(algo, dataBlockEncoder,
includesMemstoreTS);
long totalSize = 0;
final List<Integer> encodedSizes = new ArrayList<Integer>();
final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
for (int blockId = 0; blockId < numBlocks; ++blockId) {
writeEncodedBlock(encoding, hbw, encodedSizes, encodedBlocks,
blockId);
hbw.writeHeaderAndData(os);
totalSize += hbw.getOnDiskSizeWithHeader();
}
os.close();
FSDataInputStream is = fs.open(path);
HFileBlock.FSReaderV2 hbr = new HFileBlock.FSReaderV2(is, algo,
totalSize);
hbr.setDataBlockEncoder(dataBlockEncoder);
hbr.setIncludesMemstoreTS(includesMemstoreTS);
HFileBlock b;
int pos = 0;
for (int blockId = 0; blockId < numBlocks; ++blockId) {
b = hbr.readBlockData(pos, -1, -1, pread);
b.sanityCheck();
pos += b.getOnDiskSizeWithHeader();
assertEquals((int) encodedSizes.get(blockId),
b.getUncompressedSizeWithoutHeader());
ByteBuffer actualBuffer = b.getBufferWithoutHeader();
if (encoding != DataBlockEncoding.NONE) {
// We expect a two-byte big-endian encoding id.
assertEquals(0, actualBuffer.get(0));
assertEquals(encoding.getId(), actualBuffer.get(1));
actualBuffer.position(2);
actualBuffer = actualBuffer.slice();
}
ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
expectedBuffer.rewind();
// test if content matches, produce nice message
assertBuffersEqual(expectedBuffer, actualBuffer, algo, encoding,
pread);
}
is.close();
}
}
}
}
private void writeEncodedBlock(DataBlockEncoding encoding,
HFileBlock.Writer hbw, final List<Integer> encodedSizes,
final List<ByteBuffer> encodedBlocks, int blockId) throws IOException {
DataOutputStream dos = hbw.startWriting(BlockType.DATA);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DoubleOutputStream doubleOutputStream =
new DoubleOutputStream(dos, baos);
final int rawBlockSize = writeTestKeyValues(doubleOutputStream,
blockId);
ByteBuffer rawBuf = ByteBuffer.wrap(baos.toByteArray());
rawBuf.rewind();
final int encodedSize;
final ByteBuffer encodedBuf;
if (encoding == DataBlockEncoding.NONE) {
encodedSize = rawBlockSize;
encodedBuf = rawBuf;
} else {
ByteArrayOutputStream encodedOut = new ByteArrayOutputStream();
encoding.getEncoder().compressKeyValues(
new DataOutputStream(encodedOut),
rawBuf.duplicate(), includesMemstoreTS);
// We need to account for the two-byte encoding algorithm ID that
// comes after the 24-byte block header but before encoded KVs.
encodedSize = encodedOut.size() + DataBlockEncoding.ID_SIZE;
encodedBuf = ByteBuffer.wrap(encodedOut.toByteArray());
}
encodedSizes.add(encodedSize);
encodedBlocks.add(encodedBuf);
}
private void assertBuffersEqual(ByteBuffer expectedBuffer,
ByteBuffer actualBuffer, Compression.Algorithm compression,
DataBlockEncoding encoding, boolean pread) {
if (!actualBuffer.equals(expectedBuffer)) {
int prefix = 0;
int minLimit = Math.min(expectedBuffer.limit(), actualBuffer.limit());
while (prefix < minLimit &&
expectedBuffer.get(prefix) == actualBuffer.get(prefix)) {
prefix++;
}
fail(String.format(
"Content mismath for compression %s, encoding %s, " +
"pread %s, commonPrefix %d, expected %s, got %s",
compression, encoding, pread, prefix,
nextBytesToStr(expectedBuffer, prefix),
nextBytesToStr(actualBuffer, prefix)));
}
}
/**
* Convert a few next bytes in the given buffer at the given position to
* string. Used for error messages.
*/
private static String nextBytesToStr(ByteBuffer buf, int pos) {
int maxBytes = buf.limit() - pos;
int numBytes = Math.min(16, maxBytes);
return Bytes.toStringBinary(buf.array(), buf.arrayOffset() + pos,
numBytes) + (numBytes < maxBytes ? "..." : "");
}
@Test
public void testPreviousOffset() throws IOException {
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
@ -446,13 +669,17 @@ public class TestHFileBlock {
) throws IOException {
boolean cacheOnWrite = expectedContents != null;
FSDataOutputStream os = fs.create(path);
HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo);
HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo, null,
includesMemstoreTS);
Map<BlockType, Long> prevOffsetByType = new HashMap<BlockType, Long>();
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, cacheOnWrite);
DataOutputStream dos = hbw.startWriting(bt);
for (int j = 0; j < rand.nextInt(500); ++j) {
// This might compress well.
dos.writeShort(i + 1);
@ -501,7 +728,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,
true, -1);
HFileBlock.FILL_HEADER, -1, includesMemstoreTS);
long byteBufferExpectedSize =
ClassSize.align(ClassSize.estimateBase(buf.getClass(), true)
+ HFileBlock.HEADER_SIZE + size);

View File

@ -20,6 +20,10 @@
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;
@ -44,7 +48,6 @@ 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;
@ -52,8 +55,6 @@ 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 {
@ -92,6 +93,8 @@ 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;
@ -138,7 +141,8 @@ public class TestHFileBlockIndex {
@Override
public HFileBlock readBlock(long offset, long onDiskSize,
boolean cacheBlock, boolean pread, boolean isCompaction)
boolean cacheBlock, boolean pread, boolean isCompaction,
BlockType expectedBlockType)
throws IOException {
if (offset == prevOffset && onDiskSize == prevOnDiskSize &&
pread == prevPread) {
@ -210,13 +214,14 @@ public class TestHFileBlockIndex {
private void writeWholeIndex() throws IOException {
assertEquals(0, keys.size());
HFileBlock.Writer hbw = new HFileBlock.Writer(compr);
HFileBlock.Writer hbw = new HFileBlock.Writer(compr, null,
includesMemstoreTS);
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, false).write(
hbw.startWriting(BlockType.DATA).write(
String.valueOf(rand.nextInt(1000)).getBytes());
long blockOffset = outputStream.getPos();
hbw.writeHeaderAndData(outputStream);
@ -251,7 +256,7 @@ public class TestHFileBlockIndex {
boolean isClosing) throws IOException {
while (biw.shouldWriteBlock(isClosing)) {
long offset = outputStream.getPos();
biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType(), false));
biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType()));
hbw.writeHeaderAndData(outputStream);
biw.blockWritten(offset, hbw.getOnDiskSizeWithHeader(),
hbw.getUncompressedSizeWithoutHeader());
@ -479,7 +484,7 @@ public class TestHFileBlockIndex {
{
HFile.Writer writer =
HFile.getWriterFactory(conf, cacheConf).createWriter(fs,
hfilePath, SMALL_BLOCK_SIZE, compr, KeyValue.KEY_COMPARATOR);
hfilePath, SMALL_BLOCK_SIZE, compr, null, KeyValue.KEY_COMPARATOR);
Random rand = new Random(19231737);
for (int i = 0; i < NUM_KV; ++i) {

View File

@ -0,0 +1,188 @@
/*
* 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.assertEquals;
import static org.junit.Assert.assertTrue;
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.DataBlockEncoding;
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;
System.err.println("On-disk encoding: " + blockEncoder.getEncodingOnDisk()
+ ", in-cache encoding: " + blockEncoder.getEncodingInCache()
+ ", 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.diskToCacheFormat(block, false);
BlockCacheKey cacheKey = new BlockCacheKey("test", 0);
blockCache.cacheBlock(cacheKey, cacheBlock);
HeapSize heapSize = blockCache.getBlock(cacheKey, false);
assertTrue(heapSize instanceof HFileBlock);
HFileBlock returnedBlock = (HFileBlock) heapSize;;
if (blockEncoder.getEncodingInCache() ==
DataBlockEncoding.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<ByteBuffer, BlockType> 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.getEncodingOnDisk() !=
DataBlockEncoding.NONE) {
assertEquals(BlockType.ENCODED_DATA, blockOnDisk.getBlockType());
assertEquals(blockEncoder.getEncodingOnDisk().getId(),
blockOnDisk.getDataBlockEncodingId());
} else {
assertEquals(BlockType.DATA, blockOnDisk.getBlockType());
}
}
/**
* Test converting blocks from disk to cache format.
*/
@Test
public void testEncodingReadPath() {
HFileBlock origBlock = getSampleHFileBlock();
blockEncoder.diskToCacheFormat(origBlock, false);
}
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<Object[]> getAllConfigurations() {
List<Object[]> configurations =
new ArrayList<Object[]>();
for (DataBlockEncoding diskAlgo : DataBlockEncoding.values()) {
for (DataBlockEncoding cacheAlgo : DataBlockEncoding.values()) {
if (diskAlgo != cacheAlgo && diskAlgo != DataBlockEncoding.NONE) {
// We allow (1) the same encoding on disk and in cache, and
// (2) some encoding in cache but no encoding on disk (for testing).
continue;
}
for (boolean includesMemstoreTS : new boolean[] {false, true}) {
configurations.add(new Object[] {
new HFileDataBlockEncoderImpl(diskAlgo, cacheAlgo),
new Boolean(includesMemstoreTS)});
}
}
}
return configurations;
}
}

View File

@ -166,7 +166,7 @@ public class TestHFilePerformance extends TestCase {
minBlockSize, codecName, null);
// Writing value in one shot.
for (long l=0 ; l<rows ; l++ ) {
for (long l=0; l<rows; l++ ) {
generator.getKey(key);
generator.getValue(value);
writer.append(key, value);
@ -195,7 +195,7 @@ public class TestHFilePerformance extends TestCase {
BytesWritable keyBsw;
BytesWritable valBsw;
for (long l=0 ; l<rows ; l++ ) {
for (long l=0; l<rows; l++ ) {
generator.getKey(key);
keyBsw = new BytesWritable(key);
@ -241,7 +241,7 @@ public class TestHFilePerformance extends TestCase {
FSDataInputStream fin = fs.open(path);
if ("HFile".equals(fileType)){
HFile.Reader reader = HFile.createReader(path, fs.open(path),
HFile.Reader reader = HFile.createReaderFromStream(path, fs.open(path),
fs.getFileStatus(path).getLen(), new CacheConfig(conf));
reader.loadFileInfo();
switch (method) {
@ -252,7 +252,7 @@ public class TestHFilePerformance extends TestCase {
{
HFileScanner scanner = reader.getScanner(false, false);
scanner.seekTo();
for (long l=0 ; l<rows ; l++ ) {
for (long l=0; l<rows; l++ ) {
key = scanner.getKey();
val = scanner.getValue();
totalBytesRead += key.limit() + val.limit();
@ -275,7 +275,7 @@ public class TestHFilePerformance extends TestCase {
BytesWritable keyBsw = new BytesWritable();
BytesWritable valBsw = new BytesWritable();
for (long l=0 ; l<rows ; l++ ) {
for (long l=0; l<rows; l++ ) {
reader.next(keyBsw, valBsw);
totalBytesRead += keyBsw.getSize() + valBsw.getSize();
}

View File

@ -167,7 +167,7 @@ public class TestHFileSeek extends TestCase {
int miss = 0;
long totalBytes = 0;
FSDataInputStream fsdis = fs.open(path);
Reader reader = HFile.createReader(path, fsdis,
Reader reader = HFile.createReaderFromStream(path, fsdis,
fs.getFileStatus(path).getLen(), new CacheConfig(conf));
reader.loadFileInfo();
KeySampler kSampler =

View File

@ -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, KeyValue.KEY_COMPARATOR);
hfilePath, 4096, COMPRESS_ALGO, null, KeyValue.KEY_COMPARATOR);
long totalKeyLength = 0;
long totalValueLength = 0;
@ -125,10 +125,12 @@ public class TestHFileWriterV2 {
new HFileBlock.FSReaderV2(fsdis, COMPRESS_ALGO, fileSize);
// Comparator class name is stored in the trailer in version 2.
RawComparator<byte []> 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(),
@ -146,8 +148,10 @@ 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;

View File

@ -133,6 +133,8 @@ public class TestImportExport {
5, /* versions */
true /* keep deleted cells */,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
HColumnDescriptor.DEFAULT_IN_MEMORY,
HColumnDescriptor.DEFAULT_BLOCKCACHE,
HColumnDescriptor.DEFAULT_BLOCKSIZE,
@ -179,6 +181,8 @@ public class TestImportExport {
5, /* versions */
true /* keep deleted cells */,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_ENCODE_ON_DISK,
HColumnDescriptor.DEFAULT_DATA_BLOCK_ENCODING,
HColumnDescriptor.DEFAULT_IN_MEMORY,
HColumnDescriptor.DEFAULT_BLOCKCACHE,
HColumnDescriptor.DEFAULT_BLOCKSIZE,

View File

@ -184,8 +184,8 @@ public class CreateRandomStoreFile {
}
StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir, blockSize,
compr, KeyValue.COMPARATOR, conf, new CacheConfig(conf), bloomType,
numKV);
compr, null, KeyValue.COMPARATOR, conf, new CacheConfig(conf),
bloomType, numKV);
rand = new Random();
LOG.info("Writing " + numKV + " key/value pairs");

View File

@ -0,0 +1,585 @@
/*
* 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.DataBlockEncoding;
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<EncodedDataBlock> codecs = new ArrayList<EncodedDataBlock>();
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<DataBlockEncoder> dataBlockEncoders =
DataBlockEncoding.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<Iterator<KeyValue>> codecIterators =
new ArrayList<Iterator<KeyValue>>();
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<KeyValue> 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<Long> durations = new ArrayList<Long>();
for (int itTime = 0; itTime < BENCHMARK_N_TIMES; ++itTime) {
totalSize = 0;
Iterator<KeyValue> 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<Long> compressDurations = new ArrayList<Long>();
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<Long> compressDurations = new ArrayList<Long>();
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<Long> durations = new ArrayList<Long>();
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<Long> 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, NoOpDataBlockEncoder.INSTANCE);
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 <options>",
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);
}
}

View File

@ -0,0 +1,195 @@
/*
* 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.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.DataBlockEncoding;
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.io.hfile.NoOpDataBlockEncoder;
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<KeyValue> prepareListOfTestSeeks(Path path) throws IOException {
List<KeyValue> allKeyValues = new ArrayList<KeyValue>();
// read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE);
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<KeyValue> seeks = new ArrayList<KeyValue>();
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<KeyValue> seeks) throws IOException {
// read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE, blockEncoder);
long 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!
if (current.getLength() < 0) {
throw new IOException("Negative KV size: " + current);
}
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();
if (finishSeeksTime < startSeeksTime) {
throw new AssertionError("Finish time " + finishSeeksTime +
" is earlier than start time " + startSeeksTime);
}
// 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.printf(" Read speed: %8.2f (MB/s)\n", readInMbPerSec);
System.out.printf(" Seeks per second: %8.2f (#/s)\n", seeksPerSec);
System.out.printf(" Total KV size: %d\n", totalSize);
}
/**
* @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<HFileDataBlockEncoder> encoders)
throws IOException {
List<KeyValue> 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);
}
Path path = new Path(args[0]);
List<HFileDataBlockEncoder> encoders =
new ArrayList<HFileDataBlockEncoder>();
encoders.add(new HFileDataBlockEncoderImpl(DataBlockEncoding.NONE));
for (DataBlockEncoding encodingAlgo : DataBlockEncoding.values()) {
encoders.add(new HFileDataBlockEncoderImpl(DataBlockEncoding.NONE,
encodingAlgo));
}
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();
}
}

View File

@ -0,0 +1,846 @@
/*
* 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.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.SortedSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionGroup;
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.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.Compression;
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.HFilePrettyPrinter;
import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
import org.apache.hadoop.hbase.regionserver.StoreScanner.ScanType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.LoadTestTool;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.util.StringUtils;
/**
* Tests HFile read/write workloads, such as merging HFiles and random reads.
*/
public class HFileReadWriteTest {
private static final String TABLE_NAME = "MyTable";
private static enum Workload {
MERGE("merge", "Merge the specified HFiles", 1, Integer.MAX_VALUE),
RANDOM_READS("read", "Perform a random read benchmark on the given HFile",
1, 1);
private String option;
private String description;
public final int minNumInputFiles;
public final int maxNumInputFiles;
Workload(String option, String description, int minNumInputFiles,
int maxNumInputFiles) {
this.option = option;
this.description = description;
this.minNumInputFiles = minNumInputFiles;
this.maxNumInputFiles = maxNumInputFiles;
}
static OptionGroup getOptionGroup() {
OptionGroup optionGroup = new OptionGroup();
for (Workload w : values())
optionGroup.addOption(new Option(w.option, w.description));
return optionGroup;
}
private static String getOptionListStr() {
StringBuilder sb = new StringBuilder();
for (Workload w : values()) {
if (sb.length() > 0)
sb.append(", ");
sb.append("-" + w.option);
}
return sb.toString();
}
static Workload fromCmdLine(CommandLine cmdLine) {
for (Workload w : values()) {
if (cmdLine.hasOption(w.option))
return w;
}
LOG.error("No workload specified. Specify one of the options: " +
getOptionListStr());
return null;
}
public String onlyUsedFor() {
return ". Only used for the " + this + " workload.";
}
}
private static final String OUTPUT_DIR_OPTION = "output_dir";
private static final String COMPRESSION_OPTION = "compression";
private static final String BLOOM_FILTER_OPTION = "bloom";
private static final String BLOCK_SIZE_OPTION = "block_size";
private static final String DURATION_OPTION = "duration";
private static final String NUM_THREADS_OPTION = "num_threads";
private static final Log LOG = LogFactory.getLog(HFileReadWriteTest.class);
private Workload workload;
private FileSystem fs;
private Configuration conf;
private CacheConfig cacheConf;
private List<String> inputFileNames;
private Path outputDir;
private int numReadThreads;
private int durationSec;
private DataBlockEncoding dataBlockEncoding;
private boolean encodeInCacheOnly;
private HFileDataBlockEncoder dataBlockEncoder =
NoOpDataBlockEncoder.INSTANCE;
private StoreFile.BloomType bloomType = StoreFile.BloomType.NONE;
private int blockSize;
private Compression.Algorithm compression = Compression.Algorithm.NONE;
private byte[] firstRow, lastRow;
private AtomicLong numSeeks = new AtomicLong();
private AtomicLong numKV = new AtomicLong();
private AtomicLong totalBytes = new AtomicLong();
private byte[] family;
private long endTime = Long.MAX_VALUE;
private SortedSet<String> keysRead = new ConcurrentSkipListSet<String>();
private List<StoreFile> inputStoreFiles;
public HFileReadWriteTest() {
conf = HBaseConfiguration.create();
cacheConf = new CacheConfig(conf);
}
@SuppressWarnings("unchecked")
public boolean parseOptions(String args[]) {
Options options = new Options();
options.addOption(OUTPUT_DIR_OPTION, true, "Output directory" +
Workload.MERGE.onlyUsedFor());
options.addOption(COMPRESSION_OPTION, true, " Compression type, one of "
+ Arrays.toString(Compression.Algorithm.values()) +
Workload.MERGE.onlyUsedFor());
options.addOption(BLOOM_FILTER_OPTION, true, "Bloom filter type, one of "
+ Arrays.toString(StoreFile.BloomType.values()) +
Workload.MERGE.onlyUsedFor());
options.addOption(BLOCK_SIZE_OPTION, true, "HFile block size" +
Workload.MERGE.onlyUsedFor());
options.addOption(DURATION_OPTION, true, "The amount of time to run the " +
"random read workload for" + Workload.RANDOM_READS.onlyUsedFor());
options.addOption(NUM_THREADS_OPTION, true, "The number of random " +
"reader threads" + Workload.RANDOM_READS.onlyUsedFor());
options.addOption(NUM_THREADS_OPTION, true, "The number of random " +
"reader threads" + Workload.RANDOM_READS.onlyUsedFor());
options.addOption(LoadTestTool.OPT_DATA_BLOCK_ENCODING, true,
LoadTestTool.OPT_DATA_BLOCK_ENCODING_USAGE);
options.addOption(LoadTestTool.OPT_ENCODE_IN_CACHE_ONLY, false,
LoadTestTool.OPT_ENCODE_IN_CACHE_ONLY_USAGE);
options.addOptionGroup(Workload.getOptionGroup());
if (args.length == 0) {
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp(HFileReadWriteTest.class.getSimpleName(),
options, true);
return false;
}
CommandLineParser parser = new PosixParser();
CommandLine cmdLine;
try {
cmdLine = parser.parse(options, args);
} catch (ParseException ex) {
LOG.error(ex);
return false;
}
workload = Workload.fromCmdLine(cmdLine);
if (workload == null)
return false;
inputFileNames = (List<String>) cmdLine.getArgList();
if (inputFileNames.size() == 0) {
LOG.error("No input file names specified");
return false;
}
if (inputFileNames.size() < workload.minNumInputFiles) {
LOG.error("Too few input files: at least " + workload.minNumInputFiles +
" required");
return false;
}
if (inputFileNames.size() > workload.maxNumInputFiles) {
LOG.error("Too many input files: at most " + workload.minNumInputFiles +
" allowed");
return false;
}
if (cmdLine.hasOption(COMPRESSION_OPTION)) {
compression = Compression.Algorithm.valueOf(
cmdLine.getOptionValue(COMPRESSION_OPTION));
}
if (cmdLine.hasOption(BLOOM_FILTER_OPTION)) {
bloomType = StoreFile.BloomType.valueOf(cmdLine.getOptionValue(
BLOOM_FILTER_OPTION));
}
encodeInCacheOnly =
cmdLine.hasOption(LoadTestTool.OPT_ENCODE_IN_CACHE_ONLY);
if (cmdLine.hasOption(LoadTestTool.OPT_DATA_BLOCK_ENCODING)) {
dataBlockEncoding = DataBlockEncoding.valueOf(
cmdLine.getOptionValue(LoadTestTool.OPT_DATA_BLOCK_ENCODING));
// Optionally encode on disk, always encode in cache.
dataBlockEncoder = new HFileDataBlockEncoderImpl(
encodeInCacheOnly ? DataBlockEncoding.NONE : dataBlockEncoding,
dataBlockEncoding);
} else {
if (encodeInCacheOnly) {
LOG.error("The -" + LoadTestTool.OPT_ENCODE_IN_CACHE_ONLY +
" option does not make sense without -" +
LoadTestTool.OPT_DATA_BLOCK_ENCODING);
return false;
}
}
blockSize = conf.getInt("hfile.min.blocksize.size", 65536);
if (cmdLine.hasOption(BLOCK_SIZE_OPTION))
blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION));
if (workload == Workload.MERGE) {
String outputDirStr = cmdLine.getOptionValue(OUTPUT_DIR_OPTION);
if (outputDirStr == null) {
LOG.error("Output directory is not specified");
return false;
}
outputDir = new Path(outputDirStr);
// Will be checked for existence in validateConfiguration.
}
if (workload == Workload.RANDOM_READS) {
if (!requireOptions(cmdLine, new String[] { DURATION_OPTION,
NUM_THREADS_OPTION })) {
return false;
}
durationSec = Integer.parseInt(cmdLine.getOptionValue(DURATION_OPTION));
numReadThreads = Integer.parseInt(
cmdLine.getOptionValue(NUM_THREADS_OPTION));
}
Collections.sort(inputFileNames);
return true;
}
/** @return true if all the given options are specified */
private boolean requireOptions(CommandLine cmdLine,
String[] requiredOptions) {
for (String option : requiredOptions)
if (!cmdLine.hasOption(option)) {
LOG.error("Required option -" + option + " not specified");
return false;
}
return true;
}
public boolean validateConfiguration() throws IOException {
fs = FileSystem.get(conf);
for (String inputFileName : inputFileNames) {
Path path = new Path(inputFileName);
if (!fs.exists(path)) {
LOG.error("File " + inputFileName + " does not exist");
return false;
}
if (fs.getFileStatus(path).isDir()) {
LOG.error(inputFileName + " is a directory");
return false;
}
}
if (outputDir != null &&
(!fs.exists(outputDir) || !fs.getFileStatus(outputDir).isDir())) {
LOG.error(outputDir.toString() + " does not exist or is not a " +
"directory");
return false;
}
return true;
}
public void runMergeWorkload() throws IOException {
long maxKeyCount = prepareForMerge();
List<StoreFileScanner> scanners =
StoreFileScanner.getScannersForStoreFiles(inputStoreFiles, false,
false);
HColumnDescriptor columnDescriptor = new HColumnDescriptor(
HFileReadWriteTest.class.getSimpleName());
columnDescriptor.setBlocksize(blockSize);
columnDescriptor.setBloomFilterType(bloomType);
columnDescriptor.setCompressionType(compression);
columnDescriptor.setDataBlockEncoding(dataBlockEncoding);
HRegionInfo regionInfo = new HRegionInfo();
HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
HRegion region = new HRegion(outputDir, null, fs, conf, regionInfo, htd,
null);
Store store = new Store(outputDir, region, columnDescriptor, fs, conf);
StoreFile.Writer writer =
StoreFile.createWriter(fs, outputDir, blockSize, compression,
dataBlockEncoder, KeyValue.COMPARATOR, this.conf,
new CacheConfig(conf), bloomType,
maxKeyCount);
StatisticsPrinter statsPrinter = new StatisticsPrinter();
statsPrinter.startThread();
try {
performMerge(scanners, store, writer);
writer.close();
} finally {
statsPrinter.requestStop();
}
Path resultPath = writer.getPath();
resultPath = tryUsingSimpleOutputPath(resultPath);
long fileSize = fs.getFileStatus(resultPath).getLen();
LOG.info("Created " + resultPath + ", size " + fileSize);
System.out.println();
System.out.println("HFile information for " + resultPath);
System.out.println();
HFilePrettyPrinter hfpp = new HFilePrettyPrinter();
hfpp.run(new String[] { "-m", "-f", resultPath.toString() });
}
private Path tryUsingSimpleOutputPath(Path resultPath) throws IOException {
if (inputFileNames.size() == 1) {
// In case of only one input set output to be consistent with the
// input name.
Path inputPath = new Path(inputFileNames.get(0));
Path betterOutputPath = new Path(outputDir,
inputPath.getName());
if (!fs.exists(betterOutputPath)) {
fs.rename(resultPath, betterOutputPath);
resultPath = betterOutputPath;
}
}
return resultPath;
}
private void performMerge(List<StoreFileScanner> scanners, Store store,
StoreFile.Writer writer) throws IOException {
InternalScanner scanner = null;
try {
Scan scan = new Scan();
// Include deletes
scanner = new StoreScanner(store, scan, scanners,
ScanType.MAJOR_COMPACT, Long.MIN_VALUE, Long.MIN_VALUE);
ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
while (scanner.next(kvs) || kvs.size() != 0) {
numKV.addAndGet(kvs.size());
for (KeyValue kv : kvs) {
totalBytes.addAndGet(kv.getLength());
writer.append(kv);
}
kvs.clear();
}
} finally {
if (scanner != null)
scanner.close();
}
}
/**
* @return the total key count in the files being merged
* @throws IOException
*/
private long prepareForMerge() throws IOException {
LOG.info("Merging " + inputFileNames);
LOG.info("Using block size: " + blockSize);
inputStoreFiles = new ArrayList<StoreFile>();
long maxKeyCount = 0;
for (String fileName : inputFileNames) {
Path filePath = new Path(fileName);
// Open without caching.
StoreFile sf = openStoreFile(filePath, false);
sf.createReader();
inputStoreFiles.add(sf);
StoreFile.Reader r = sf.getReader();
if (r != null) {
long keyCount = r.getFilterEntries();
maxKeyCount += keyCount;
LOG.info("Compacting: " + sf + "; keyCount = " + keyCount
+ "; Bloom Type = " + r.getBloomFilterType().toString()
+ "; Size = " + StringUtils.humanReadableInt(r.length()));
}
}
return maxKeyCount;
}
public HFile.Reader[] getHFileReaders() {
HFile.Reader readers[] = new HFile.Reader[inputStoreFiles.size()];
for (int i = 0; i < inputStoreFiles.size(); ++i)
readers[i] = inputStoreFiles.get(i).getReader().getHFileReader();
return readers;
}
private StoreFile openStoreFile(Path filePath, boolean blockCache)
throws IOException {
// We are passing the ROWCOL Bloom filter type, but StoreFile will still
// use the Bloom filter type specified in the HFile.
return new StoreFile(fs, filePath, conf, cacheConf,
StoreFile.BloomType.ROWCOL, dataBlockEncoder);
}
public static int charToHex(int c) {
if ('0' <= c && c <= '9')
return c - '0';
if ('a' <= c && c <= 'f')
return 10 + c - 'a';
return -1;
}
public static int hexToChar(int h) {
h &= 0xff;
if (0 <= h && h <= 9)
return '0' + h;
if (10 <= h && h <= 15)
return 'a' + h - 10;
return -1;
}
public static byte[] createRandomRow(Random rand, byte[] first, byte[] last)
{
int resultLen = Math.max(first.length, last.length);
int minLen = Math.min(first.length, last.length);
byte[] result = new byte[resultLen];
boolean greaterThanFirst = false;
boolean lessThanLast = false;
for (int i = 0; i < resultLen; ++i) {
// Generate random hex characters if both first and last row are hex
// at this position.
boolean isHex = i < minLen && charToHex(first[i]) != -1
&& charToHex(last[i]) != -1;
// If our key is already greater than the first key, we can use
// arbitrarily low values.
int low = greaterThanFirst || i >= first.length ? 0 : first[i] & 0xff;
// If our key is already less than the last key, we can use arbitrarily
// high values.
int high = lessThanLast || i >= last.length ? 0xff : last[i] & 0xff;
// Randomly select the next byte between the lowest and the highest
// value allowed for this position. Restrict to hex characters if
// necessary. We are generally biased towards border cases, which is OK
// for test.
int r;
if (isHex) {
// Use hex chars.
if (low < '0')
low = '0';
if (high > 'f')
high = 'f';
int lowHex = charToHex(low);
int highHex = charToHex(high);
r = hexToChar(lowHex + rand.nextInt(highHex - lowHex + 1));
} else {
r = low + rand.nextInt(high - low + 1);
}
if (r > low)
greaterThanFirst = true;
if (r < high)
lessThanLast = true;
result[i] = (byte) r;
}
if (Bytes.compareTo(result, first) < 0) {
throw new IllegalStateException("Generated key " +
Bytes.toStringBinary(result) + " is less than the first key " +
Bytes.toStringBinary(first));
}
if (Bytes.compareTo(result, last) > 0) {
throw new IllegalStateException("Generated key " +
Bytes.toStringBinary(result) + " is greater than te last key " +
Bytes.toStringBinary(last));
}
return result;
}
private static byte[] createRandomQualifier(Random rand) {
byte[] q = new byte[10 + rand.nextInt(30)];
rand.nextBytes(q);
return q;
}
private class RandomReader implements Callable<Boolean> {
private int readerId;
private StoreFile.Reader reader;
private boolean pread;
public RandomReader(int readerId, StoreFile.Reader reader,
boolean pread)
{
this.readerId = readerId;
this.reader = reader;
this.pread = pread;
}
@Override
public Boolean call() throws Exception {
Thread.currentThread().setName("reader " + readerId);
Random rand = new Random();
StoreFileScanner scanner = reader.getStoreFileScanner(true, pread);
while (System.currentTimeMillis() < endTime) {
byte[] row = createRandomRow(rand, firstRow, lastRow);
KeyValue kvToSeek = new KeyValue(row, family,
createRandomQualifier(rand));
if (rand.nextDouble() < 0.0001) {
LOG.info("kvToSeek=" + kvToSeek);
}
boolean seekResult;
try {
seekResult = scanner.seek(kvToSeek);
} catch (IOException ex) {
throw new IOException("Seek failed for key " + kvToSeek + ", pread="
+ pread, ex);
}
numSeeks.incrementAndGet();
if (!seekResult) {
error("Seek returned false for row " + Bytes.toStringBinary(row));
return false;
}
for (int i = 0; i < rand.nextInt(10) + 1; ++i) {
KeyValue kv = scanner.next();
numKV.incrementAndGet();
if (i == 0 && kv == null) {
error("scanner.next() returned null at the first iteration for " +
"row " + Bytes.toStringBinary(row));
return false;
}
if (kv == null)
break;
String keyHashStr = MD5Hash.getMD5AsHex(kv.getKey());
keysRead.add(keyHashStr);
totalBytes.addAndGet(kv.getLength());
}
}
return true;
}
private void error(String msg) {
LOG.error("error in reader " + readerId + " (pread=" + pread + "): "
+ msg);
}
}
private class StatisticsPrinter implements Callable<Boolean> {
private volatile boolean stopRequested;
private volatile Thread thread;
private long totalSeekAndReads, totalPositionalReads;
/**
* Run the statistics collector in a separate thread without an executor.
*/
public void startThread() {
new Thread() {
@Override
public void run() {
try {
call();
} catch (Exception e) {
LOG.error(e);
}
}
}.start();
}
@Override
public Boolean call() throws Exception {
LOG.info("Starting statistics printer");
thread = Thread.currentThread();
thread.setName(StatisticsPrinter.class.getSimpleName());
long startTime = System.currentTimeMillis();
long curTime;
while ((curTime = System.currentTimeMillis()) < endTime &&
!stopRequested) {
long elapsedTime = curTime - startTime;
printStats(elapsedTime);
try {
Thread.sleep(1000 - elapsedTime % 1000);
} catch (InterruptedException iex) {
Thread.currentThread().interrupt();
if (stopRequested)
break;
}
}
printStats(curTime - startTime);
LOG.info("Stopping statistics printer");
return true;
}
private void printStats(long elapsedTime) {
long numSeeksL = numSeeks.get();
double timeSec = elapsedTime / 1000.0;
double seekPerSec = numSeeksL / timeSec;
long kvCount = numKV.get();
double kvPerSec = kvCount / timeSec;
long bytes = totalBytes.get();
double bytesPerSec = bytes / timeSec;
// readOps and preadOps counters get reset on access, so we have to
// accumulate them here. HRegion metrics publishing thread should not
// be running in this tool, so no one else should be resetting these
// metrics.
totalSeekAndReads += HFile.getReadOps();
totalPositionalReads += HFile.getPreadOps();
long totalBlocksRead = totalSeekAndReads + totalPositionalReads;
double blkReadPerSec = totalBlocksRead / timeSec;
double seekReadPerSec = totalSeekAndReads / timeSec;
double preadPerSec = totalPositionalReads / timeSec;
boolean isRead = workload == Workload.RANDOM_READS;
StringBuilder sb = new StringBuilder();
sb.append("Time: " + (long) timeSec + " sec");
if (isRead)
sb.append(", seek/sec: " + (long) seekPerSec);
sb.append(", kv/sec: " + (long) kvPerSec);
sb.append(", bytes/sec: " + (long) bytesPerSec);
sb.append(", blk/sec: " + (long) blkReadPerSec);
sb.append(", total KV: " + numKV);
sb.append(", total bytes: " + totalBytes);
sb.append(", total blk: " + totalBlocksRead);
sb.append(", seekRead/sec: " + (long) seekReadPerSec);
sb.append(", pread/sec: " + (long) preadPerSec);
if (isRead)
sb.append(", unique keys: " + (long) keysRead.size());
LOG.info(sb.toString());
}
public void requestStop() {
stopRequested = true;
if (thread != null)
thread.interrupt();
}
}
public boolean runRandomReadWorkload() throws IOException {
if (inputFileNames.size() != 1) {
throw new IOException("Need exactly one input file for random reads: " +
inputFileNames);
}
Path inputPath = new Path(inputFileNames.get(0));
// Make sure we are using caching.
StoreFile storeFile = openStoreFile(inputPath, true);
StoreFile.Reader reader = storeFile.createReader();
LOG.info("First key: " + Bytes.toStringBinary(reader.getFirstKey()));
LOG.info("Last key: " + Bytes.toStringBinary(reader.getLastKey()));
KeyValue firstKV = KeyValue.createKeyValueFromKey(reader.getFirstKey());
firstRow = firstKV.getRow();
KeyValue lastKV = KeyValue.createKeyValueFromKey(reader.getLastKey());
lastRow = lastKV.getRow();
byte[] family = firstKV.getFamily();
if (!Bytes.equals(family, lastKV.getFamily())) {
LOG.error("First and last key have different families: "
+ Bytes.toStringBinary(family) + " and "
+ Bytes.toStringBinary(lastKV.getFamily()));
return false;
}
if (Bytes.equals(firstRow, lastRow)) {
LOG.error("First and last row are the same, cannot run read workload: " +
"firstRow=" + Bytes.toStringBinary(firstRow) + ", " +
"lastRow=" + Bytes.toStringBinary(lastRow));
return false;
}
ExecutorService exec = Executors.newFixedThreadPool(numReadThreads + 1);
int numCompleted = 0;
int numFailed = 0;
try {
ExecutorCompletionService<Boolean> ecs =
new ExecutorCompletionService<Boolean>(exec);
endTime = System.currentTimeMillis() + 1000 * durationSec;
boolean pread = true;
for (int i = 0; i < numReadThreads; ++i)
ecs.submit(new RandomReader(i, reader, pread));
ecs.submit(new StatisticsPrinter());
Future<Boolean> result;
while (true) {
try {
result = ecs.poll(endTime + 1000 - System.currentTimeMillis(),
TimeUnit.MILLISECONDS);
if (result == null)
break;
try {
if (result.get()) {
++numCompleted;
} else {
++numFailed;
}
} catch (ExecutionException e) {
LOG.error("Worker thread failure", e.getCause());
++numFailed;
}
} catch (InterruptedException ex) {
LOG.error("Interrupted after " + numCompleted +
" workers completed");
Thread.currentThread().interrupt();
continue;
}
}
} finally {
storeFile.closeReader(true);
exec.shutdown();
BlockCache c = cacheConf.getBlockCache();
if (c != null) {
c.shutdown();
}
}
LOG.info("Worker threads completed: " + numCompleted);
LOG.info("Worker threads failed: " + numFailed);
return true;
}
public boolean run() throws IOException {
LOG.info("Workload: " + workload);
switch (workload) {
case MERGE:
runMergeWorkload();
break;
case RANDOM_READS:
return runRandomReadWorkload();
default:
LOG.error("Unknown workload: " + workload);
return false;
}
return true;
}
private static void failure() {
System.exit(1);
}
public static void main(String[] args) {
HFileReadWriteTest app = new HFileReadWriteTest();
if (!app.parseOptions(args))
failure();
try {
if (!app.validateConfiguration() ||
!app.run())
failure();
} catch (IOException ex) {
LOG.error(ex);
failure();
}
}
}

View File

@ -35,6 +35,7 @@ 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.hfile.NoOpDataBlockEncoder;
import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes;
@ -67,7 +68,7 @@ public class TestCompactSelection extends TestCase {
this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
this.conf.setInt("hbase.hstore.compaction.min", minFiles);
this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
this.conf.setLong("hbase.hregion.memstore.flush.size", minSize);
this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
@ -102,7 +103,8 @@ 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);
new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE);
this.length = length;
this.isRef = isRef;
}

View File

@ -24,8 +24,10 @@ 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;
@ -33,20 +35,24 @@ 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.*;
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.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.DataBlockEncoding;
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;
@ -61,6 +67,7 @@ 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;
@ -70,13 +77,12 @@ public class TestCompaction extends HBaseTestCase {
private byte[] firstRowBytes, secondRowBytes, thirdRowBytes;
final private byte[] col1, col2;
/** constructor */
public TestCompaction() throws Exception {
super();
// Set cache flush size to 1MB
conf.setInt("hbase.hregion.memstore.flush.size", 1024*1024);
conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024*1024);
conf.setInt("hbase.hregion.memstore.block.multiplier", 100);
compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
@ -94,7 +100,7 @@ public class TestCompaction extends HBaseTestCase {
@Override
public void setUp() throws Exception {
super.setUp();
HTableDescriptor htd = createTableDescriptor(getName());
this.htd = createTableDescriptor(getName());
this.r = createNewHRegion(htd, null, null);
}
@ -146,6 +152,42 @@ public class TestCompaction extends HBaseTestCase {
* @throws Exception
*/
public void testMajorCompaction() throws Exception {
majorCompaction();
}
public void testDataBlockEncodingInCacheOnly() throws Exception {
majorCompactionWithDataBlockEncoding(true);
}
public void testDataBlockEncodingEverywhere() throws Exception {
majorCompactionWithDataBlockEncoding(false);
}
public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly)
throws Exception {
Map<Store, HFileDataBlockEncoder> replaceBlockCache =
new HashMap<Store, HFileDataBlockEncoder>();
for (Entry<byte[], Store> pair : r.getStores().entrySet()) {
Store store = pair.getValue();
HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder();
replaceBlockCache.put(pair.getValue(), blockEncoder);
final DataBlockEncoding inCache = DataBlockEncoding.PREFIX;
final DataBlockEncoding onDisk = inCacheOnly ? DataBlockEncoding.NONE :
inCache;
store.setDataBlockEncoderInTest(new HFileDataBlockEncoderImpl(
onDisk, inCache));
}
majorCompaction();
// restore settings
for (Entry<Store, HFileDataBlockEncoder> entry :
replaceBlockCache.entrySet()) {
entry.getKey().setDataBlockEncoderInTest(entry.getValue());
}
}
private void majorCompaction() throws Exception {
createStoreFile(r);
for (int i = 0; i < compactionThreshold; i++) {
createStoreFile(r);
@ -174,10 +216,10 @@ public class TestCompaction extends HBaseTestCase {
CompactionProgress progress = store.getCompactionProgress();
if( progress != null ) {
++storeCount;
assert(progress.currentCompactedKVs > 0);
assert(progress.totalCompactingKVs > 0);
assertTrue(progress.currentCompactedKVs > 0);
assertTrue(progress.totalCompactingKVs > 0);
}
assert(storeCount > 0);
assertTrue(storeCount > 0);
}
// look at the second row
@ -186,14 +228,20 @@ 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);
assertEquals(compactionThreshold, result.size());
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());
// 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]);

View File

@ -41,6 +41,7 @@ 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;
@ -193,7 +194,8 @@ public class TestCompoundBloomFilter {
private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
Path sfPath) throws IOException {
StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt);
StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt,
NoOpDataBlockEncoder.INSTANCE);
StoreFile.Reader r = sf.createReader();
final boolean pread = true; // does not really matter
StoreFileScanner scanner = r.getStoreFileScanner(true, pread);
@ -293,7 +295,7 @@ public class TestCompoundBloomFilter {
cacheConf = new CacheConfig(conf);
StoreFile.Writer w = StoreFile.createWriter(fs,
TEST_UTIL.getDataTestDir(), BLOCK_SIZES[t], null, null, conf,
TEST_UTIL.getDataTestDir(), BLOCK_SIZES[t], null, null, null, conf,
cacheConf, bt, 0);
assertTrue(w.hasGeneralBloom());

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
@ -76,7 +77,9 @@ public class TestFSErrorsExposed {
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
StoreFile sf = new StoreFile(fs, writer.getPath(),
util.getConfiguration(), cacheConf, StoreFile.BloomType.NONE);
util.getConfiguration(), cacheConf, StoreFile.BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE);
StoreFile.Reader reader = sf.createReader();
HFileScanner scanner = reader.getScanner(false, true);
@ -119,7 +122,8 @@ public class TestFSErrorsExposed {
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(),
cacheConf, BloomType.NONE);
cacheConf, BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
Collections.singletonList(sf), false, true, false);
KeyValueScanner scanner = scanners.get(0);

View File

@ -35,16 +35,17 @@ import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
@ -107,8 +108,9 @@ public class TestMultiColumnScanner {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Compression.Algorithm comprAlgo;
private StoreFile.BloomType bloomType;
private final Compression.Algorithm comprAlgo;
private final StoreFile.BloomType bloomType;
private final DataBlockEncoding dataBlockEncoding;
// Some static sanity-checking.
static {
@ -127,20 +129,30 @@ public class TestMultiColumnScanner {
@Parameters
public static final Collection<Object[]> parameters() {
return HBaseTestingUtility.BLOOM_AND_COMPRESSION_COMBINATIONS;
List<Object[]> parameters = new ArrayList<Object[]>();
for (Object[] bloomAndCompressionParams :
HBaseTestingUtility.BLOOM_AND_COMPRESSION_COMBINATIONS) {
for (boolean useDataBlockEncoding : new boolean[]{false, true}) {
parameters.add(ArrayUtils.add(bloomAndCompressionParams,
useDataBlockEncoding));
}
}
return parameters;
}
public TestMultiColumnScanner(Compression.Algorithm comprAlgo,
StoreFile.BloomType bloomType) {
StoreFile.BloomType bloomType, boolean useDataBlockEncoding) {
this.comprAlgo = comprAlgo;
this.bloomType = bloomType;
this.dataBlockEncoding = useDataBlockEncoding ? DataBlockEncoding.PREFIX :
DataBlockEncoding.NONE;
}
@Test
public void testMultiColumnScanner() throws IOException {
HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME, FAMILY, comprAlgo,
bloomType, MAX_VERSIONS, HColumnDescriptor.DEFAULT_BLOCKCACHE,
HFile.DEFAULT_BLOCKSIZE);
bloomType, MAX_VERSIONS, HFile.DEFAULT_BLOCKSIZE,
dataBlockEncoding, true);
List<String> rows = sequentialStrings("row", NUM_ROWS);
List<String> qualifiers = sequentialStrings("qual", NUM_COLUMNS);
List<KeyValue> kvs = new ArrayList<KeyValue>();

View File

@ -35,12 +35,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter;
@ -102,10 +102,10 @@ public class TestScanWithBloomError {
@Test
public void testThreeStoreFiles() throws IOException {
region = TEST_UTIL.createTestRegion(TABLE_NAME,
FAMILY, Compression.Algorithm.GZ, bloomType,
TestMultiColumnScanner.MAX_VERSIONS,
HColumnDescriptor.DEFAULT_BLOCKCACHE, HFile.DEFAULT_BLOCKSIZE);
region = TEST_UTIL.createTestRegion(TABLE_NAME, FAMILY,
Compression.Algorithm.GZ, bloomType,
TestMultiColumnScanner.MAX_VERSIONS, HFile.DEFAULT_BLOCKSIZE,
DataBlockEncoding.NONE, true);
createStoreFile(new int[] {1, 2, 6});
createStoreFile(new int[] {1, 2, 3, 7});
createStoreFile(new int[] {1, 9});

View File

@ -37,13 +37,13 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.Bytes;
@ -141,10 +141,9 @@ public class TestSeekOptimizations {
@Test
public void testMultipleTimestampRanges() throws IOException {
region = TEST_UTIL.createTestRegion(
TestSeekOptimizations.class.getName(), FAMILY, comprAlgo, bloomType,
Integer.MAX_VALUE, HColumnDescriptor.DEFAULT_BLOCKCACHE,
HFile.DEFAULT_BLOCKSIZE);
region = TEST_UTIL.createTestRegion(TestSeekOptimizations.class.getName(),
FAMILY, comprAlgo, bloomType, Integer.MAX_VALUE,
HFile.DEFAULT_BLOCKSIZE, DataBlockEncoding.NONE, true);
// Delete the given timestamp and everything before.
final long latestDelTS = USE_MANY_STORE_FILES ? 1397 : -1;

View File

@ -34,18 +34,24 @@ 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.*;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.Reference.Range;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
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.io.hfile.NoOpDataBlockEncoder;
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;
@ -88,7 +94,7 @@ public class TestStoreFile extends HBaseTestCase {
conf, cacheConf);
writeStoreFile(writer);
checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
StoreFile.BloomType.NONE));
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE));
}
private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
@ -129,7 +135,7 @@ public class TestStoreFile extends HBaseTestCase {
conf, cacheConf);
writeStoreFile(writer);
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
StoreFile.BloomType.NONE);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
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
@ -141,7 +147,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
// 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);
@ -177,10 +183,12 @@ 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,
StoreFile.BloomType.NONE).createReader();
StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
StoreFile.BloomType.NONE).createReader();
StoreFile.Reader top =
new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath,
conf, cacheConf, BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
ByteBuffer previous = null;
LOG.info("Midkey: " + midKV.toString());
ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey);
@ -236,9 +244,11 @@ 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).createReader();
StoreFile.BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
StoreFile.BloomType.NONE).createReader();
StoreFile.BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
bottomScanner = bottom.getScanner(false, false);
int count = 0;
while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
@ -281,9 +291,11 @@ 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).createReader();
StoreFile.BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
StoreFile.BloomType.NONE).createReader();
StoreFile.BloomType.NONE,
NoOpDataBlockEncoder.INSTANCE).createReader();
first = true;
bottomScanner = bottom.getScanner(false, false);
while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
@ -339,7 +351,8 @@ public class TestStoreFile extends HBaseTestCase {
}
writer.close();
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf,
DataBlockEncoding.NONE);
reader.loadFileInfo();
reader.loadBloomfilter();
StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
@ -378,10 +391,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,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
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);
bloomWriteRead(writer, fs);
}
@ -398,7 +411,8 @@ public class TestStoreFile extends HBaseTestCase {
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE,
2000);
// add delete family
long now = System.currentTimeMillis();
@ -410,7 +424,8 @@ public class TestStoreFile extends HBaseTestCase {
}
writer.close();
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf,
DataBlockEncoding.NONE);
reader.loadFileInfo();
reader.loadBloomfilter();
@ -465,7 +480,7 @@ public class TestStoreFile extends HBaseTestCase {
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL,
HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
null, conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
long now = System.currentTimeMillis();
for (int i = 0; i < rowCount*2; i += 2) { // rows
@ -482,7 +497,8 @@ public class TestStoreFile extends HBaseTestCase {
}
writer.close();
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf,
DataBlockEncoding.NONE);
reader.loadFileInfo();
reader.loadBloomfilter();
StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
@ -536,7 +552,8 @@ 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,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
2000);
assertFalse(writer.hasGeneralBloom());
writer.close();
fs.delete(f, true);
@ -559,7 +576,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,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
null, conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
Integer.MAX_VALUE);
assertFalse(writer.hasGeneralBloom());
writer.close();
@ -664,7 +681,7 @@ public class TestStoreFile extends HBaseTestCase {
writer.close();
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
StoreFile.BloomType.NONE);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
StoreFile.Reader reader = hsf.createReader();
StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
TreeSet<byte[]> columns = new TreeSet<byte[]>();
@ -707,7 +724,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
LOG.debug(hsf.getPath().toString());
// Read this file, we should see 3 misses
@ -729,7 +746,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
// Read this file, we should see 3 hits
reader = hsf.createReader();
@ -745,13 +762,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
StoreFile.Reader readerTwo = hsf.createReader();
readerTwo.loadFileInfo();
StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true);
@ -782,7 +799,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
reader = hsf.createReader();
reader.close(cacheConf.shouldEvictOnClose());
@ -796,7 +813,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);
StoreFile.BloomType.NONE, NoOpDataBlockEncoder.INSTANCE);
reader = hsf.createReader();
reader.close(cacheConf.shouldEvictOnClose());
@ -823,7 +840,8 @@ public class TestStoreFile extends HBaseTestCase {
int blockSize = totalSize / numBlocks;
StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize,
HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
null, 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) {
@ -834,6 +852,42 @@ 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");
DataBlockEncoding dataBlockEncoderAlgo =
DataBlockEncoding.FAST_DIFF;
HFileDataBlockEncoder dataBlockEncoder =
new HFileDataBlockEncoderImpl(
dataBlockEncoderAlgo,
dataBlockEncoderAlgo);
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<byte[], byte[]> 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();

View File

@ -111,7 +111,8 @@ public class TestLogRolling {
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
// We flush the cache after every 8192 bytes
TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.flush.size", 8192);
TEST_UTIL.getConfiguration().setInt(
HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
// Increase the amount of time between client retries
TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);

View File

@ -481,7 +481,7 @@ public class TestWALReplay {
runWALSplit(newConf);
FileSystem newFS = FileSystem.get(newConf);
// 100k seems to make for about 4 flushes during HRegion#initialize.
newConf.setInt("hbase.hregion.memstore.flush.size", 1024 * 100);
newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100);
// Make a new wal for new region.
HLog newWal = createWAL(newConf);
final AtomicInteger flushcount = new AtomicInteger(0);

View File

@ -68,7 +68,7 @@ public class LoadTestKVGenerator {
String md5hash = MD5Hash.getMD5AsHex(Bytes.toBytes(stringKey));
// flip the key to randomize
return md5hash + ":" + stringKey;
return md5hash + "-" + stringKey;
}
/**

View File

@ -28,6 +28,7 @@ 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.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -70,8 +71,21 @@ public class LoadTestTool extends AbstractHBaseTool {
private static final String OPT_USAGE_COMPRESSION = "Compression type, " +
"one of " + Arrays.toString(Compression.Algorithm.values());
public static final String OPT_DATA_BLOCK_ENCODING_USAGE =
"Encoding algorithm (e.g. prefix "
+ "compression) to use for data blocks in the test column family, "
+ "one of " + Arrays.toString(DataBlockEncoding.values()) + ".";
private static final String OPT_BLOOM = "bloom";
private static final String OPT_COMPRESSION = "compression";
public static final String OPT_DATA_BLOCK_ENCODING =
HColumnDescriptor.DATA_BLOCK_ENCODING.toLowerCase();
public static final String OPT_ENCODE_IN_CACHE_ONLY =
"encode_in_cache_only";
public static final String OPT_ENCODE_IN_CACHE_ONLY_USAGE =
"If this is specified, data blocks will only be encoded in block " +
"cache but not on disk";
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";
@ -82,6 +96,8 @@ 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;
@ -92,6 +108,12 @@ public class LoadTestTool extends AbstractHBaseTool {
private boolean isWrite, isRead;
// Column family options
private DataBlockEncoding dataBlockEncodingAlgo;
private boolean encodeInCacheOnly;
private Compression.Algorithm compressAlgo;
private StoreFile.BloomType bloomType;
// Writer options
private int numWriterThreads = DEFAULT_NUM_THREADS;
private long minColsPerKey, maxColsPerKey;
@ -104,13 +126,6 @@ public class LoadTestTool extends AbstractHBaseTool {
private int maxReadErrors = MultiThreadedReader.DEFAULT_MAX_ERRORS;
private int verifyPercent;
/** Create tables if needed. */
public void createTables() throws IOException {
HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableName,
COLUMN_FAMILY);
applyBloomFilterAndCompression(tableName, COLUMN_FAMILIES);
}
private String[] splitColonSeparated(String option,
int minNumCols, int maxNumCols) {
String optVal = cmd.getOptionValue(option);
@ -129,31 +144,27 @@ public class LoadTestTool extends AbstractHBaseTool {
}
/**
* Apply the given Bloom filter type to all column families we care about.
* Apply column family options such as Bloom filters, compression, and data
* block encoding.
*/
private void applyBloomFilterAndCompression(byte[] tableName,
private void applyColumnFamilyOptions(byte[] tableName,
byte[][] columnFamilies) throws IOException {
String bloomStr = cmd.getOptionValue(OPT_BLOOM);
StoreFile.BloomType bloomType = bloomStr == null ? null :
StoreFile.BloomType.valueOf(bloomStr);
String compressStr = cmd.getOptionValue(OPT_COMPRESSION);
Compression.Algorithm compressAlgo = compressStr == null ? null :
Compression.Algorithm.valueOf(compressStr);
if (bloomStr == null && compressStr == null)
return;
HBaseAdmin admin = new HBaseAdmin(conf);
HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
LOG.info("Disabling table " + Bytes.toString(tableName));
admin.disableTable(tableName);
for (byte[] cf : columnFamilies) {
HColumnDescriptor columnDesc = tableDesc.getFamily(cf);
if (bloomStr != null)
if (bloomType != null) {
columnDesc.setBloomFilterType(bloomType);
if (compressStr != null)
}
if (compressAlgo != null) {
columnDesc.setCompressionType(compressAlgo);
}
if (dataBlockEncodingAlgo != null) {
columnDesc.setDataBlockEncoding(dataBlockEncodingAlgo);
columnDesc.setEncodeOnDisk(!encodeInCacheOnly);
}
admin.modifyColumn(tableName, columnDesc);
}
LOG.info("Enabling table " + Bytes.toString(tableName));
@ -169,17 +180,22 @@ 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, OPT_DATA_BLOCK_ENCODING_USAGE);
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_ENCODE_IN_CACHE_ONLY, OPT_ENCODE_IN_CACHE_ONLY_USAGE);
addRequiredOptWithArg(OPT_NUM_KEYS, "The number of keys to read/write");
addRequiredOptWithArg(OPT_START_KEY, "The first key to read/write");
addOptWithArg(OPT_START_KEY, "The first key to read/write " +
"(a 0-based index). The default value is " +
DEFAULT_START_KEY + ".");
}
@Override
@ -188,8 +204,8 @@ public class LoadTestTool extends AbstractHBaseTool {
tableName = Bytes.toBytes(cmd.getOptionValue(OPT_TABLE_NAME,
DEFAULT_TABLE_NAME));
startKey = parseLong(cmd.getOptionValue(OPT_START_KEY), 0,
Long.MAX_VALUE);
startKey = parseLong(cmd.getOptionValue(OPT_START_KEY,
String.valueOf(DEFAULT_START_KEY)), 0, Long.MAX_VALUE);
long numKeys = parseLong(cmd.getOptionValue(OPT_NUM_KEYS), 1,
Long.MAX_VALUE - startKey);
endKey = startKey + numKeys;
@ -202,6 +218,9 @@ public class LoadTestTool extends AbstractHBaseTool {
"-" + OPT_READ + " has to be specified");
}
encodeInCacheOnly = cmd.hasOption(OPT_ENCODE_IN_CACHE_ONLY);
parseColumnFamilyOptions(cmd);
if (isWrite) {
String[] writeOpts = splitColonSeparated(OPT_WRITE, 2, 3);
@ -248,7 +267,25 @@ 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) + "]");
}
private void parseColumnFamilyOptions(CommandLine cmd) {
String dataBlockEncodingStr = cmd.getOptionValue(OPT_DATA_BLOCK_ENCODING);
dataBlockEncodingAlgo = dataBlockEncodingStr == null ? null :
DataBlockEncoding.valueOf(dataBlockEncodingStr);
if (dataBlockEncodingAlgo == DataBlockEncoding.NONE && encodeInCacheOnly) {
throw new IllegalArgumentException("-" + OPT_ENCODE_IN_CACHE_ONLY + " " +
"does not make sense when data block encoding is not used");
}
String compressStr = cmd.getOptionValue(OPT_COMPRESSION);
compressAlgo = compressStr == null ? null :
Compression.Algorithm.valueOf(compressStr);
String bloomStr = cmd.getOptionValue(OPT_BLOOM);
bloomType = bloomStr == null ? null :
StoreFile.BloomType.valueOf(bloomStr);
}
@Override
@ -257,7 +294,9 @@ public class LoadTestTool extends AbstractHBaseTool {
conf.set(HConstants.ZOOKEEPER_QUORUM, cmd.getOptionValue(OPT_ZK_QUORUM));
}
createTables();
HBaseTestingUtility.createPreSplitLoadTestTable(conf, tableName,
COLUMN_FAMILY, compressAlgo, dataBlockEncodingAlgo);
applyColumnFamilyOptions(tableName, COLUMN_FAMILIES);
if (isWrite) {
writerThreads = new MultiThreadedWriter(conf, tableName, COLUMN_FAMILY);

View File

@ -242,7 +242,7 @@ public class MultiThreadedReader extends MultiThreadedAction
}
public void queryKey(Get get, boolean verify) throws IOException {
String rowKey = new String(get.getRow());
String rowKey = Bytes.toString(get.getRow());
// read the data
long start = System.currentTimeMillis();

View File

@ -123,7 +123,6 @@ public class MultiThreadedWriter extends MultiThreadedAction {
private class HBaseWriterThread extends Thread {
private final HTable table;
private final int writerId;
private final Random random = new Random();
private final LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(
@ -132,7 +131,6 @@ public class MultiThreadedWriter extends MultiThreadedAction {
public HBaseWriterThread(int writerId) throws IOException {
setName(getClass().getSimpleName() + "_" + writerId);
table = new HTable(conf, tableName);
this.writerId = writerId;
}
public void run() {
@ -166,7 +164,7 @@ public class MultiThreadedWriter extends MultiThreadedAction {
public void insert(long rowKey, long col) {
Put put = new Put(longToByteArrayKey(rowKey));
String colAsStr = String.valueOf(col);
put.add(columnFamily, colAsStr.getBytes(),
put.add(columnFamily, Bytes.toBytes(colAsStr),
dataGenerator.generateRandomSizeValue(rowKey, colAsStr));
try {
long start = System.currentTimeMillis();

View File

@ -28,6 +28,8 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
/**
* A command-line tool that spins up a local process-based cluster, loads
@ -96,7 +98,8 @@ public class RestartMetaTest extends AbstractHBaseTool {
// create tables if needed
HBaseTestingUtility.createPreSplitLoadTestTable(conf, TABLE_NAME,
LoadTestTool.COLUMN_FAMILY);
LoadTestTool.COLUMN_FAMILY, Compression.Algorithm.NONE,
DataBlockEncoding.NONE);
LOG.debug("Loading data....\n\n");
loadData();
@ -133,6 +136,8 @@ public class RestartMetaTest extends AbstractHBaseTool {
protected void addOptions() {
addRequiredOptWithArg(OPT_HBASE_HOME, "HBase home directory");
addOptWithArg(OPT_NUM_RS, "Number of Region Servers");
addOptWithArg(LoadTestTool.OPT_DATA_BLOCK_ENCODING,
LoadTestTool.OPT_DATA_BLOCK_ENCODING_USAGE);
}
@Override

View File

@ -0,0 +1,313 @@
/*
* 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<Long> testNumbers;
private static void addNumber(Set<Long> 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<Long> a = new TreeSet<Long>();
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 testMoveBufferToStream() {
final int arrayOffset = 7;
final int initialPosition = 10;
final int endPadding = 5;
byte[] arrayWrapper =
new byte[arrayOffset + initialPosition + array.length + endPadding];
System.arraycopy(array, 0, arrayWrapper,
arrayOffset + initialPosition, array.length);
ByteBuffer buffer = ByteBuffer.wrap(arrayWrapper, arrayOffset,
initialPosition + array.length).slice();
assertEquals(initialPosition + array.length, buffer.limit());
assertEquals(0, buffer.position());
buffer.position(initialPosition);
ByteArrayOutputStream bos = new ByteArrayOutputStream();
try {
ByteBufferUtils.moveBufferToStream(bos, buffer, array.length);
} catch (IOException e) {
fail("IOException in testCopyToStream()");
}
assertArrayEquals(array, bos.toByteArray());
assertEquals(initialPosition + array.length, buffer.position());
}
/**
* 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.copyBufferToStream(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.copyFromStreamToBuffer(buffer, dis, array.length / 2);
ByteBufferUtils.copyFromStreamToBuffer(buffer, dis,
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.copyFromBufferToBuffer(dstBuffer, srcBuffer,
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 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 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);
}
}
}

View File

@ -66,7 +66,7 @@ public class TestLoadTestKVGenerator {
for (int i = 0; i < 1000; ++i) {
String k = LoadTestKVGenerator.md5PrefixedKey(i);
assertFalse(keys.contains(k));
assertTrue(k.endsWith(":" + i));
assertTrue(k.endsWith("-" + i));
keys.add(k);
}
}

View File

@ -0,0 +1,61 @@
/*
* 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.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.junit.experimental.categories.Category;
import org.junit.runners.Parameterized.Parameters;
/**
* Runs a load test on a mini HBase cluster with data block encoding turned on.
* Compared to other load-test-style unit tests, this one writes a smaller
* amount of data, but goes through all available data block encoding
* algorithms.
*/
@Category(LargeTests.class)
public class TestMiniClusterLoadEncoded extends TestMiniClusterLoadParallel {
/** We do not alternate the multi-put flag in this test. */
private static final boolean USE_MULTI_PUT = true;
@Parameters
public static Collection<Object[]> parameters() {
List<Object[]> parameters = new ArrayList<Object[]>();
for (DataBlockEncoding dataBlockEncoding : DataBlockEncoding.values() ) {
parameters.add(new Object[]{dataBlockEncoding});
}
return parameters;
}
public TestMiniClusterLoadEncoded(DataBlockEncoding encoding) {
super(USE_MULTI_PUT, encoding);
}
/**
* Use a smaller number of keys in in this test.
*/
@Override
protected int numKeys() {
return 3000;
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
import static org.junit.Assert.assertEquals;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
@ -33,18 +34,19 @@ import org.junit.runners.Parameterized;
public class TestMiniClusterLoadParallel
extends TestMiniClusterLoadSequential {
public TestMiniClusterLoadParallel(boolean isMultiPut) {
super(isMultiPut);
public TestMiniClusterLoadParallel(boolean isMultiPut,
DataBlockEncoding encoding) {
super(isMultiPut, encoding);
}
@Test(timeout=120000)
@Test(timeout=TIMEOUT_MS)
public void loadTest() throws Exception {
prepareForLoadTest();
readerThreads.linkToWriter(writerThreads);
writerThreads.start(0, NUM_KEYS, NUM_THREADS);
readerThreads.start(0, NUM_KEYS, NUM_THREADS);
writerThreads.start(0, numKeys, NUM_THREADS);
readerThreads.start(0, numKeys, NUM_THREADS);
writerThreads.waitForFinish();
readerThreads.waitForFinish();
@ -52,7 +54,7 @@ public class TestMiniClusterLoadParallel
assertEquals(0, writerThreads.getNumWriteFailures());
assertEquals(0, readerThreads.getNumReadFailures());
assertEquals(0, readerThreads.getNumReadErrors());
assertEquals(NUM_KEYS, readerThreads.getNumUniqueKeysVerified());
assertEquals(numKeys, readerThreads.getNumUniqueKeysVerified());
}
}

View File

@ -19,14 +19,21 @@ package org.apache.hadoop.hbase.util;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -48,25 +55,39 @@ public class TestMiniClusterLoadSequential {
protected static final byte[] TABLE = Bytes.toBytes("load_test_tbl");
protected static final byte[] CF = Bytes.toBytes("load_test_cf");
protected static final long NUM_KEYS = 10000;
protected static final int NUM_THREADS = 8;
protected static final int NUM_RS = 2;
protected static final int TIMEOUT_MS = 120000;
protected static final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
protected final Configuration conf = TEST_UTIL.getConfiguration();
protected final boolean isMultiPut;
protected final DataBlockEncoding dataBlockEncoding;
protected MultiThreadedWriter writerThreads;
protected MultiThreadedReader readerThreads;
protected int numKeys;
public TestMiniClusterLoadSequential(boolean isMultiPut) {
protected Compression.Algorithm compression = Compression.Algorithm.NONE;
public TestMiniClusterLoadSequential(boolean isMultiPut,
DataBlockEncoding dataBlockEncoding) {
this.isMultiPut = isMultiPut;
this.dataBlockEncoding = dataBlockEncoding;
conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
}
@Parameters
public static Collection<Object[]> parameters() {
return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
List<Object[]> parameters = new ArrayList<Object[]>();
for (boolean multiPut : new boolean[]{false, true}) {
for (DataBlockEncoding dataBlockEncoding : new DataBlockEncoding[] {
DataBlockEncoding.NONE, DataBlockEncoding.PREFIX }) {
parameters.add(new Object[]{multiPut, dataBlockEncoding});
}
}
return parameters;
}
@Before
@ -81,22 +102,28 @@ public class TestMiniClusterLoadSequential {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout=120000)
@Test(timeout=TIMEOUT_MS)
public void loadTest() throws Exception {
prepareForLoadTest();
runLoadTestOnExistingTable();
}
writerThreads.start(0, NUM_KEYS, NUM_THREADS);
protected void runLoadTestOnExistingTable() throws IOException {
writerThreads.start(0, numKeys, NUM_THREADS);
writerThreads.waitForFinish();
assertEquals(0, writerThreads.getNumWriteFailures());
readerThreads.start(0, NUM_KEYS, NUM_THREADS);
readerThreads.start(0, numKeys, NUM_THREADS);
readerThreads.waitForFinish();
assertEquals(0, readerThreads.getNumReadFailures());
assertEquals(0, readerThreads.getNumReadErrors());
assertEquals(NUM_KEYS, readerThreads.getNumKeysVerified());
assertEquals(numKeys, readerThreads.getNumKeysVerified());
}
protected void prepareForLoadTest() throws IOException {
LOG.info("Starting load test: dataBlockEncoding=" + dataBlockEncoding +
", isMultiPut=" + isMultiPut);
numKeys = numKeys();
HBaseAdmin admin = new HBaseAdmin(conf);
while (admin.getClusterStatus().getServers().size() < NUM_RS) {
LOG.info("Sleeping until " + NUM_RS + " RSs are online");
@ -104,8 +131,9 @@ public class TestMiniClusterLoadSequential {
}
admin.close();
int numRegions =
HBaseTestingUtility.createPreSplitLoadTestTable(conf, TABLE, CF);
int numRegions = HBaseTestingUtility.createPreSplitLoadTestTable(conf,
TABLE, CF, compression, dataBlockEncoding);
TEST_UTIL.waitUntilAllRegionsAssigned(numRegions);
writerThreads = new MultiThreadedWriter(conf, TABLE, CF);
@ -113,4 +141,13 @@ public class TestMiniClusterLoadSequential {
readerThreads = new MultiThreadedReader(conf, TABLE, CF, 100);
}
protected int numKeys() {
return 10000;
}
protected HColumnDescriptor getColumnDesc(HBaseAdmin admin)
throws TableNotFoundException, IOException {
return admin.getTableDescriptor(TABLE).getFamily(CF);
}
}