HBASE-61 Create an HBase-specific MapFile implementation

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@747666 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-02-25 05:34:29 +00:00
parent 3ca8865ad6
commit be84747ad3
47 changed files with 1383 additions and 1028 deletions

View File

@ -5,6 +5,8 @@ Release 0.20.0 - Unreleased
HBASE-1144 Store the ROOT region location in Zookeeper
(Nitay Joffe via Stack)
HBASE-1146 Replace the HRS leases with Zookeeper
HBASE-61 Create an HBase-specific MapFile implementation
(Ryan Rawson via Stack)
BUG FIXES
HBASE-1140 "ant clean test" fails (Nitay Joffe via Stack)

View File

@ -315,6 +315,14 @@
Downside to high index interval is lowered access times.
</description>
</property>
<property>
<name>hfile.min.blocksize.size</name>
<value>65536</value>
<description>Minimum store file block size. The smaller you make this, the
bigger your index and the less you fetch on a random-access. Set size down
if you have small cells and want faster random-access of individual cells.
</description>
</property>
<property>
<name>hbase.io.seqfile.compression.type</name>
<value>NONE</value>

View File

@ -335,7 +335,6 @@ class HMerge implements HConstants {
TreeMap<byte [], Cell> results =
new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
while(rootScanner.next(key, results)) {
key.setHRegionInfo(HRegionInfo.ROOT_REGIONINFO);
for(Cell c: results.values()) {
HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
if (info != null) {

View File

@ -427,13 +427,13 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
}
// Compare start keys.
result = HStoreKey.compareTwoRowKeys(o, this.startKey, o.startKey);
result = HStoreKey.compareTwoRowKeys(this.startKey, o.startKey);
if (result != 0) {
return result;
}
// Compare end keys.
return HStoreKey.compareTwoRowKeys(o, this.endKey, o.endKey);
return HStoreKey.compareTwoRowKeys(this.endKey, o.endKey);
}
/**

View File

@ -20,14 +20,21 @@
package org.apache.hadoop.hbase;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.WritableUtils;
/**
* A Key for a stored row.
@ -42,12 +49,6 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
private byte [] column = HConstants.EMPTY_BYTE_ARRAY;
private long timestamp = Long.MAX_VALUE;
/*
* regionInfo is only used as a hack to compare HSKs.
* It is not serialized. See https://issues.apache.org/jira/browse/HBASE-832
*/
private HRegionInfo regionInfo = null;
/**
* Estimated size tax paid for each instance of HSK. Estimate based on
* study of jhat and jprofiler numbers.
@ -55,6 +56,9 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
// In jprofiler, says shallow size is 48 bytes. Add to it cost of two
// byte arrays and then something for the HRI hosting.
public static final int ESTIMATED_HEAP_TAX = 48;
public static final StoreKeyByteComparator BYTECOMPARATOR =
new StoreKeyByteComparator();
/** Default constructor used in conjunction with Writable interface */
public HStoreKey() {
@ -80,18 +84,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param row - row key
*/
public HStoreKey(final String row) {
this(row, Long.MAX_VALUE);
}
/**
* Create an HStoreKey specifying the row and timestamp
* The column and table names default to the empty string
*
* @param row row key
* @param hri
*/
public HStoreKey(final byte [] row, final HRegionInfo hri) {
this(row, HConstants.EMPTY_BYTE_ARRAY, hri);
this(Bytes.toBytes(row), Long.MAX_VALUE);
}
/**
@ -102,32 +95,10 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param timestamp timestamp value
* @param hri HRegionInfo
*/
public HStoreKey(final byte [] row, long timestamp, final HRegionInfo hri) {
this(row, HConstants.EMPTY_BYTE_ARRAY, timestamp, hri);
}
/**
* Create an HStoreKey specifying the row and timestamp
* The column and table names default to the empty string
*
* @param row row key
* @param timestamp timestamp value
*/
public HStoreKey(final byte [] row, long timestamp) {
public HStoreKey(final byte [] row, final long timestamp) {
this(row, HConstants.EMPTY_BYTE_ARRAY, timestamp);
}
/**
* Create an HStoreKey specifying the row and timestamp
* The column and table names default to the empty string
*
* @param row row key
* @param timestamp timestamp value
*/
public HStoreKey(final String row, long timestamp) {
this (row, "", timestamp, new HRegionInfo());
}
/**
* Create an HStoreKey specifying the row and column names
* The timestamp defaults to LATEST_TIMESTAMP
@ -137,7 +108,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param column column key
*/
public HStoreKey(final String row, final String column) {
this(row, column, HConstants.LATEST_TIMESTAMP, new HRegionInfo());
this(row, column, HConstants.LATEST_TIMESTAMP);
}
/**
@ -151,19 +122,6 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
public HStoreKey(final byte [] row, final byte [] column) {
this(row, column, HConstants.LATEST_TIMESTAMP);
}
/**
* Create an HStoreKey specifying the row, column names and table name
* The timestamp defaults to LATEST_TIMESTAMP
*
* @param row row key
* @param column column key
* @param regionInfo region info
*/
public HStoreKey(final byte [] row,
final byte [] column, final HRegionInfo regionInfo) {
this(row, column, HConstants.LATEST_TIMESTAMP, regionInfo);
}
/**
* Create an HStoreKey specifying all the fields
@ -174,24 +132,10 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param timestamp timestamp value
* @param regionInfo region info
*/
public HStoreKey(final String row,
final String column, long timestamp, final HRegionInfo regionInfo) {
this (Bytes.toBytes(row), Bytes.toBytes(column),
timestamp, regionInfo);
public HStoreKey(final String row, final String column, final long timestamp) {
this (Bytes.toBytes(row), Bytes.toBytes(column), timestamp);
}
/**
* Create an HStoreKey specifying all the fields with unspecified table
* Does not make copies of the passed byte arrays. Presumes the passed
* arrays immutable.
* @param row row key
* @param column column key
* @param timestamp timestamp value
*/
public HStoreKey(final byte [] row, final byte [] column, long timestamp) {
this(row, column, timestamp, null);
}
/**
* Create an HStoreKey specifying all the fields with specified table
* Does not make copies of the passed byte arrays. Presumes the passed
@ -201,13 +145,11 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param timestamp timestamp value
* @param regionInfo region info
*/
public HStoreKey(final byte [] row,
final byte [] column, long timestamp, final HRegionInfo regionInfo) {
public HStoreKey(final byte [] row, final byte [] column, final long timestamp) {
// Make copies
this.row = row;
this.column = column;
this.timestamp = timestamp;
this.regionInfo = regionInfo;
}
/**
@ -215,17 +157,20 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*
* @param other the source key
*/
public HStoreKey(HStoreKey other) {
this(other.getRow(), other.getColumn(), other.getTimestamp(),
other.getHRegionInfo());
public HStoreKey(final HStoreKey other) {
this(other.getRow(), other.getColumn(), other.getTimestamp());
}
public HStoreKey(final ByteBuffer bb) {
this(getRow(bb), getColumn(bb), getTimestamp(bb));
}
/**
* Change the value of the row key
*
* @param newrow new row key value
*/
public void setRow(byte [] newrow) {
public void setRow(final byte [] newrow) {
this.row = newrow;
}
@ -234,7 +179,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*
* @param c new column family value
*/
public void setColumn(byte [] c) {
public void setColumn(final byte [] c) {
this.column = c;
}
@ -243,7 +188,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*
* @param timestamp new timestamp value
*/
public void setVersion(long timestamp) {
public void setVersion(final long timestamp) {
this.timestamp = timestamp;
}
@ -252,7 +197,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*
* @param k key value to copy
*/
public void set(HStoreKey k) {
public void set(final HStoreKey k) {
this.row = k.getRow();
this.column = k.getColumn();
this.timestamp = k.getTimestamp();
@ -272,19 +217,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
public long getTimestamp() {
return this.timestamp;
}
/** @return value of regioninfo */
public HRegionInfo getHRegionInfo() {
return this.regionInfo;
}
/**
* @param hri
*/
public void setHRegionInfo(final HRegionInfo hri) {
this.regionInfo = hri;
}
/**
* Compares the row and column of two keys
* @param other Key to compare against. Compares row and column.
@ -292,8 +225,8 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @see #matchesWithoutColumn(HStoreKey)
* @see #matchesRowFamily(HStoreKey)
*/
public boolean matchesRowCol(HStoreKey other) {
return HStoreKey.equalsTwoRowKeys(getHRegionInfo(), getRow(), other.getRow()) &&
public boolean matchesRowCol(final HStoreKey other) {
return HStoreKey.equalsTwoRowKeys(getRow(), other.getRow()) &&
Bytes.equals(getColumn(), other.getColumn());
}
@ -306,8 +239,8 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @see #matchesRowCol(HStoreKey)
* @see #matchesRowFamily(HStoreKey)
*/
public boolean matchesWithoutColumn(HStoreKey other) {
return equalsTwoRowKeys(getHRegionInfo(), getRow(), other.getRow()) &&
public boolean matchesWithoutColumn(final HStoreKey other) {
return equalsTwoRowKeys(getRow(), other.getRow()) &&
getTimestamp() >= other.getTimestamp();
}
@ -320,9 +253,9 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @see #matchesRowCol(HStoreKey)
* @see #matchesWithoutColumn(HStoreKey)
*/
public boolean matchesRowFamily(HStoreKey that) {
int delimiterIndex = getFamilyDelimiterIndex(getColumn());
return equalsTwoRowKeys(getHRegionInfo(), getRow(), that.getRow()) &&
public boolean matchesRowFamily(final HStoreKey that) {
final int delimiterIndex = getFamilyDelimiterIndex(getColumn());
return equalsTwoRowKeys(getRow(), that.getRow()) &&
Bytes.compareTo(getColumn(), 0, delimiterIndex, that.getColumn(), 0,
delimiterIndex) == 0;
}
@ -334,8 +267,8 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
}
@Override
public boolean equals(Object obj) {
HStoreKey other = (HStoreKey)obj;
public boolean equals(final Object obj) {
final HStoreKey other = (HStoreKey)obj;
// Do a quick check.
if (this.row.length != other.row.length ||
this.column.length != other.column.length ||
@ -356,17 +289,15 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
// Comparable
public int compareTo(final HStoreKey o) {
return compareTo(this.regionInfo, this, o);
return compareTo(this, o);
}
static int compareTo(final HRegionInfo hri, final HStoreKey left,
final HStoreKey right) {
static int compareTo(final HStoreKey left, final HStoreKey right) {
// We can be passed null
if (left == null && right == null) return 0;
if (left == null) return -1;
if (right == null) return 1;
int result = compareTwoRowKeys(hri, left.getRow(), right.getRow());
int result = compareTwoRowKeys(left.getRow(), right.getRow());
if (result != 0) {
return result;
}
@ -386,11 +317,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
} else if (left.getTimestamp() > right.getTimestamp()) {
result = -1;
}
// Because of HBASE-877, our BeforeThisStoreKey trick no longer works in
// mapfiles and so instead we need to do this weird check here below.
return result == 0 && left instanceof BeforeThisStoreKey? -1:
result == 0 && right instanceof BeforeThisStoreKey? 1:
result;
return result;
}
/**
@ -402,13 +329,13 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*/
public static byte [] getFamily(final byte [] column)
throws ColumnNameParseException {
int index = getFamilyDelimiterIndex(column);
final int index = getFamilyDelimiterIndex(column);
if (index <= 0) {
throw new ColumnNameParseException("Missing ':' delimiter between " +
"column family and qualifier in the passed column name <" +
Bytes.toString(column) + ">");
}
byte [] result = new byte[index];
final byte [] result = new byte[index];
System.arraycopy(column, 0, result, 0, index);
return result;
}
@ -418,7 +345,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @return Return hash of family portion of passed column.
*/
public static Integer getFamilyMapKey(final byte [] column) {
int index = getFamilyDelimiterIndex(column);
final int index = getFamilyDelimiterIndex(column);
// If index < -1, presume passed column is a family name absent colon
// delimiter
return Bytes.mapKey(column, index > 0? index: column.length);
@ -432,7 +359,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
public static boolean matchingFamily(final byte [] family,
final byte [] column) {
// Make sure index of the ':' is at same offset.
int index = getFamilyDelimiterIndex(column);
final int index = getFamilyDelimiterIndex(column);
if (index != family.length) {
return false;
}
@ -445,7 +372,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*/
public static byte [] addDelimiter(final byte [] family) {
// Manufacture key by adding delimiter to the passed in colFamily.
byte [] familyPlusDelimiter = new byte [family.length + 1];
final byte [] familyPlusDelimiter = new byte [family.length + 1];
System.arraycopy(family, 0, familyPlusDelimiter, 0, family.length);
familyPlusDelimiter[family.length] = HStoreKey.COLUMN_FAMILY_DELIMITER;
return familyPlusDelimiter;
@ -457,9 +384,9 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @see #parseColumn(byte[])
*/
public static byte [] getQualifier(final byte [] column) {
int index = getFamilyDelimiterIndex(column);
int len = column.length - (index + 1);
byte [] result = new byte[len];
final int index = getFamilyDelimiterIndex(column);
final int len = column.length - (index + 1);
final byte [] result = new byte[len];
System.arraycopy(column, index + 1, result, 0, len);
return result;
}
@ -473,14 +400,14 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
*/
public static byte [][] parseColumn(final byte [] c)
throws ColumnNameParseException {
byte [][] result = new byte [2][];
int index = getFamilyDelimiterIndex(c);
final byte [][] result = new byte [2][];
final int index = getFamilyDelimiterIndex(c);
if (index == -1) {
throw new ColumnNameParseException("Impossible column name: " + c);
}
result[0] = new byte [index];
System.arraycopy(c, 0, result[0], 0, index);
int len = c.length - (index + 1);
final int len = c.length - (index + 1);
result[1] = new byte[len];
System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0,
len);
@ -506,15 +433,6 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
return result;
}
/**
* Returns row and column bytes out of an HStoreKey.
* @param hsk Store key.
* @return byte array encoding of HStoreKey
*/
public static byte[] getBytes(final HStoreKey hsk) {
return Bytes.add(hsk.getRow(), hsk.getColumn());
}
/**
* Utility method to compare two row keys.
* This is required because of the meta delimiters.
@ -524,16 +442,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* @param rowB
* @return value of the comparison
*/
public static int compareTwoRowKeys(HRegionInfo regionInfo,
byte[] rowA, byte[] rowB) {
if (regionInfo != null && regionInfo.isMetaRegion()) {
byte[][] keysA = stripStartKeyMeta(rowA);
byte[][] KeysB = stripStartKeyMeta(rowB);
int rowCompare = Bytes.compareTo(keysA[0], KeysB[0]);
if(rowCompare == 0)
rowCompare = Bytes.compareTo(keysA[1], KeysB[1]);
return rowCompare;
}
public static int compareTwoRowKeys(final byte[] rowA, final byte[] rowB) {
return Bytes.compareTo(rowA, rowB);
}
@ -541,54 +450,48 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* Utility method to check if two row keys are equal.
* This is required because of the meta delimiters
* This is a hack
* @param regionInfo
* @param rowA
* @param rowB
* @return if it's equal
*/
public static boolean equalsTwoRowKeys(HRegionInfo regionInfo,
byte[] rowA, byte[] rowB) {
public static boolean equalsTwoRowKeys(final byte[] rowA, final byte[] rowB) {
return ((rowA == null) && (rowB == null)) ? true:
(rowA == null) || (rowB == null) || (rowA.length != rowB.length) ? false:
compareTwoRowKeys(regionInfo,rowA,rowB) == 0;
compareTwoRowKeys(rowA,rowB) == 0;
}
private static byte[][] stripStartKeyMeta(byte[] rowKey) {
int offset = -1;
for (int i = rowKey.length - 1; i > 0; i--) {
if (rowKey[i] == HConstants.META_ROW_DELIMITER) {
offset = i;
break;
}
}
byte [] row = rowKey;
byte [] timestamp = HConstants.EMPTY_BYTE_ARRAY;
if (offset != -1) {
row = new byte[offset];
System.arraycopy(rowKey, 0, row, 0, offset);
timestamp = new byte[rowKey.length - offset - 1];
System.arraycopy(rowKey, offset+1, timestamp, 0,rowKey.length - offset - 1);
}
byte[][] elements = new byte[2][];
elements[0] = row;
elements[1] = timestamp;
return elements;
}
// Writable
public void write(DataOutput out) throws IOException {
public void write(final DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.row);
Bytes.writeByteArray(out, this.column);
out.writeLong(timestamp);
}
public void readFields(DataInput in) throws IOException {
public void readFields(final DataInput in) throws IOException {
this.row = Bytes.readByteArray(in);
this.column = Bytes.readByteArray(in);
this.timestamp = in.readLong();
}
/**
* @param hsk
* @return Size of this key in serialized bytes.
*/
public static int getSerializedSize(final HStoreKey hsk) {
return getSerializedSize(hsk.getRow()) +
getSerializedSize(hsk.getColumn()) +
Bytes.SIZEOF_LONG;
}
/**
* @param b
* @return Length of buffer when its been serialized.
*/
private static int getSerializedSize(final byte [] b) {
return b == null? 1: b.length + WritableUtils.getVIntSize(b.length);
}
public long heapSize() {
return getRow().length + Bytes.ESTIMATED_HEAP_TAX +
getColumn().length + Bytes.ESTIMATED_HEAP_TAX +
@ -596,22 +499,47 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
}
/**
* Passed as comparator for memcache and for store files. See HBASE-868.
* @return The bytes of <code>hsk</code> gotten by running its
* {@link Writable#write(java.io.DataOutput)} method.
* @throws IOException
*/
public static class HStoreKeyWritableComparator extends WritableComparator {
private final HRegionInfo hri;
/** @param hri */
public HStoreKeyWritableComparator(final HRegionInfo hri) {
super(HStoreKey.class);
this.hri = hri;
public byte [] getBytes() throws IOException {
return getBytes(this);
}
/**
* Return serialize <code>hsk</code> bytes.
* Note, this method's implementation has changed. Used to just return
* row and column. This is a customized version of
* {@link Writables#getBytes(Writable)}
* @param hsk Instance
* @return The bytes of <code>hsk</code> gotten by running its
* {@link Writable#write(java.io.DataOutput)} method.
* @throws IOException
*/
public static byte [] getBytes(final HStoreKey hsk) throws IOException {
// TODO: Redo with system.arraycopy instead of DOS.
if (hsk == null) {
throw new IllegalArgumentException("Writable cannot be null");
}
@SuppressWarnings("unchecked")
@Override
public int compare(final WritableComparable left,
final WritableComparable right) {
return compareTo(this.hri, (HStoreKey)left, (HStoreKey)right);
final int serializedSize = getSerializedSize(hsk);
final ByteArrayOutputStream byteStream = new ByteArrayOutputStream(serializedSize);
DataOutputStream out = new DataOutputStream(byteStream);
try {
hsk.write(out);
out.close();
out = null;
final byte [] serializedKey = byteStream.toByteArray();
if (serializedKey.length != serializedSize) {
// REMOVE THIS AFTER CONFIDENCE THAT OUR SIZING IS BEING DONE PROPERLY
throw new AssertionError("Sizes do not agree " + serializedKey.length +
", " + serializedSize);
}
return serializedKey;
} finally {
if (out != null) {
out.close();
}
}
}
@ -624,6 +552,7 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
* returning us the deleted key (getClosest gets exact or nearest before when
* you pass true argument). TODO: Throw this class away when MapFile has
* a real 'previous' method. See HBASE-751.
* @deprecated
*/
public static class BeforeThisStoreKey extends HStoreKey {
private final HStoreKey beforeThisKey;
@ -638,12 +567,12 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
@Override
public int compareTo(final HStoreKey o) {
int result = this.beforeThisKey.compareTo(o);
final int result = this.beforeThisKey.compareTo(o);
return result == 0? -1: result;
}
@Override
public boolean equals(Object obj) {
public boolean equals(final Object obj) {
return false;
}
@ -673,42 +602,42 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
}
@Override
public boolean matchesRowCol(HStoreKey other) {
public boolean matchesRowCol(final HStoreKey other) {
return this.beforeThisKey.matchesRowCol(other);
}
@Override
public boolean matchesRowFamily(HStoreKey that) {
public boolean matchesRowFamily(final HStoreKey that) {
return this.beforeThisKey.matchesRowFamily(that);
}
@Override
public boolean matchesWithoutColumn(HStoreKey other) {
public boolean matchesWithoutColumn(final HStoreKey other) {
return this.beforeThisKey.matchesWithoutColumn(other);
}
@Override
public void readFields(DataInput in) throws IOException {
public void readFields(final DataInput in) throws IOException {
this.beforeThisKey.readFields(in);
}
@Override
public void set(HStoreKey k) {
public void set(final HStoreKey k) {
this.beforeThisKey.set(k);
}
@Override
public void setColumn(byte[] c) {
public void setColumn(final byte[] c) {
this.beforeThisKey.setColumn(c);
}
@Override
public void setRow(byte[] newrow) {
public void setRow(final byte[] newrow) {
this.beforeThisKey.setRow(newrow);
}
@Override
public void setVersion(long timestamp) {
public void setVersion(final long timestamp) {
this.beforeThisKey.setVersion(timestamp);
}
@ -718,18 +647,266 @@ public class HStoreKey implements WritableComparable<HStoreKey>, HeapSize {
}
@Override
public void write(DataOutput out) throws IOException {
public void write(final DataOutput out) throws IOException {
this.beforeThisKey.write(out);
}
@Override
public HRegionInfo getHRegionInfo() {
return this.beforeThisKey.getHRegionInfo();
}
/**
* Passed as comparator for memcache and for store files. See HBASE-868.
*/
public static class HStoreKeyWritableComparator extends WritableComparator {
public HStoreKeyWritableComparator() {
super(HStoreKey.class);
}
@Override
public void setHRegionInfo(final HRegionInfo hri) {
this.beforeThisKey.setHRegionInfo(hri);
@SuppressWarnings("unchecked")
public int compare(final WritableComparable left,
final WritableComparable right) {
return compareTo((HStoreKey)left, (HStoreKey)right);
}
}
/**
* @param bb ByteBuffer that contains serialized HStoreKey
* @return Row
*/
public static byte [] getRow(final ByteBuffer bb) {
byte firstByte = bb.get(0);
int vint = firstByte;
int vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(), bb.arrayOffset());
}
byte [] b = new byte [vint];
System.arraycopy(bb.array(), bb.arrayOffset() + vintWidth, b, 0, vint);
return b;
}
/**
* @param bb ByteBuffer that contains serialized HStoreKey
* @return Column
*/
public static byte [] getColumn(final ByteBuffer bb) {
byte firstByte = bb.get(0);
int vint = firstByte;
int vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(), bb.arrayOffset());
}
// Skip over row.
int offset = vint + vintWidth;
firstByte = bb.get(offset);
vint = firstByte;
vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(),
bb.arrayOffset() + offset);
}
byte [] b = new byte [vint];
System.arraycopy(bb.array(), bb.arrayOffset() + offset + vintWidth, b, 0,
vint);
return b;
}
/**
* @param bb ByteBuffer that contains serialized HStoreKey
* @return Timestamp
*/
public static long getTimestamp(final ByteBuffer bb) {
byte firstByte = bb.get(0);
int vint = firstByte;
int vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(), bb.arrayOffset());
}
// Skip over row.
int offset = vint + vintWidth;
firstByte = bb.get(offset);
vint = firstByte;
vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(),
bb.arrayOffset() + offset);
}
// Skip over column
offset += (vint + vintWidth);
return bb.getLong(offset);
}
/**
* RawComparator for plain -- i.e. non-catalog table keys such as
* -ROOT- and .META. -- HStoreKeys. Compares at byte level.
*/
public static class StoreKeyByteComparator implements RawComparator<byte []> {
public StoreKeyByteComparator() {
super();
}
public int compare(final byte[] b1, final byte[] b2) {
return compare(b1, 0, b1.length, b2, 0, b2.length);
}
public int compare(final byte [] b1, int o1, int l1,
final byte [] b2, int o2, int l2) {
// Below is byte compare without creating new objects. Its awkward but
// seems no way around getting vint width, value, and compare result any
// other way. The passed byte arrays, b1 and b2, have a vint, row, vint,
// column, timestamp in them. The byte array was written by the
// #write(DataOutputStream) method above. See it to better understand the
// below.
// Calculate vint and vint width for rows in b1 and b2.
byte firstByte1 = b1[o1];
int vint1 = firstByte1;
int vintWidth1 = WritableUtils.decodeVIntSize(firstByte1);
if (vintWidth1 != 1) {
vint1 = getBigVint(vintWidth1, firstByte1, b1, o1);
}
byte firstByte2 = b2[o2];
int vint2 = firstByte2;
int vintWidth2 = WritableUtils.decodeVIntSize(firstByte2);
if (vintWidth2 != 1) {
vint2 = getBigVint(vintWidth2, firstByte2, b2, o2);
}
// Compare the rows.
int result = WritableComparator.compareBytes(b1, o1 + vintWidth1, vint1,
b2, o2 + vintWidth2, vint2);
if (result != 0) {
return result;
}
// Update offsets and lengths so we are aligned on columns.
int diff1 = vintWidth1 + vint1;
o1 += diff1;
l1 -= diff1;
int diff2 = vintWidth2 + vint2;
o2 += diff2;
l2 -= diff2;
// Calculate vint and vint width for columns in b1 and b2.
firstByte1 = b1[o1];
vint1 = firstByte1;
vintWidth1 = WritableUtils.decodeVIntSize(firstByte1);
if (vintWidth1 != 1) {
vint1 = getBigVint(vintWidth1, firstByte1, b1, o1);
}
firstByte2 = b2[o2];
vint2 = firstByte2;
vintWidth2 = WritableUtils.decodeVIntSize(firstByte2);
if (vintWidth2 != 1) {
vint2 = getBigVint(vintWidth2, firstByte2, b2, o2);
}
// Compare columns.
result = WritableComparator.compareBytes(b1, o1 + vintWidth1, vint1,
b2, o2 + vintWidth2, vint2);
if (result != 0) {
return result;
}
// Update offsets and lengths.
diff1 = vintWidth1 + vint1;
o1 += diff1;
l1 -= diff1;
diff2 = vintWidth2 + vint2;
o2 += diff2;
l2 -= diff2;
// The below older timestamps sorting ahead of newer timestamps looks
// wrong but it is intentional. This way, newer timestamps are first
// found when we iterate over a memcache and newer versions are the
// first we trip over when reading from a store file.
for (int i = 0; i < l1; i++) {
int leftb = b1[o1 + i] & 0xff;
int rightb = b2[o2 + i] & 0xff;
if (leftb < rightb) {
return 1;
} else if (leftb > rightb) {
return -1;
}
}
return 0;
}
}
/*
* Vint is wider than one byte. Find out how much bigger it is.
* @param vintWidth
* @param firstByte
* @param buffer
* @param offset
* @return
*/
static int getBigVint(final int vintWidth, final byte firstByte,
final byte [] buffer, final int offset) {
long i = 0;
for (int idx = 0; idx < vintWidth - 1; idx++) {
final byte b = buffer[offset + 1 + idx];
i = i << 8;
i = i | (b & 0xFF);
}
i = (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
if (i > Integer.MAX_VALUE) {
throw new IllegalArgumentException("Calculated vint too large");
}
return (int)i;
}
/**
* Create a store key.
* @param bb
* @return HStoreKey instance made of the passed <code>b</code>.
* @throws IOException
*/
public static HStoreKey create(final ByteBuffer bb)
throws IOException {
byte firstByte = bb.get(0);
int vint = firstByte;
int vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(), bb.arrayOffset());
}
byte [] row = new byte [vint];
System.arraycopy(bb.array(), bb.arrayOffset() + vintWidth,
row, 0, row.length);
// Skip over row.
int offset = vint + vintWidth;
firstByte = bb.get(offset);
vint = firstByte;
vintWidth = WritableUtils.decodeVIntSize(firstByte);
if (vintWidth != 1) {
vint = getBigVint(vintWidth, firstByte, bb.array(),
bb.arrayOffset() + offset);
}
byte [] column = new byte [vint];
System.arraycopy(bb.array(), bb.arrayOffset() + offset + vintWidth,
column, 0, column.length);
// Skip over column
offset += (vint + vintWidth);
long ts = bb.getLong(offset);
return new HStoreKey(row, column, ts);
}
/**
* Create a store key.
* @param b Serialized HStoreKey; a byte array with a row only in it won't do.
* It must have all the vints denoting r/c/ts lengths.
* @return HStoreKey instance made of the passed <code>b</code>.
* @throws IOException
*/
public static HStoreKey create(final byte [] b) throws IOException {
return create(b, 0, b.length);
}
/**
* Create a store key.
* @param b Serialized HStoreKey
* @param offset
* @param length
* @return HStoreKey instance made of the passed <code>b</code>.
* @throws IOException
*/
public static HStoreKey create(final byte [] b, final int offset,
final int length)
throws IOException {
return (HStoreKey)Writables.getWritable(b, offset, length, new HStoreKey());
}
}

View File

@ -385,8 +385,8 @@ public class HConnectionManager implements HConstants {
}
}
endKey = currentRegion.getEndKey();
} while (!(endKey == null || HStoreKey.equalsTwoRowKeys(currentRegion,
endKey, HConstants.EMPTY_BYTE_ARRAY)));
} while (!(endKey == null || HStoreKey.equalsTwoRowKeys(endKey,
HConstants.EMPTY_BYTE_ARRAY)));
}
finally {
s.setClose();
@ -645,10 +645,8 @@ public class HConnectionManager implements HConstants {
// this one. the exception case is when the endkey is EMPTY_START_ROW,
// signifying that the region we're checking is actually the last
// region in the table.
if (HStoreKey.equalsTwoRowKeys(possibleRegion.getRegionInfo(),
endKey, HConstants.EMPTY_END_ROW) ||
HStoreKey.compareTwoRowKeys(possibleRegion.getRegionInfo(),
endKey, row) > 0) {
if (HStoreKey.equalsTwoRowKeys(endKey, HConstants.EMPTY_END_ROW) ||
HStoreKey.compareTwoRowKeys(endKey, row) > 0) {
return possibleRegion;
}
}
@ -685,8 +683,7 @@ public class HConnectionManager implements HConstants {
// by nature of the map, we know that the start key has to be <
// otherwise it wouldn't be in the headMap.
if (HStoreKey.compareTwoRowKeys(possibleRegion.getRegionInfo(),
endKey, row) <= 0) {
if (HStoreKey.compareTwoRowKeys(endKey, row) <= 0) {
// delete any matching entry
HRegionLocation rl =
tableLocations.remove(matchingRegions.lastKey());

View File

@ -226,9 +226,8 @@ public class HTable {
* @return Array of region starting row keys
* @throws IOException
*/
public byte[][] getStartKeys() throws IOException {
public byte [][] getStartKeys() throws IOException {
final List<byte[]> keyList = new ArrayList<byte[]>();
MetaScannerVisitor visitor = new MetaScannerVisitor() {
public boolean processRow(RowResult rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
@ -240,7 +239,6 @@ public class HTable {
}
return true;
}
};
MetaScanner.metaScan(configuration, visitor, this.tableName);
return keyList.toArray(new byte[keyList.size()][]);

View File

@ -57,7 +57,6 @@ class MetaScanner implements HConstants {
RowResult r = null;
do {
RowResult[] rrs = connection.getRegionServerWithRetries(callable);
if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
break;
}
@ -70,7 +69,7 @@ class MetaScanner implements HConstants {
callable.setClose();
connection.getRegionServerWithRetries(callable);
}
} while (HStoreKey.compareTwoRowKeys(callable.getHRegionInfo(), startRow, LAST_ROW) != 0);
} while (HStoreKey.compareTwoRowKeys(startRow, LAST_ROW) != 0);
}
/**

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.io;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
@ -76,6 +77,16 @@ public class Cell implements Writable, Iterable<Map.Entry<Long, byte[]>>,
valueMap.put(timestamp, value);
}
/**
* Create a new Cell with a given value and timestamp. Used by HStore.
*
* @param bb
* @param timestamp
*/
public Cell(final ByteBuffer bb, long timestamp) {
this.valueMap.put(timestamp, Bytes.toBytes(bb));
}
/**
* @param vals
* array of values

View File

@ -72,10 +72,10 @@ public class HBaseMapFile extends MapFile {
public HBaseReader(FileSystem fs, String dirName, Configuration conf,
boolean blockCacheEnabled, HRegionInfo hri)
throws IOException {
super(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri),
super(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(),
conf, false); // defer opening streams
this.blockCacheEnabled = blockCacheEnabled;
open(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri), conf);
open(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(), conf);
// Force reading of the mapfile index by calling midKey. Reading the
// index will bring the index into memory over here on the client and
@ -121,7 +121,7 @@ public class HBaseMapFile extends MapFile {
public HBaseWriter(Configuration conf, FileSystem fs, String dirName,
SequenceFile.CompressionType compression, final HRegionInfo hri)
throws IOException {
super(conf, fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri),
super(conf, fs, dirName, new HStoreKey.HStoreKeyWritableComparator(),
VALUE_CLASS, compression);
// Default for mapfiles is 128. Makes random reads faster if we
// have more keys indexed and we're not 'next'-ing around in the

View File

@ -90,7 +90,6 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.midkey = new HStoreKey((HStoreKey)mk);
this.midkey.setHRegionInfo(hri);
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r);
}
@ -212,4 +211,4 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
checkKey(key);
return super.seek(key);
}
}
}

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.util.ReflectionUtils;
* if passed a value type that it has not already been told about. Its been
* primed with hbase Writables and byte []. Keys are always byte arrays.
*
* @param <byte []> key TODO: Parameter K is never used, could be removed.
* @param <K> <byte []> key TODO: Parameter K is never used, could be removed.
* @param <V> value Expects a Writable or byte [].
*/
public class HbaseMapWritable <K, V>
@ -191,7 +191,7 @@ implements SortedMap<byte [], V>, Writable, Configurable {
// Then write out each key/value pair
for (Map.Entry<byte [], V> e: instance.entrySet()) {
Bytes.writeByteArray(out, e.getKey());
Byte id =getId(e.getValue().getClass());
Byte id = getId(e.getValue().getClass());
out.writeByte(id);
Object value = e.getValue();
if (value instanceof byte []) {

View File

@ -171,11 +171,13 @@ public class MapFile {
CompressionType.BLOCK, progress);
}
/** The number of entries that are added before an index entry is added.*/
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileWriter#getIndexInterval()
*/
public int getIndexInterval() { return indexInterval; }
/** Sets the index interval.
* @see #getIndexInterval()
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileWriter#setIndexInterval(int)
*/
public void setIndexInterval(int interval) { indexInterval = interval; }
@ -186,14 +188,17 @@ public class MapFile {
conf.setInt(INDEX_INTERVAL, interval);
}
/** Close the map. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileWriter#close()
*/
public synchronized void close() throws IOException {
data.close();
index.close();
}
/** Append a key/value pair to the map. The key must be greater or equal
* to the previous key added to the map. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileWriter#append(org.apache.hadoop.io.WritableComparable, org.apache.hadoop.io.Writable)
*/
public synchronized void append(WritableComparable key, Writable val)
throws IOException {
@ -250,10 +255,14 @@ public class MapFile {
private WritableComparable[] keys;
private long[] positions;
/** Returns the class of keys in this file. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#getKeyClass()
*/
public Class<?> getKeyClass() { return data.getKeyClass(); }
/** Returns the class of values in this file. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#getValueClass()
*/
public Class<?> getValueClass() { return data.getValueClass(); }
/** Construct a map reader for the named map.*/
@ -362,14 +371,15 @@ public class MapFile {
}
}
/** Re-positions the reader before its first key. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#reset()
*/
public synchronized void reset() throws IOException {
data.seek(firstPosition);
}
/** Get the key at approximately the middle of the file.
*
* @throws IOException
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#midKey()
*/
public synchronized WritableComparable midKey() throws IOException {
@ -382,9 +392,8 @@ public class MapFile {
return keys[pos];
}
/** Reads the final key from the file.
*
* @param key key to read into
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#finalKey(org.apache.hadoop.io.WritableComparable)
*/
public synchronized void finalKey(WritableComparable key)
throws IOException {
@ -404,9 +413,8 @@ public class MapFile {
}
}
/** Positions the reader at the named key, or if none such exists, at the
* first entry after the named key. Returns true iff the named key exists
* in this map.
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#seek(org.apache.hadoop.io.WritableComparable)
*/
public synchronized boolean seek(WritableComparable key) throws IOException {
return seekInternal(key) == 0;
@ -517,15 +525,17 @@ public class MapFile {
return -(low + 1); // key not found.
}
/** Read the next key/value pair in the map into <code>key</code> and
* <code>val</code>. Returns true if such a pair exists and false when at
* the end of the map */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#next(org.apache.hadoop.io.WritableComparable, org.apache.hadoop.io.Writable)
*/
public synchronized boolean next(WritableComparable key, Writable val)
throws IOException {
return data.next(key, val);
}
/** Return the value for the named key, or null if none exists. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#get(org.apache.hadoop.io.WritableComparable, org.apache.hadoop.io.Writable)
*/
public synchronized Writable get(WritableComparable key, Writable val)
throws IOException {
if (seek(key)) {
@ -535,14 +545,8 @@ public class MapFile {
return null;
}
/**
* Finds the record that is the closest match to the specified key.
* Returns <code>key</code> or if it does not exist, at the first entry
* after the named key.
*
- * @param key - key that we're trying to find
- * @param val - data value if key is found
- * @return - the key that was the closest match or null if eof.
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#getClosest(org.apache.hadoop.io.WritableComparable, org.apache.hadoop.io.Writable)
*/
public synchronized WritableComparable getClosest(WritableComparable key,
Writable val)
@ -550,15 +554,8 @@ public class MapFile {
return getClosest(key, val, false);
}
/**
* Finds the record that is the closest match to the specified key.
*
* @param key - key that we're trying to find
* @param val - data value if key is found
* @param before - IF true, and <code>key</code> does not exist, return
* the first entry that falls just before the <code>key</code>. Otherwise,
* return the record that sorts just after.
* @return - the key that was the closest match or null if eof.
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#getClosest(org.apache.hadoop.io.WritableComparable, org.apache.hadoop.io.Writable, boolean)
*/
public synchronized WritableComparable getClosest(WritableComparable key,
Writable val, final boolean before)
@ -578,7 +575,9 @@ public class MapFile {
return nextKey;
}
/** Close the map. */
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.io.StoreFileReader#close()
*/
public synchronized void close() throws IOException {
if (!indexClosed) {
index.close();

View File

@ -7,41 +7,34 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.Writable;
/**
* A reference to a part of a store file. The file referenced usually lives
* under a different region. The part referenced is usually the top or bottom
* half of the file. References are made at region split time. Being lazy
* about copying data between the parent of the split and the split daughters
* makes splitting faster.
* A reference to the top or bottom half of a store file. The file referenced
* lives under a different region. References are made at region split time.
*
* <p>References work with {@link HalfMapFileReader}. References know how to
* write out the reference format in the file system and are whats juggled when
* references are mixed in with direct store files. The
* {@link HalfMapFileReader} is used reading the referred to file.
* <p>References work with a special half store file type. References know how
* to write out the reference format in the file system and are whats juggled
* when references are mixed in with direct store files. The half store file
* type is used reading the referred to file.
*
* <p>References to store files located over in some other region look like
* this in the file system
* <code>1278437856009925445.hbaserepository,qAReLZD-OyQORZWq_vqR1k==,959247014679548184</code>:
* i.e. an id followed by the name of the referenced region. The data
* ('mapfiles') of references are empty. The accompanying <code>info</code> file
* contains the <code>midkey</code> that demarks top and bottom of the
* referenced storefile, the id of the remote store we're referencing and
* whether we're to serve the top or bottom region of the remote store file.
* <code>1278437856009925445.3323223323</code>:
* i.e. an id followed by hash of the referenced region.
* Note, a region is itself not splitable if it has instances of store file
* references. References are cleaned up by compactions.
*/
public class Reference implements Writable {
// TODO: see if it makes sense making a ReferenceMapFile whose Writer is this
// class and whose Reader is the {@link HalfMapFileReader}.
private int encodedRegionName;
private long fileid;
private byte [] splitkey;
private Range region;
private HStoreKey midkey;
/**
* For split HStoreFiles, it specifies if the file covers the lower half or
* the upper half of the key range
@ -52,66 +45,86 @@ public class Reference implements Writable {
/** HStoreFile contains lower half of key range */
bottom
}
public Reference(final int ern, final long fid, final HStoreKey m,
final Range fr) {
this.encodedRegionName = ern;
this.fileid = fid;
/**
* Constructor
* @param r
* @param s This is a serialized storekey with the row we are to split on,
* an empty column and a timestamp of the LATEST_TIMESTAMP. This is the first
* possible entry in a row. This is what we are splitting around.
* @param fr
*/
public Reference(final byte [] s, final Range fr) {
this.splitkey = s;
this.region = fr;
this.midkey = m;
}
public Reference() {
this(-1, -1, null, Range.bottom);
}
public long getFileId() {
return fileid;
/**
* Used by serializations.
*/
public Reference() {
this(null, Range.bottom);
}
public Range getFileRegion() {
return region;
}
public HStoreKey getMidkey() {
return midkey;
}
public int getEncodedRegionName() {
return this.encodedRegionName;
return this.region;
}
public byte [] getSplitKey() {
return splitkey;
}
@Override
public String toString() {
return encodedRegionName + "/" + fileid + "/" + region;
return "" + this.region;
}
// Make it serializable.
public void write(DataOutput out) throws IOException {
// Write out the encoded region name as a String. Doing it as a String
// keeps a Reference's serialization backword compatible with
// pre-HBASE-82 serializations. ALternative is rewriting all
// info files in hbase (Serialized References are written into the
// 'info' file that accompanies HBase Store files).
out.writeUTF(Integer.toString(encodedRegionName));
out.writeLong(fileid);
// Write true if we're doing top of the file.
out.writeBoolean(isTopFileRegion(region));
this.midkey.write(out);
out.writeBoolean(isTopFileRegion(this.region));
Bytes.writeByteArray(out, this.splitkey);
}
public void readFields(DataInput in) throws IOException {
this.encodedRegionName = Integer.parseInt(in.readUTF());
fileid = in.readLong();
boolean tmp = in.readBoolean();
// If true, set region to top.
region = tmp? Range.top: Range.bottom;
midkey = new HStoreKey();
midkey.readFields(in);
this.region = tmp? Range.top: Range.bottom;
this.splitkey = Bytes.readByteArray(in);
}
public static boolean isTopFileRegion(final Range r) {
return r.equals(Range.top);
}
public Path write(final FileSystem fs, final Path p)
throws IOException {
FSUtils.create(fs, p);
FSDataOutputStream out = fs.create(p);
try {
write(out);
} finally {
out.close();
}
return p;
}
/**
* Read a Reference from FileSystem.
* @param fs
* @param p
* @return New Reference made from passed <code>p</code>
* @throws IOException
*/
public static Reference read(final FileSystem fs, final Path p)
throws IOException {
FSDataInputStream in = fs.open(p);
try {
Reference r = new Reference();
r.readFields(in);
return r;
} finally {
in.close();
}
}
}

View File

@ -20,36 +20,34 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.ipc.RemoteException;
/**
@ -292,19 +290,16 @@ abstract class BaseScanner extends Chore implements HConstants {
if (split == null) {
return result;
}
Path tabledir = HTableDescriptor.getTableDir(this.master.rootdir,
split.getTableDesc().getName());
Path tabledir = new Path(this.master.rootdir, split.getTableDesc().getNameAsString());
for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
Path p = HStoreFile.getMapDir(tabledir, split.getEncodedName(),
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
family.getName());
// Look for reference files. Call listStatus with an anonymous
// instance of PathFilter.
FileStatus [] ps = this.master.fs.listStatus(p,
new PathFilter () {
public boolean accept(Path path) {
return HStore.isReference(path);
return StoreFile.isReference(path);
}
}
);

View File

@ -21,10 +21,10 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.regionserver.Store;
/** Instantiated to remove a column family from a table */
class DeleteColumn extends ColumnOperation {
@ -40,13 +40,14 @@ class DeleteColumn extends ColumnOperation {
@Override
protected void postProcessMeta(MetaRegion m, HRegionInterface server)
throws IOException {
Path tabledir = new Path(this.master.rootdir, tableName.toString());
for (HRegionInfo i: unservedRegions) {
i.getTableDesc().removeFamily(columnName);
updateRegionInfo(server, m.getRegionName(), i);
// Delete the directories used by the column
FSUtils.deleteColumnFamily(this.master.fs, tabledir, i.getEncodedName(),
this.columnName);
Path tabledir =
new Path(this.master.rootdir, i.getTableDesc().getNameAsString());
this.master.fs.delete(Store.getStoreHomedir(tabledir, i.getEncodedName(),
this.columnName), true);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.util.Bytes;
@ -88,8 +87,7 @@ public class MetaRegion implements Comparable<MetaRegion> {
public int compareTo(MetaRegion other) {
int result = Bytes.compareTo(this.regionName, other.getRegionName());
if(result == 0) {
result = HStoreKey.compareTwoRowKeys(HRegionInfo.FIRST_META_REGIONINFO,
this.startKey, other.getStartKey());
result = HStoreKey.compareTwoRowKeys(this.startKey, other.getStartKey());
if (result == 0) {
// Might be on different host?
result = this.server.compareTo(other.server);

View File

@ -655,7 +655,7 @@ public class HLog implements HConstants, Syncable {
}
synchronized (updateLock) {
this.writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId),
new HLogEdit(HLog.METACOLUMN, HLogEdit.completeCacheFlush.get(),
new HLogEdit(HLog.METACOLUMN, HLogEdit.COMPLETE_CACHE_FLUSH,
System.currentTimeMillis()));
this.numEntries++;
Long seq = this.lastSeqWritten.get(regionName);

View File

@ -20,11 +20,11 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.io.BatchOperation;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.*;
import java.io.*;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.HConstants;
@ -38,19 +38,15 @@ import org.apache.hadoop.hbase.HConstants;
public class HLogEdit implements Writable, HConstants {
/** Value stored for a deleted item */
public static ImmutableBytesWritable deleteBytes = null;
public static byte [] DELETED_BYTES = null;
/** Value written to HLog on a complete cache flush */
public static ImmutableBytesWritable completeCacheFlush = null;
public static byte [] COMPLETE_CACHE_FLUSH = null;
static {
try {
deleteBytes =
new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes(UTF8_ENCODING));
completeCacheFlush =
new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING));
DELETED_BYTES = "HBASE::DELETEVAL".getBytes(UTF8_ENCODING);
COMPLETE_CACHE_FLUSH = "HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING);
} catch (UnsupportedEncodingException e) {
assert(false);
}
@ -58,12 +54,31 @@ public class HLogEdit implements Writable, HConstants {
/**
* @param value
* @return True if an entry and its content is {@link #deleteBytes}.
* @return True if an entry and its content is {@link #DELETED_BYTES}.
*/
public static boolean isDeleted(final byte [] value) {
return (value == null)? false: deleteBytes.compareTo(value) == 0;
return isDeleted(value, 0, value.length);
}
/**
* @param value
* @return True if an entry and its content is {@link #DELETED_BYTES}.
*/
public static boolean isDeleted(final ByteBuffer value) {
return isDeleted(value.array(), value.arrayOffset(), value.limit());
}
/**
* @param value
* @return True if an entry and its content is {@link #DELETED_BYTES}.
*/
public static boolean isDeleted(final byte [] value, final int offset,
final int length) {
return (value == null)? false:
Bytes.BYTES_RAWCOMPARATOR.compare(DELETED_BYTES, 0, DELETED_BYTES.length,
value, offset, length) == 0;
}
/** If transactional log entry, these are the op codes */
public enum TransactionalOperation {
/** start transaction */

View File

@ -26,7 +26,6 @@ import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
@ -58,8 +57,8 @@ import org.apache.hadoop.hbase.io.BatchOperation;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.io.Reference.Range;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -105,10 +104,9 @@ import org.apache.hadoop.util.StringUtils;
* defines the keyspace for this HRegion.
*/
public class HRegion implements HConstants {
static final Log LOG = LogFactory.getLog(HRegion.class);
static final String SPLITDIR = "splits";
static final String MERGEDIR = "merges";
static final Random rand = new Random();
static final Log LOG = LogFactory.getLog(HRegion.class);
final AtomicBoolean closed = new AtomicBoolean(false);
/* Closing can take some time; use the closing flag if there is stuff we don't want
* to do while in closing state; e.g. like offer this region up to the master as a region
@ -125,11 +123,11 @@ public class HRegion implements HConstants {
new ConcurrentHashMap<Integer, byte []>();
private final Map<Integer, TreeMap<HStoreKey, byte []>> targetColumns =
new ConcurrentHashMap<Integer, TreeMap<HStoreKey, byte []>>();
// Default access because read by tests.
protected final Map<Integer, HStore> stores =
new ConcurrentHashMap<Integer, HStore>();
protected final Map<Integer, Store> stores =
new ConcurrentHashMap<Integer, Store>();
final AtomicLong memcacheSize = new AtomicLong(0);
// This is the table subdirectory.
final Path basedir;
final HLog log;
final FileSystem fs;
@ -137,7 +135,7 @@ public class HRegion implements HConstants {
final HRegionInfo regionInfo;
final Path regiondir;
private final Path regionCompactionDir;
/*
* Set this when scheduling compaction if want the next compaction to be a
* major compaction. Cleared each time through compaction code.
@ -158,7 +156,7 @@ public class HRegion implements HConstants {
// Gets set in close. If set, cannot compact or flush again.
volatile boolean writesEnabled = true;
// Set if region is read-only
private volatile boolean readOnly = false;
volatile boolean readOnly = false;
/**
* Set flags that make this region read-only.
@ -233,34 +231,33 @@ public class HRegion implements HConstants {
String encodedNameStr = Integer.toString(this.regionInfo.getEncodedName());
this.regiondir = new Path(basedir, encodedNameStr);
this.historian = RegionHistorian.getInstance();
if (LOG.isDebugEnabled()) {
// Write out region name as string and its encoded name.
LOG.debug("Opening region " + this + "/" +
this.regionInfo.getEncodedName());
}
this.regionCompactionDir =
new Path(getCompactionDir(basedir), encodedNameStr);
int flushSize = regionInfo.getTableDesc().getMemcacheFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE) {
flushSize = conf.getInt("hbase.hregion.memcache.flush.size",
HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE);
}
this.memcacheFlushSize = flushSize;
this.blockingMemcacheSize = this.memcacheFlushSize *
conf.getInt("hbase.hregion.memcache.block.multiplier", 1);
}
/** Initialize this region and get it ready to roll.
/**
* Initialize this region and get it ready to roll.
* Called after construction.
*
* @param initialFiles
* @param reporter
* @throws IOException
*/
public void initialize( Path initialFiles,
final Progressable reporter) throws IOException {
public void initialize( Path initialFiles, final Progressable reporter)
throws IOException {
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
// Move prefab HStore files into place (if any). This picks up split files
@ -273,19 +270,19 @@ public class HRegion implements HConstants {
long maxSeqId = -1;
long minSeqId = Integer.MAX_VALUE;
for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) {
HStore store = instantiateHStore(this.basedir, c, oldLogFile, reporter);
stores.put(Bytes.mapKey(c.getName()), store);
Store store = instantiateHStore(this.basedir, c, oldLogFile, reporter);
this.stores.put(Bytes.mapKey(c.getName()), store);
long storeSeqId = store.getMaxSequenceId();
if (storeSeqId > maxSeqId) {
maxSeqId = storeSeqId;
}
}
if (storeSeqId < minSeqId) {
minSeqId = storeSeqId;
}
}
// Play log if one. Delete when done.
doReconstructionLog(oldLogFile, minSeqId, maxSeqId, reporter);
if (fs.exists(oldLogFile)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Deleting old log file: " + oldLogFile);
@ -302,14 +299,9 @@ public class HRegion implements HConstants {
}
// Get rid of any splits or merges that were lost in-progress
Path splits = new Path(regiondir, SPLITDIR);
if (fs.exists(splits)) {
fs.delete(splits, true);
}
Path merges = new Path(regiondir, MERGEDIR);
if (fs.exists(merges)) {
fs.delete(merges, true);
}
FSUtils.deleteDirectory(this.fs, new Path(regiondir, SPLITDIR));
FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
// See if region is meant to run read-only.
if (this.regionInfo.getTableDesc().isReadOnly()) {
this.writestate.setReadOnly(true);
@ -346,7 +338,7 @@ public class HRegion implements HConstants {
public boolean isClosing() {
return this.closing.get();
}
/**
* Close down this HRegion. Flush the cache, shut down each HStore, don't
* service any more calls.
@ -360,10 +352,10 @@ public class HRegion implements HConstants {
*
* @throws IOException
*/
public List<HStoreFile> close() throws IOException {
public List<StoreFile> close() throws IOException {
return close(false);
}
/**
* Close down this HRegion. Flush the cache unless abort parameter is true,
* Shut down each HStore, don't service any more calls.
@ -378,7 +370,7 @@ public class HRegion implements HConstants {
*
* @throws IOException
*/
List<HStoreFile> close(boolean abort) throws IOException {
List<StoreFile> close(final boolean abort) throws IOException {
if (isClosed()) {
LOG.warn("region " + this + " already closed");
return null;
@ -433,8 +425,8 @@ public class HRegion implements HConstants {
internalFlushcache();
}
List<HStoreFile> result = new ArrayList<HStoreFile>();
for (HStore store: stores.values()) {
List<StoreFile> result = new ArrayList<StoreFile>();
for (Store store: stores.values()) {
result.addAll(store.close());
}
this.closed.set(true);
@ -513,7 +505,7 @@ public class HRegion implements HConstants {
/** @return returns size of largest HStore. */
public long getLargestHStoreSize() {
long size = 0;
for (HStore h: stores.values()) {
for (Store h: stores.values()) {
long storeSize = h.getSize();
if (storeSize > size) {
size = storeSize;
@ -521,17 +513,17 @@ public class HRegion implements HConstants {
}
return size;
}
/*
* Split the HRegion to create two brand-new ones. This also closes
* current HRegion. Split should be fast since we don't rewrite store files
* but instead create new 'reference' store files that read off the top and
* bottom ranges of parent store files.
* @param midKey key on which to split region
* @param splitRow row on which to split region
* @return two brand-new (and open) HRegions or null if a split is not needed
* @throws IOException
*/
HRegion[] splitRegion(final byte [] midKey) throws IOException {
HRegion[] splitRegion(final byte [] splitRow) throws IOException {
synchronized (splitLock) {
if (closed.get()) {
return null;
@ -539,11 +531,11 @@ public class HRegion implements HConstants {
// Add start/end key checking: hbase-428.
byte [] startKey = this.regionInfo.getStartKey();
byte [] endKey = this.regionInfo.getEndKey();
if (HStoreKey.equalsTwoRowKeys(this.regionInfo,startKey, midKey)) {
if (HStoreKey.equalsTwoRowKeys(startKey, splitRow)) {
LOG.debug("Startkey and midkey are same, not splitting");
return null;
}
if (HStoreKey.equalsTwoRowKeys(this.regionInfo,midKey, endKey)) {
if (HStoreKey.equalsTwoRowKeys(splitRow, endKey)) {
LOG.debug("Endkey and midkey are same, not splitting");
return null;
}
@ -561,14 +553,14 @@ public class HRegion implements HConstants {
rid = this.regionInfo.getRegionId() + 1;
}
HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
startKey, midKey, false, rid);
startKey, splitRow, false, rid);
Path dirA =
new Path(splits, Integer.toString(regionAInfo.getEncodedName()));
if(fs.exists(dirA)) {
throw new IOException("Cannot split; target file collision at " + dirA);
}
HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
midKey, endKey, false, rid);
splitRow, endKey, false, rid);
Path dirB =
new Path(splits, Integer.toString(regionBInfo.getEncodedName()));
if(this.fs.exists(dirB)) {
@ -578,38 +570,31 @@ public class HRegion implements HConstants {
// Now close the HRegion. Close returns all store files or null if not
// supposed to close (? What to do in this case? Implement abort of close?)
// Close also does wait on outstanding rows and calls a flush just-in-case.
List<HStoreFile> hstoreFilesToSplit = close(false);
List<StoreFile> hstoreFilesToSplit = close(false);
if (hstoreFilesToSplit == null) {
LOG.warn("Close came back null (Implement abort of close?)");
throw new RuntimeException("close returned empty vector of HStoreFiles");
}
// Split each store file.
for(HStoreFile h: hstoreFilesToSplit) {
// A reference to the bottom half of the hsf store file.
Reference aReference = new Reference(
this.regionInfo.getEncodedName(), h.getFileId(),
new HStoreKey(midKey, this.regionInfo), Reference.Range.bottom);
HStoreFile a = new HStoreFile(this.conf, fs, splits,
regionAInfo, h.getColFamily(), -1, aReference);
// Reference to top half of the hsf store file.
Reference bReference = new Reference(
this.regionInfo.getEncodedName(), h.getFileId(),
new HStoreKey(midKey, this.regionInfo), Reference.Range.top);
HStoreFile b = new HStoreFile(this.conf, fs, splits,
regionBInfo, h.getColFamily(), -1, bReference);
h.splitStoreFile(a, b, this.fs);
for(StoreFile h: hstoreFilesToSplit) {
StoreFile.split(fs,
Store.getStoreHomedir(splits, regionAInfo.getEncodedName(),
h.getFamily()),
h, splitRow, Range.bottom);
StoreFile.split(fs,
Store.getStoreHomedir(splits, regionBInfo.getEncodedName(),
h.getFamily()),
h, splitRow, Range.top);
}
// Done!
// Opening the region copies the splits files from the splits directory
// under each region.
HRegion regionA =
new HRegion(basedir, log, fs, conf, regionAInfo, null);
HRegion regionA = new HRegion(basedir, log, fs, conf, regionAInfo, null);
regionA.initialize(dirA, null);
regionA.close();
HRegion regionB =
new HRegion(basedir, log, fs, conf, regionBInfo, null);
HRegion regionB = new HRegion(basedir, log, fs, conf, regionBInfo, null);
regionB.initialize(dirB, null);
regionB.close();
@ -619,10 +604,8 @@ public class HRegion implements HConstants {
LOG.debug("Cleaned up " + FSUtils.getPath(splits) + " " + deleted);
}
HRegion regions[] = new HRegion [] {regionA, regionB};
this.historian.addRegionSplit(this.regionInfo,
regionA.getRegionInfo(), regionB.getRegionInfo());
return regions;
}
}
@ -649,15 +632,13 @@ public class HRegion implements HConstants {
* @throws IOException
*/
private void doRegionCompactionCleanup() throws IOException {
if (this.fs.exists(this.regionCompactionDir)) {
this.fs.delete(this.regionCompactionDir, true);
}
FSUtils.deleteDirectory(this.fs, this.regionCompactionDir);
}
void setForceMajorCompaction(final boolean b) {
this.forceMajorCompaction = b;
}
boolean getForceMajorCompaction() {
return this.forceMajorCompaction;
}
@ -694,16 +675,16 @@ public class HRegion implements HConstants {
* server does them sequentially and not in parallel.
*
* @param majorCompaction True to force a major compaction regardless of thresholds
* @return mid key if split is needed
* @return split row if split is needed
* @throws IOException
*/
byte [] compactStores(final boolean majorCompaction)
throws IOException {
splitsAndClosesLock.readLock().lock();
try {
byte [] midKey = null;
byte [] splitRow = null;
if (this.closed.get()) {
return midKey;
return splitRow;
}
try {
synchronized (writestate) {
@ -713,7 +694,7 @@ public class HRegion implements HConstants {
LOG.info("NOT compacting region " + this +
": compacting=" + writestate.compacting + ", writesEnabled=" +
writestate.writesEnabled);
return midKey;
return splitRow;
}
}
LOG.info("starting " + (majorCompaction? "major" : "") +
@ -721,11 +702,11 @@ public class HRegion implements HConstants {
long startTime = System.currentTimeMillis();
doRegionCompactionPrep();
long maxSize = -1;
for (HStore store: stores.values()) {
final HStore.StoreSize size = store.compact(majorCompaction);
for (Store store: stores.values()) {
final Store.StoreSize size = store.compact(majorCompaction);
if (size != null && size.getSize() > maxSize) {
maxSize = size.getSize();
midKey = size.getKey();
splitRow = size.getSplitRow();
}
}
doRegionCompactionCleanup();
@ -739,7 +720,7 @@ public class HRegion implements HConstants {
writestate.notifyAll();
}
}
return midKey;
return splitRow;
} finally {
splitsAndClosesLock.readLock().unlock();
}
@ -859,7 +840,7 @@ public class HRegion implements HConstants {
// Get current size of memcaches.
final long currentMemcacheSize = this.memcacheSize.get();
try {
for (HStore s: stores.values()) {
for (Store s: stores.values()) {
s.snapshot();
}
sequenceId = log.startCacheFlush();
@ -877,7 +858,7 @@ public class HRegion implements HConstants {
// A. Flush memcache to all the HStores.
// Keep running vector of all store files that includes both old and the
// just-made new flush store file.
for (HStore hstore: stores.values()) {
for (Store hstore: stores.values()) {
boolean needsCompaction = hstore.flushCache(completeSequenceId);
if (needsCompaction) {
compactionRequested = true;
@ -971,7 +952,7 @@ public class HRegion implements HConstants {
checkRow(row);
checkColumn(column);
// Don't need a row lock for a simple get
HStoreKey key = new HStoreKey(row, column, timestamp, this.regionInfo);
HStoreKey key = new HStoreKey(row, column, timestamp);
Cell[] result = getStore(column).get(key, numVersions);
// Guarantee that we return null instead of a zero-length array,
// if there are no results to return.
@ -1009,16 +990,16 @@ public class HRegion implements HConstants {
checkColumn(column);
}
}
HStoreKey key = new HStoreKey(row, ts, this.regionInfo);
HStoreKey key = new HStoreKey(row, ts);
Integer lid = getLock(lockid,row);
HashSet<HStore> storeSet = new HashSet<HStore>();
HashSet<Store> storeSet = new HashSet<Store>();
try {
HbaseMapWritable<byte [], Cell> result =
new HbaseMapWritable<byte [], Cell>();
// Get the concerned columns or all of them
if (columns != null) {
for (byte[] bs : columns) {
HStore store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs)));
Store store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs)));
if (store != null) {
storeSet.add(store);
}
@ -1033,14 +1014,14 @@ public class HRegion implements HConstants {
if (columns != null) {
for (byte[] bs : columns) {
if (HStoreKey.getFamilyDelimiterIndex(bs) == (bs.length - 1)) {
HStore store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs)));
Store store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs)));
store.getFull(key, null, numVersions, result);
storeSet.remove(store);
}
}
}
for (HStore targetStore: storeSet) {
for (Store targetStore: storeSet) {
targetStore.getFull(key, columns, numVersions, result);
}
@ -1083,17 +1064,17 @@ public class HRegion implements HConstants {
checkRow(row);
splitsAndClosesLock.readLock().lock();
try {
HStore store = getStore(columnFamily);
Store store = getStore(columnFamily);
// get the closest key. (HStore.getRowKeyAtOrBefore can return null)
byte [] closestKey = store.getRowKeyAtOrBefore(row);
// If it happens to be an exact match, we can stop.
// Otherwise, we need to check if it's the max and move to the next
if (closestKey != null) {
if (HStoreKey.equalsTwoRowKeys(regionInfo, row, closestKey)) {
key = new HStoreKey(closestKey, this.regionInfo);
if (HStoreKey.equalsTwoRowKeys(row, closestKey)) {
key = new HStoreKey(closestKey);
}
if (key == null) {
key = new HStoreKey(closestKey, this.regionInfo);
key = new HStoreKey(closestKey);
}
}
if (key == null) {
@ -1124,16 +1105,16 @@ public class HRegion implements HConstants {
private Set<HStoreKey> getKeys(final HStoreKey origin, final int versions)
throws IOException {
Set<HStoreKey> keys = new TreeSet<HStoreKey>();
Collection<HStore> storesToCheck = null;
Collection<Store> storesToCheck = null;
if (origin.getColumn() == null || origin.getColumn().length == 0) {
// All families
storesToCheck = this.stores.values();
} else {
storesToCheck = new ArrayList<HStore>(1);
storesToCheck = new ArrayList<Store>(1);
storesToCheck.add(getStore(origin.getColumn()));
}
long now = System.currentTimeMillis();
for (HStore targetStore: storesToCheck) {
for (Store targetStore: storesToCheck) {
if (targetStore != null) {
// Pass versions without modification since in the store getKeys, it
// includes the size of the passed <code>keys</code> array when counting.
@ -1170,15 +1151,15 @@ public class HRegion implements HConstants {
if (this.closed.get()) {
throw new IOException("Region " + this + " closed");
}
HashSet<HStore> storeSet = new HashSet<HStore>();
HashSet<Store> storeSet = new HashSet<Store>();
for (int i = 0; i < cols.length; i++) {
HStore s = stores.get(Bytes.mapKey(HStoreKey.getFamily(cols[i])));
Store s = stores.get(Bytes.mapKey(HStoreKey.getFamily(cols[i])));
if (s != null) {
storeSet.add(s);
}
}
return new HScanner(cols, firstRow, timestamp,
storeSet.toArray(new HStore [storeSet.size()]), filter);
storeSet.toArray(new Store [storeSet.size()]), filter);
} finally {
newScannerLock.readLock().unlock();
}
@ -1246,8 +1227,7 @@ public class HRegion implements HConstants {
try {
List<byte []> deletes = null;
for (BatchOperation op: b) {
HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime,
this.regionInfo);
HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime);
byte[] val = null;
if (op.isPut()) {
val = op.getValue();
@ -1262,7 +1242,7 @@ public class HRegion implements HConstants {
}
deletes.add(op.getColumn());
} else {
val = HLogEdit.deleteBytes.get();
val = HLogEdit.DELETED_BYTES;
}
}
if (val != null) {
@ -1339,8 +1319,7 @@ public class HRegion implements HConstants {
System.currentTimeMillis(): b.getTimestamp();
List<byte []> deletes = null;
for (BatchOperation op: b) {
HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime,
this.regionInfo);
HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime);
byte[] val = null;
if (op.isPut()) {
val = op.getValue();
@ -1355,7 +1334,7 @@ public class HRegion implements HConstants {
}
deletes.add(op.getColumn());
} else {
val = HLogEdit.deleteBytes.get();
val = HLogEdit.DELETED_BYTES;
}
}
if (val != null) {
@ -1460,14 +1439,14 @@ public class HRegion implements HConstants {
Integer lid = getLock(lockid, row);
long now = System.currentTimeMillis();
try {
for (HStore store : stores.values()) {
for (Store store : stores.values()) {
List<HStoreKey> keys =
store.getKeys(new HStoreKey(row, ts, this.regionInfo),
store.getKeys(new HStoreKey(row, ts),
ALL_VERSIONS, now, null);
TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
for (HStoreKey key: keys) {
edits.put(key, HLogEdit.deleteBytes.get());
edits.put(key, HLogEdit.DELETED_BYTES);
}
update(edits);
}
@ -1494,14 +1473,14 @@ public class HRegion implements HConstants {
Integer lid = getLock(lockid, row);
long now = System.currentTimeMillis();
try {
for (HStore store : stores.values()) {
for (Store store : stores.values()) {
List<HStoreKey> keys =
store.getKeys(new HStoreKey(row, timestamp, this.regionInfo),
store.getKeys(new HStoreKey(row, timestamp),
ALL_VERSIONS, now, columnPattern);
TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
for (HStoreKey key: keys) {
edits.put(key, HLogEdit.deleteBytes.get());
edits.put(key, HLogEdit.DELETED_BYTES);
}
update(edits);
}
@ -1529,15 +1508,15 @@ public class HRegion implements HConstants {
long now = System.currentTimeMillis();
try {
// find the HStore for the column family
HStore store = getStore(family);
Store store = getStore(family);
// find all the keys that match our criteria
List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp,
this.regionInfo), ALL_VERSIONS, now, null);
List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp),
ALL_VERSIONS, now, null);
// delete all the cells
TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
for (HStoreKey key: keys) {
edits.put(key, HLogEdit.deleteBytes.get());
edits.put(key, HLogEdit.DELETED_BYTES);
}
update(edits);
} finally {
@ -1565,18 +1544,18 @@ public class HRegion implements HConstants {
Integer lid = getLock(lockid, row);
long now = System.currentTimeMillis();
try {
for(HStore store : stores.values()) {
for(Store store: stores.values()) {
String familyName = Bytes.toString(store.getFamily().getName());
// check the family name match the family pattern.
if(!(familyPattern.matcher(familyName).matches()))
continue;
List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp,
this.regionInfo), ALL_VERSIONS, now, null);
List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp),
ALL_VERSIONS, now, null);
TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
for (HStoreKey key: keys) {
edits.put(key, HLogEdit.deleteBytes.get());
edits.put(key, HLogEdit.DELETED_BYTES);
}
update(edits);
}
@ -1601,13 +1580,13 @@ public class HRegion implements HConstants {
final long ts, final int versions)
throws IOException {
checkReadOnly();
HStoreKey origin = new HStoreKey(row, column, ts, this.regionInfo);
HStoreKey origin = new HStoreKey(row, column, ts);
Set<HStoreKey> keys = getKeys(origin, versions);
if (keys.size() > 0) {
TreeMap<HStoreKey, byte []> edits = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
for (HStoreKey key: keys) {
edits.put(key, HLogEdit.deleteBytes.get());
edits.put(key, HLogEdit.DELETED_BYTES);
}
update(edits);
}
@ -1672,7 +1651,7 @@ public class HRegion implements HConstants {
TreeMap<HStoreKey, byte []> targets = this.targetColumns.get(lockid);
if (targets == null) {
targets = new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(regionInfo));
new HStoreKey.HStoreKeyWritableComparator());
this.targetColumns.put(lockid, targets);
}
targets.put(key, val);
@ -1759,10 +1738,10 @@ public class HRegion implements HConstants {
// Nothing to do (Replaying is done in HStores)
}
protected HStore instantiateHStore(Path baseDir,
protected Store instantiateHStore(Path baseDir,
HColumnDescriptor c, Path oldLogFile, Progressable reporter)
throws IOException {
return new HStore(baseDir, this.regionInfo, c, this.fs, oldLogFile,
return new Store(baseDir, this.regionInfo, c, this.fs, oldLogFile,
this.conf, reporter);
}
@ -1773,7 +1752,7 @@ public class HRegion implements HConstants {
* @return Store that goes with the family on passed <code>column</code>.
* TODO: Make this lookup faster.
*/
public HStore getStore(final byte [] column) {
public Store getStore(final byte [] column) {
return this.stores.get(HStoreKey.getFamilyMapKey(column));
}
@ -1962,7 +1941,7 @@ public class HRegion implements HConstants {
/** Create an HScanner with a handle on many HStores. */
@SuppressWarnings("unchecked")
HScanner(byte [][] cols, byte [] firstRow, long timestamp, HStore[] stores,
HScanner(byte [][] cols, byte [] firstRow, long timestamp, Store [] stores,
RowFilterInterface filter)
throws IOException {
this.filter = filter;
@ -2004,7 +1983,7 @@ public class HRegion implements HConstants {
this.resultSets = new TreeMap[scanners.length];
this.keys = new HStoreKey[scanners.length];
for (int i = 0; i < scanners.length; i++) {
keys[i] = new HStoreKey(HConstants.EMPTY_BYTE_ARRAY,regionInfo);
keys[i] = new HStoreKey(HConstants.EMPTY_BYTE_ARRAY);
resultSets[i] = new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
closeScanner(i);
@ -2016,7 +1995,6 @@ public class HRegion implements HConstants {
activeScannerCount.incrementAndGet();
}
@SuppressWarnings("null")
public boolean next(HStoreKey key, SortedMap<byte [], Cell> results)
throws IOException {
boolean moreToFollow = false;
@ -2029,10 +2007,8 @@ public class HRegion implements HConstants {
for (int i = 0; i < this.keys.length; i++) {
if (scanners[i] != null &&
(chosenRow == null ||
(HStoreKey.compareTwoRowKeys(regionInfo,
keys[i].getRow(), chosenRow) < 0) ||
((HStoreKey.compareTwoRowKeys(regionInfo, keys[i].getRow(),
chosenRow) == 0) &&
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) < 0) ||
((HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) == 0) &&
(keys[i].getTimestamp() > chosenTimestamp)))) {
chosenRow = keys[i].getRow();
chosenTimestamp = keys[i].getTimestamp();
@ -2049,7 +2025,7 @@ public class HRegion implements HConstants {
for (int i = 0; i < scanners.length; i++) {
if (scanners[i] != null &&
HStoreKey.compareTwoRowKeys(regionInfo,keys[i].getRow(), chosenRow) == 0) {
HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) == 0) {
// NOTE: We used to do results.putAll(resultSets[i]);
// but this had the effect of overwriting newer
// values with older ones. So now we only insert
@ -2071,7 +2047,7 @@ public class HRegion implements HConstants {
// If the current scanner is non-null AND has a lower-or-equal
// row label, then its timestamp is bad. We need to advance it.
while ((scanners[i] != null) &&
(HStoreKey.compareTwoRowKeys(regionInfo,keys[i].getRow(), chosenRow) <= 0)) {
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), chosenRow) <= 0)) {
resultSets[i].clear();
if (!scanners[i].next(keys[i], resultSets[i])) {
closeScanner(i);
@ -2193,8 +2169,8 @@ public class HRegion implements HConstants {
RegionHistorian.getInstance().addRegionCreation(info);
}
HRegion region = new HRegion(tableDir,
new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
fs, conf, info, null);
new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
fs, conf, info, null);
region.initialize(null, null);
return region;
}
@ -2250,9 +2226,9 @@ public class HRegion implements HConstants {
Integer lid = meta.obtainRowLock(row);
try {
HStoreKey key = new HStoreKey(row, COL_REGIONINFO,
System.currentTimeMillis(), r.getRegionInfo());
System.currentTimeMillis());
TreeMap<HStoreKey, byte[]> edits = new TreeMap<HStoreKey, byte[]>(
new HStoreKey.HStoreKeyWritableComparator(meta.getRegionInfo()));
new HStoreKey.HStoreKeyWritableComparator());
edits.put(key, Writables.getBytes(r.getRegionInfo()));
meta.update(edits);
} finally {
@ -2336,7 +2312,9 @@ public class HRegion implements HConstants {
if (LOG.isDebugEnabled()) {
LOG.debug("DELETING region " + regiondir.toString());
}
fs.delete(regiondir, true);
if (!fs.delete(regiondir, true)) {
LOG.warn("Failed delete of " + regiondir);
}
}
/**
@ -2373,28 +2351,29 @@ public class HRegion implements HConstants {
*/
public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
return ((info.getStartKey().length == 0) ||
(HStoreKey.compareTwoRowKeys(info,info.getStartKey(), row) <= 0)) &&
(HStoreKey.compareTwoRowKeys(info.getStartKey(), row) <= 0)) &&
((info.getEndKey().length == 0) ||
(HStoreKey.compareTwoRowKeys(info,info.getEndKey(), row) > 0));
(HStoreKey.compareTwoRowKeys(info.getEndKey(), row) > 0));
}
/**
* Make the directories for a specific column family
*
* @param fs the file system
* @param basedir base directory where region will live (usually the table dir)
* @param encodedRegionName encoded region name
* @param tabledir base directory where region will live (usually the table dir)
* @param hri
* @param colFamily the column family
* @param tabledesc table descriptor of table
* @throws IOException
*/
public static void makeColumnFamilyDirs(FileSystem fs, Path basedir,
int encodedRegionName, byte [] colFamily, HTableDescriptor tabledesc)
public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
final HRegionInfo hri, byte [] colFamily)
throws IOException {
fs.mkdirs(HStoreFile.getMapDir(basedir, encodedRegionName, colFamily));
fs.mkdirs(HStoreFile.getInfoDir(basedir, encodedRegionName, colFamily));
Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
if (!fs.mkdirs(dir)) {
LOG.warn("Failed to create " + dir);
}
}
/**
* Merge two HRegions. The regions must be adjacent andmust not overlap.
*
@ -2416,15 +2395,13 @@ public class HRegion implements HConstants {
throw new IOException("Cannot merge two regions with null start key");
}
// A's start key is null but B's isn't. Assume A comes before B
} else if ((srcB.getStartKey() == null) // A is not null but B is
|| (HStoreKey.compareTwoRowKeys(srcA.getRegionInfo(),
srcA.getStartKey(), srcB.getStartKey()) > 0)) { // A > B
} else if ((srcB.getStartKey() == null) ||
(HStoreKey.compareTwoRowKeys(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
a = srcB;
b = srcA;
}
if (!HStoreKey.equalsTwoRowKeys(srcA.getRegionInfo(),
a.getEndKey(), b.getStartKey())) {
if (!HStoreKey.equalsTwoRowKeys(a.getEndKey(), b.getStartKey())) {
throw new IOException("Cannot merge non-adjacent regions");
}
return merge(a, b);
@ -2468,20 +2445,17 @@ public class HRegion implements HConstants {
HTableDescriptor tabledesc = a.getTableDesc();
HLog log = a.getLog();
Path basedir = a.getBaseDir();
final byte [] startKey = HStoreKey.equalsTwoRowKeys(a.getRegionInfo(),
a.getStartKey(), EMPTY_BYTE_ARRAY) ||
HStoreKey.equalsTwoRowKeys(a.getRegionInfo(),
b.getStartKey(), EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY :
HStoreKey.compareTwoRowKeys(a.getRegionInfo(), a.getStartKey(),
b.getStartKey()) <= 0 ?
a.getStartKey() : b.getStartKey();
final byte [] endKey = HStoreKey.equalsTwoRowKeys(a.getRegionInfo(),
a.getEndKey(), EMPTY_BYTE_ARRAY) ||
HStoreKey.equalsTwoRowKeys(b.getRegionInfo(), b.getEndKey(),
EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY :
HStoreKey.compareTwoRowKeys(a.getRegionInfo(), a.getEndKey(),
b.getEndKey()) <= 0 ?
b.getEndKey() : a.getEndKey();
final byte [] startKey = HStoreKey.equalsTwoRowKeys(a.getStartKey(),
EMPTY_BYTE_ARRAY) ||
HStoreKey.equalsTwoRowKeys(b.getStartKey(), EMPTY_BYTE_ARRAY)?
EMPTY_BYTE_ARRAY: HStoreKey.compareTwoRowKeys(a.getStartKey(),
b.getStartKey()) <= 0?
a.getStartKey(): b.getStartKey();
final byte [] endKey = HStoreKey.equalsTwoRowKeys(a.getEndKey(),
EMPTY_BYTE_ARRAY) ||
HStoreKey.equalsTwoRowKeys(b.getEndKey(), EMPTY_BYTE_ARRAY)?
EMPTY_BYTE_ARRAY:
HStoreKey.compareTwoRowKeys(a.getEndKey(), b.getEndKey()) <= 0? b.getEndKey(): a.getEndKey();
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
LOG.info("Creating new region " + newRegionInfo.toString());
@ -2499,37 +2473,31 @@ public class HRegion implements HConstants {
// Move HStoreFiles under new region directory
Map<byte [], List<HStoreFile>> byFamily =
new TreeMap<byte [], List<HStoreFile>>(Bytes.BYTES_COMPARATOR);
Map<byte [], List<StoreFile>> byFamily =
new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
byFamily = filesByFamily(byFamily, a.close());
byFamily = filesByFamily(byFamily, b.close());
for (Map.Entry<byte [], List<HStoreFile>> es : byFamily.entrySet()) {
for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
byte [] colFamily = es.getKey();
makeColumnFamilyDirs(fs, basedir, encodedName, colFamily, tabledesc);
makeColumnFamilyDirs(fs, basedir, newRegionInfo, colFamily);
// Because we compacted the source regions we should have no more than two
// HStoreFiles per family and there will be no reference store
List<HStoreFile> srcFiles = es.getValue();
List<StoreFile> srcFiles = es.getValue();
if (srcFiles.size() == 2) {
long seqA = srcFiles.get(0).loadInfo(fs);
long seqB = srcFiles.get(1).loadInfo(fs);
long seqA = srcFiles.get(0).getMaxSequenceId();
long seqB = srcFiles.get(1).getMaxSequenceId();
if (seqA == seqB) {
// We can't have duplicate sequence numbers
if (LOG.isDebugEnabled()) {
LOG.debug("Adjusting sequence id of storeFile " + srcFiles.get(1) +
" down by one; sequence id A=" + seqA + ", sequence id B=" +
seqB);
}
srcFiles.get(1).writeInfo(fs, seqB - 1);
// Can't have same sequenceid since on open of a store, this is what
// distingushes the files (see the map of stores how its keyed by
// sequenceid).
throw new IOException("Files have same sequenceid");
}
}
for (HStoreFile hsf: srcFiles) {
HStoreFile dst = new HStoreFile(conf, fs, basedir,
newRegionInfo, colFamily, -1, null);
if (LOG.isDebugEnabled()) {
LOG.debug("Renaming " + hsf + " to " + dst);
}
hsf.rename(fs, dst);
for (StoreFile hsf: srcFiles) {
StoreFile.rename(fs, hsf.getPath(),
StoreFile.getUniqueFile(fs, Store.getStoreHomedir(basedir,
newRegionInfo.getEncodedName(), colFamily)));
}
}
if (LOG.isDebugEnabled()) {
@ -2555,15 +2523,17 @@ public class HRegion implements HConstants {
* Fills a map with a vector of store files keyed by column family.
* @param byFamily Map to fill.
* @param storeFiles Store files to process.
* @param family
* @return Returns <code>byFamily</code>
*/
private static Map<byte [], List<HStoreFile>> filesByFamily(
Map<byte [], List<HStoreFile>> byFamily, List<HStoreFile> storeFiles) {
for (HStoreFile src: storeFiles) {
List<HStoreFile> v = byFamily.get(src.getColFamily());
private static Map<byte [], List<StoreFile>> filesByFamily(
Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
for (StoreFile src: storeFiles) {
byte [] family = src.getFamily();
List<StoreFile> v = byFamily.get(family);
if (v == null) {
v = new ArrayList<HStoreFile>();
byFamily.put(src.getColFamily(), v);
v = new ArrayList<StoreFile>();
byFamily.put(family, v);
}
v.add(src);
}
@ -2582,7 +2552,7 @@ public class HRegion implements HConstants {
* @throws IOException
*/
boolean isMajorCompaction() throws IOException {
for (HStore store: this.stores.values()) {
for (Store store: this.stores.values()) {
if (store.isMajorCompaction()) {
return true;
}

View File

@ -647,7 +647,7 @@ public class HRegionServer implements HConstants, HRegionInterface, HBaseRPCErro
int storefileIndexSizeMB = 0;
synchronized (r.stores) {
stores += r.stores.size();
for (HStore store: r.stores.values()) {
for (Store store: r.stores.values()) {
storefiles += store.getStorefilesCount();
storefileIndexSizeMB +=
(int)(store.getStorefilesIndexSize()/1024/1024);
@ -955,8 +955,8 @@ public class HRegionServer implements HConstants, HRegionInterface, HBaseRPCErro
memcacheSize += r.memcacheSize.get();
synchronized (r.stores) {
stores += r.stores.size();
for(Map.Entry<Integer, HStore> ee: r.stores.entrySet()) {
HStore store = ee.getValue();
for(Map.Entry<Integer, Store> ee: r.stores.entrySet()) {
Store store = ee.getValue();
storefiles += store.getStorefilesCount();
try {
storefileIndexSize += store.getStorefilesIndexSize();

View File

@ -40,7 +40,6 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.util.Bytes;
@ -57,8 +56,6 @@ class Memcache {
private static final Log LOG = LogFactory.getLog(Memcache.class);
private final long ttl;
private HRegionInfo regionInfo;
// Note that since these structures are always accessed with a lock held,
// so no additional synchronization is required.
@ -76,8 +73,6 @@ class Memcache {
*/
public Memcache() {
this.ttl = HConstants.FOREVER;
// Set default to be the first meta region.
this.regionInfo = HRegionInfo.FIRST_META_REGIONINFO;
this.memcache = createSynchronizedSortedMap();
this.snapshot = createSynchronizedSortedMap();
}
@ -87,21 +82,21 @@ class Memcache {
* @param ttl The TTL for cache entries, in milliseconds.
* @param regionInfo The HRI for this cache
*/
public Memcache(final long ttl, HRegionInfo regionInfo) {
public Memcache(final long ttl) {
this.ttl = ttl;
this.regionInfo = regionInfo;
this.memcache = createSynchronizedSortedMap();
this.snapshot = createSynchronizedSortedMap();
}
/*
* Utility method using HSKWritableComparator
* @return sycnhronized sorted map of HStoreKey to byte arrays.
* @return synchronized sorted map of HStoreKey to byte arrays.
*/
@SuppressWarnings("unchecked")
private SortedMap<HStoreKey, byte[]> createSynchronizedSortedMap() {
return Collections.synchronizedSortedMap(
new TreeMap<HStoreKey, byte []>(
new HStoreKey.HStoreKeyWritableComparator(this.regionInfo)));
new HStoreKey.HStoreKeyWritableComparator()));
}
/**
@ -266,7 +261,7 @@ class Memcache {
if (b == null) {
return a;
}
return HStoreKey.compareTwoRowKeys(regionInfo, a, b) <= 0? a: b;
return HStoreKey.compareTwoRowKeys(a, b) <= 0? a: b;
}
/**
@ -296,12 +291,12 @@ class Memcache {
synchronized (map) {
// Make an HSK with maximum timestamp so we get past most of the current
// rows cell entries.
HStoreKey hsk = new HStoreKey(row, HConstants.LATEST_TIMESTAMP, this.regionInfo);
HStoreKey hsk = new HStoreKey(row, HConstants.LATEST_TIMESTAMP);
SortedMap<HStoreKey, byte []> tailMap = map.tailMap(hsk);
// Iterate until we fall into the next row; i.e. move off current row
for (Map.Entry<HStoreKey, byte []> es: tailMap.entrySet()) {
HStoreKey itKey = es.getKey();
if (HStoreKey.compareTwoRowKeys(regionInfo, itKey.getRow(), row) <= 0)
if (HStoreKey.compareTwoRowKeys(itKey.getRow(), row) <= 0)
continue;
// Note: Not suppressing deletes or expired cells.
result = itKey.getRow();
@ -372,8 +367,7 @@ class Memcache {
}
}
}
} else if (HStoreKey.compareTwoRowKeys(regionInfo, key.getRow(),
itKey.getRow()) < 0) {
} else if (HStoreKey.compareTwoRowKeys(key.getRow(), itKey.getRow()) < 0) {
break;
}
}
@ -422,8 +416,8 @@ class Memcache {
// We want the earliest possible to start searching from. Start before
// the candidate key in case it turns out a delete came in later.
HStoreKey search_key = candidateKeys.isEmpty()?
new HStoreKey(row, this.regionInfo):
new HStoreKey(candidateKeys.firstKey().getRow(), this.regionInfo);
new HStoreKey(row):
new HStoreKey(candidateKeys.firstKey().getRow());
List<HStoreKey> victims = new ArrayList<HStoreKey>();
long now = System.currentTimeMillis();
@ -434,8 +428,8 @@ class Memcache {
// the search key, or a range of values between the first candidate key
// and the ultimate search key (or the end of the cache)
if (!tailMap.isEmpty() &&
HStoreKey.compareTwoRowKeys(this.regionInfo,
tailMap.firstKey().getRow(), search_key.getRow()) <= 0) {
HStoreKey.compareTwoRowKeys(tailMap.firstKey().getRow(),
search_key.getRow()) <= 0) {
Iterator<HStoreKey> key_iterator = tailMap.keySet().iterator();
// Keep looking at cells as long as they are no greater than the
@ -443,18 +437,16 @@ class Memcache {
HStoreKey deletedOrExpiredRow = null;
for (HStoreKey found_key = null; key_iterator.hasNext() &&
(found_key == null ||
HStoreKey.compareTwoRowKeys(this.regionInfo,
found_key.getRow(), row) <= 0);) {
HStoreKey.compareTwoRowKeys(found_key.getRow(), row) <= 0);) {
found_key = key_iterator.next();
if (HStoreKey.compareTwoRowKeys(this.regionInfo,
found_key.getRow(), row) <= 0) {
if (HStoreKey.compareTwoRowKeys(found_key.getRow(), row) <= 0) {
if (HLogEdit.isDeleted(tailMap.get(found_key))) {
HStore.handleDeleted(found_key, candidateKeys, deletes);
Store.handleDeleted(found_key, candidateKeys, deletes);
if (deletedOrExpiredRow == null) {
deletedOrExpiredRow = found_key;
}
} else {
if (HStore.notExpiredAndNotInDeletes(this.ttl,
if (Store.notExpiredAndNotInDeletes(this.ttl,
found_key, now, deletes)) {
candidateKeys.put(stripTimestamp(found_key),
new Long(found_key.getTimestamp()));
@ -515,15 +507,15 @@ class Memcache {
// not a delete record.
boolean deleted = HLogEdit.isDeleted(headMap.get(found_key));
if (lastRowFound != null &&
!HStoreKey.equalsTwoRowKeys(this.regionInfo, lastRowFound,
found_key.getRow()) && !deleted) {
!HStoreKey.equalsTwoRowKeys(lastRowFound, found_key.getRow()) &&
!deleted) {
break;
}
// If this isn't a delete, record it as a candidate key. Also
// take note of the row of this candidate so that we'll know when
// we cross the row boundary into the previous row.
if (!deleted) {
if (HStore.notExpiredAndNotInDeletes(this.ttl, found_key, now, deletes)) {
if (Store.notExpiredAndNotInDeletes(this.ttl, found_key, now, deletes)) {
lastRowFound = found_key.getRow();
candidateKeys.put(stripTimestamp(found_key),
new Long(found_key.getTimestamp()));
@ -543,12 +535,12 @@ class Memcache {
// smaller acceptable candidate keys would have caused us to start
// our search earlier in the list, and we wouldn't be searching here.
SortedMap<HStoreKey, byte[]> thisRowTailMap =
headMap.tailMap(new HStoreKey(headMap.lastKey().getRow(), this.regionInfo));
headMap.tailMap(new HStoreKey(headMap.lastKey().getRow()));
Iterator<HStoreKey> key_iterator = thisRowTailMap.keySet().iterator();
do {
HStoreKey found_key = key_iterator.next();
if (HLogEdit.isDeleted(thisRowTailMap.get(found_key))) {
HStore.handleDeleted(found_key, candidateKeys, deletes);
Store.handleDeleted(found_key, candidateKeys, deletes);
} else {
if (ttl == HConstants.FOREVER ||
now < found_key.getTimestamp() + ttl ||
@ -568,7 +560,7 @@ class Memcache {
}
static HStoreKey stripTimestamp(HStoreKey key) {
return new HStoreKey(key.getRow(), key.getColumn(), key.getHRegionInfo());
return new HStoreKey(key.getRow(), key.getColumn());
}
/*
@ -595,7 +587,7 @@ class Memcache {
if (itKey.matchesRowCol(key)) {
if (!isDeleted(es.getValue())) {
// Filter out expired results
if (HStore.notExpiredAndNotInDeletes(ttl, itKey, now, deletes)) {
if (Store.notExpiredAndNotInDeletes(ttl, itKey, now, deletes)) {
result.add(new Cell(tailMap.get(itKey), itKey.getTimestamp()));
if (numVersions > 0 && result.size() >= numVersions) {
break;
@ -692,8 +684,7 @@ class Memcache {
if (origin.getColumn() != null && origin.getColumn().length == 0) {
// if the current and origin row don't match, then we can jump
// out of the loop entirely.
if (!HStoreKey.equalsTwoRowKeys(regionInfo, key.getRow(),
origin.getRow())) {
if (!HStoreKey.equalsTwoRowKeys( key.getRow(), origin.getRow())) {
break;
}
// if the column pattern is not null, we use it for column matching.
@ -716,7 +707,7 @@ class Memcache {
}
}
if (!isDeleted(es.getValue())) {
if (HStore.notExpiredAndNotInDeletes(this.ttl, key, now, deletes)) {
if (Store.notExpiredAndNotInDeletes(this.ttl, key, now, deletes)) {
result.add(key);
if (versions > 0 && result.size() >= versions) {
break;
@ -777,7 +768,7 @@ class Memcache {
private class MemcacheScanner extends HAbstractScanner {
private byte [] currentRow;
private Set<byte []> columns = null;
MemcacheScanner(final long timestamp, final byte [] targetCols[],
final byte [] firstRow)
throws IOException {
@ -828,7 +819,7 @@ class Memcache {
rowResults);
for (Map.Entry<byte [], Long> e: deletes.entrySet()) {
rowResults.put(e.getKey(),
new Cell(HLogEdit.deleteBytes.get(), e.getValue().longValue()));
new Cell(HLogEdit.DELETED_BYTES, e.getValue().longValue()));
}
for (Map.Entry<byte [], Cell> e: rowResults.entrySet()) {
byte [] column = e.getKey();

View File

@ -21,15 +21,18 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.io.MapFile;
/**
* A scanner that iterates through HStore files
@ -39,13 +42,13 @@ implements ChangedReadersObserver {
// Keys retrieved from the sources
private volatile HStoreKey keys[];
// Values that correspond to those keys
private volatile byte [][] vals;
private ByteBuffer [] vals;
// Readers we go against.
private volatile MapFile.Reader[] readers;
private volatile HFileScanner [] scanners;
// Store this scanner came out of.
private final HStore store;
private final Store store;
// Used around replacement of Readers if they change while we're scanning.
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
@ -57,14 +60,14 @@ implements ChangedReadersObserver {
* @param firstRow
* @throws IOException
*/
public StoreFileScanner(final HStore store, final long timestamp,
public StoreFileScanner(final Store store, final long timestamp,
final byte [][] targetCols, final byte [] firstRow)
throws IOException {
super(timestamp, targetCols);
this.store = store;
this.store.addChangedReaderObserver(this);
try {
openReaders(firstRow);
openScanner(firstRow);
} catch (Exception ex) {
close();
IOException e = new IOException("HStoreScanner failed construction");
@ -74,34 +77,23 @@ implements ChangedReadersObserver {
}
/*
* Go open new Reader iterators and cue them at <code>firstRow</code>.
* Go open new scanners and cue them at <code>firstRow</code>.
* Closes existing Readers if any.
* @param firstRow
* @throws IOException
*/
private void openReaders(final byte [] firstRow) throws IOException {
if (this.readers != null) {
for (int i = 0; i < this.readers.length; i++) {
if (this.readers[i] != null) {
this.readers[i].close();
}
}
private void openScanner(final byte [] firstRow) throws IOException {
List<HFileScanner> s =
new ArrayList<HFileScanner>(this.store.getStorefiles().size());
Map<Long, StoreFile> map = this.store.getStorefiles().descendingMap();
for (StoreFile f: map.values()) {
s.add(f.getReader().getScanner());
}
// Open our own copies of the Readers here inside in the scanner.
this.readers = new MapFile.Reader[this.store.getStorefiles().size()];
// Most recent map file should be first
int i = readers.length - 1;
for(HStoreFile curHSF: store.getStorefiles().values()) {
readers[i--] = curHSF.getReader(store.fs, false, false);
}
this.keys = new HStoreKey[readers.length];
this.vals = new byte[readers.length][];
this.scanners = s.toArray(new HFileScanner [] {});
this.keys = new HStoreKey[this.scanners.length];
this.vals = new ByteBuffer[this.scanners.length];
// Advance the readers to the first pos.
for (i = 0; i < readers.length; i++) {
keys[i] = new HStoreKey(HConstants.EMPTY_BYTE_ARRAY, this.store.getHRegionInfo());
for (int i = 0; i < this.scanners.length; i++) {
if (firstRow != null && firstRow.length != 0) {
if (findFirstRow(i, firstRow)) {
continue;
@ -158,9 +150,9 @@ implements ChangedReadersObserver {
for (int i = 0; i < keys.length; i++) {
// Fetch the data
while ((keys[i] != null)
&& (HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), viableRow.getRow()) == 0)) {
while ((keys[i] != null) &&
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(),
viableRow.getRow()) == 0)) {
// If we are doing a wild card match or there are multiple matchers
// per column, we need to scan all the older versions of this row
@ -184,12 +176,11 @@ implements ChangedReadersObserver {
closeSubScanner(i);
}
}
// Advance the current scanner beyond the chosen row, to
// a valid timestamp, so we're ready next time.
while ((keys[i] != null)
&& ((HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), viableRow.getRow()) <= 0)
while ((keys[i] != null) &&
((HStoreKey.compareTwoRowKeys(this.keys[i].getRow(),
viableRow.getRow()) <= 0)
|| (keys[i].getTimestamp() > this.timestamp)
|| (! columnMatch(i)))) {
getNext(i);
@ -231,7 +222,7 @@ implements ChangedReadersObserver {
long viableTimestamp = -1;
long now = System.currentTimeMillis();
long ttl = store.ttl;
for(int i = 0; i < keys.length; i++) {
for (int i = 0; i < keys.length; i++) {
// The first key that we find that matches may have a timestamp greater
// than the one we're looking for. We have to advance to see if there
// is an older version present, since timestamps are sorted descending
@ -247,12 +238,10 @@ implements ChangedReadersObserver {
// If we get here and keys[i] is not null, we already know that the
// column matches and the timestamp of the row is less than or equal
// to this.timestamp, so we do not need to test that here
&& ((viableRow == null)
|| (HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), viableRow) < 0)
|| ((HStoreKey.compareTwoRowKeys(store.getHRegionInfo(),
keys[i].getRow(), viableRow) == 0)
&& (keys[i].getTimestamp() > viableTimestamp)))) {
&& ((viableRow == null) ||
(HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), viableRow) < 0) ||
((HStoreKey.compareTwoRowKeys(this.keys[i].getRow(), viableRow) == 0) &&
(keys[i].getTimestamp() > viableTimestamp)))) {
if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) {
viableRow = keys[i].getRow();
viableTimestamp = keys[i].getTimestamp();
@ -266,7 +255,7 @@ implements ChangedReadersObserver {
return new ViableRow(viableRow, viableTimestamp);
}
/**
/*
* The user didn't want to start scanning at the first row. This method
* seeks to the requested row.
*
@ -275,28 +264,30 @@ implements ChangedReadersObserver {
* @return true if this is the first row or if the row was not found
*/
private boolean findFirstRow(int i, final byte [] firstRow) throws IOException {
ImmutableBytesWritable ibw = new ImmutableBytesWritable();
HStoreKey firstKey
= (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow, this.store.getHRegionInfo()), ibw);
if (firstKey == null) {
// Didn't find it. Close the scanner and return TRUE
closeSubScanner(i);
return true;
if (firstRow == null || firstRow.length <= 0) {
if (!this.scanners[i].seekTo()) {
closeSubScanner(i);
return true;
}
} else {
if (!Store.getClosest(this.scanners[i],
new HStoreKey(firstRow).getBytes())) {
closeSubScanner(i);
return true;
}
}
this.keys[i] = HStoreKey.create(this.scanners[i].getKey());
this.vals[i] = this.scanners[i].getValue();
long now = System.currentTimeMillis();
long ttl = store.ttl;
if (ttl != HConstants.FOREVER && now >= firstKey.getTimestamp() + ttl) {
if (ttl != HConstants.FOREVER && now >= this.keys[i].getTimestamp() + ttl) {
// Didn't find it. Close the scanner and return TRUE
closeSubScanner(i);
return true;
}
this.vals[i] = ibw.get();
keys[i].setRow(firstKey.getRow());
keys[i].setColumn(firstKey.getColumn());
keys[i].setVersion(firstKey.getTimestamp());
return columnMatch(i);
}
/**
* Get the next value from the specified reader.
*
@ -305,17 +296,18 @@ implements ChangedReadersObserver {
*/
private boolean getNext(int i) throws IOException {
boolean result = false;
ImmutableBytesWritable ibw = new ImmutableBytesWritable();
long now = System.currentTimeMillis();
long ttl = store.ttl;
while (true) {
if (!readers[i].next(keys[i], ibw)) {
if ((this.scanners[i].isSeeked() && !this.scanners[i].next()) ||
(!this.scanners[i].isSeeked() && !this.scanners[i].seekTo())) {
closeSubScanner(i);
break;
}
this.keys[i] = HStoreKey.create(this.scanners[i].getKey());
if (keys[i].getTimestamp() <= this.timestamp) {
if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) {
vals[i] = ibw.get();
vals[i] = this.scanners[i].getValue();
result = true;
break;
}
@ -326,23 +318,12 @@ implements ChangedReadersObserver {
}
return result;
}
/** Close down the indicated reader. */
private void closeSubScanner(int i) {
try {
if(readers[i] != null) {
try {
readers[i].close();
} catch(IOException e) {
LOG.error(store.storeName + " closing sub-scanner", e);
}
}
} finally {
readers[i] = null;
keys[i] = null;
vals[i] = null;
}
this.scanners[i] = null;
this.keys[i] = null;
this.vals[i] = null;
}
/** Shut it down! */
@ -350,16 +331,9 @@ implements ChangedReadersObserver {
if (!this.scannerClosed) {
this.store.deleteChangedReaderObserver(this);
try {
for(int i = 0; i < readers.length; i++) {
if(readers[i] != null) {
try {
readers[i].close();
} catch(IOException e) {
LOG.error(store.storeName + " closing scanner", e);
}
}
for(int i = 0; i < this.scanners.length; i++) {
closeSubScanner(i);
}
} finally {
this.scannerClosed = true;
}
@ -375,7 +349,7 @@ implements ChangedReadersObserver {
// the current row as 'first' row and readers will be opened and cue'd
// up so future call to next will start here.
ViableRow viableRow = getNextViableRow();
openReaders(viableRow.getRow());
openScanner(viableRow.getRow());
LOG.debug("Replaced Scanner Readers at row " +
(viableRow == null || viableRow.getRow() == null? "null":
Bytes.toString(viableRow.getRow())));
@ -383,4 +357,4 @@ implements ChangedReadersObserver {
this.lock.writeLock().unlock();
}
}
}
}

View File

@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.transactional.TransactionState.Status;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.Progressable;
@ -369,7 +369,7 @@ public class TransactionalRegion extends HRegion {
TransactionState state = getTransactionState(transactionId);
long now = System.currentTimeMillis();
for (HStore store : super.stores.values()) {
for (Store store : super.stores.values()) {
List<HStoreKey> keys = store.getKeys(new HStoreKey(row, timestamp),
ALL_VERSIONS, now, null);
BatchUpdate deleteUpdate = new BatchUpdate(row, timestamp);

View File

@ -8,6 +8,8 @@ import java.nio.ByteBuffer;
import java.util.Comparator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.WritableUtils;
@ -43,16 +45,37 @@ public class Bytes {
*/
// JHat says BU is 56 bytes.
public static final int ESTIMATED_HEAP_TAX = 16;
/**
* Byte array comparator class.
* Does byte ordering.
*/
public static class ByteArrayComparator implements RawComparator<byte []> {
public ByteArrayComparator() {
super();
}
@Override
public int compare(byte [] left, byte [] right) {
return compareTo(left, right);
}
@Override
public int compare(byte [] b1, int s1, int l1, byte [] b2, int s2, int l2) {
return compareTo(b1, s1, l1, b2, s2, l2);
}
}
/**
* Pass this to TreeMaps where byte [] are keys.
*/
public static Comparator<byte []> BYTES_COMPARATOR =
new Comparator<byte []>() {
public int compare(byte [] left, byte [] right) {
return compareTo(left, right);
}
};
new ByteArrayComparator();
/**
* Pass this to TreeMaps where byte [] are keys.
*/
public static RawComparator<byte []> BYTES_RAWCOMPARATOR =
new ByteArrayComparator();
/**
* @param in Input to read from.
@ -70,6 +93,18 @@ public class Bytes {
return result;
}
/**
* @param in Input to read from.
* @return byte array read off <code>in</code>
*/
public static byte [] readByteArrayThrowsRuntime(final DataInput in) {
try {
return readByteArray(in);
} catch (Exception e) {
throw new RuntimeException(e);
}
}
/**
* @param out
* @param b
@ -80,22 +115,69 @@ public class Bytes {
WritableUtils.writeVInt(out, b.length);
out.write(b, 0, b.length);
}
/**
* Reads a zero-compressed encoded long from input stream and returns it.
* @param buffer Binary array
* @param offset Offset into array at which vint begins.
* @throws java.io.IOException
* @return deserialized long from stream.
*/
public static long readVLong(final byte [] buffer, final int offset)
throws IOException {
byte firstByte = buffer[offset];
int len = WritableUtils.decodeVIntSize(firstByte);
if (len == 1) {
return firstByte;
}
long i = 0;
for (int idx = 0; idx < len-1; idx++) {
byte b = buffer[offset + 1 + idx];
i = i << 8;
i = i | (b & 0xFF);
}
return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
}
/**
* @param b Presumed UTF-8 encoded byte array.
* @return String made from <code>b</code>
*/
public static String toString(final byte [] b) {
return toString(b, 0, b.length);
}
public static String toString(final byte [] b, int off, int len) {
String result = null;
try {
result = new String(b, HConstants.UTF8_ENCODING);
result = new String(b, off, len, HConstants.UTF8_ENCODING);
} catch (UnsupportedEncodingException e) {
e.printStackTrace();
}
return result;
}
/**
* @param b
* @return <code>b</code> encoded in a byte array.
*/
public static byte [] toBytes(final boolean b) {
byte [] bb = new byte[1];
bb[0] = b? (byte)-1: (byte)0;
return bb;
}
/**
* @param b
* @return True or false.
*/
public static boolean toBoolean(final byte [] b) {
if (b == null || b.length > 1) {
throw new IllegalArgumentException("Array is wrong size");
}
return b[0] != (byte)0;
}
/**
* Converts a string to a UTF-8 byte array.
* @param s
@ -113,6 +195,17 @@ public class Bytes {
}
return result;
}
/**
* @param bb
* @return Byte array represented by passed <code>bb</code>
*/
public static byte [] toBytes(final ByteBuffer bb) {
int length = bb.limit();
byte [] result = new byte[length];
System.arraycopy(bb.array(), bb.arrayOffset(), result, 0, length);
return result;
}
/**
* Convert a long value to a byte array
@ -159,7 +252,7 @@ public class Bytes {
}
return ByteBuffer.wrap(bytes).getInt();
}
/**
* Convert an float value to a byte array
* @param val
@ -237,6 +330,7 @@ public class Bytes {
* @return True if equal
*/
public static boolean equals(final byte [] left, final byte [] right) {
// Could use Arrays.equals?
return left == null && right == null? true:
(left == null || right == null || (left.length != right.length))? false:
compareTo(left, right) == 0;
@ -337,4 +431,4 @@ public class Bytes {
result[0] = column;
return result;
}
}
}

View File

@ -27,17 +27,14 @@ import java.net.URISyntaxException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hdfs.DistributedFileSystem;
/**
* Utility methods for interacting with the underlying file system.
@ -51,7 +48,50 @@ public class FSUtils {
private FSUtils() {
super();
}
/**
* Delete if exists.
* @param fs
* @param dir
* @return True if deleted <code>dir</code>
* @throws IOException
*/
public static boolean deleteDirectory(final FileSystem fs, final Path dir)
throws IOException {
return fs.exists(dir)? fs.delete(dir, true): false;
}
/**
* Check if directory exists. If it does not, create it.
* @param dir
* @return
* @throws IOException
*/
public Path checkdir(final FileSystem fs, final Path dir) throws IOException {
if (!fs.exists(dir)) {
fs.mkdirs(dir);
}
return dir;
}
/**
* Create file.
* @param fs
* @param p
* @return
* @throws IOException
*/
public static Path create(final FileSystem fs, final Path p)
throws IOException {
if (fs.exists(p)) {
throw new IOException("File already exists " + p.toString());
}
if (!fs.createNewFile(p)) {
throw new IOException("Failed create of " + p);
}
return p;
}
/**
* Checks to see if the specified file system is available
*
@ -178,21 +218,6 @@ public class FSUtils {
return p.toUri().getPath();
}
/**
* Delete the directories used by the column family under the passed region.
* @param fs Filesystem to use.
* @param tabledir The directory under hbase.rootdir for this table.
* @param encodedRegionName The region name encoded.
* @param family Family to delete.
* @throws IOException
*/
public static void deleteColumnFamily(final FileSystem fs,
final Path tabledir, final int encodedRegionName, final byte [] family)
throws IOException {
fs.delete(HStoreFile.getMapDir(tabledir, encodedRegionName, family), true);
fs.delete(HStoreFile.getInfoDir(tabledir, encodedRegionName, family), true);
}
/**
* @param c
* @return Path to hbase root directory: i.e. <code>hbase.rootdir</code> as a

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.regionserver.HLog;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
/**
* Contains utility methods for manipulating HBase meta tables.
@ -358,7 +359,6 @@ public class MetaUtils {
public void deleteColumn(final byte [] tableName,
final byte [] columnFamily) throws IOException {
List<HRegionInfo> metas = getMETARows(tableName);
final Path tabledir = new Path(rootdir, Bytes.toString(tableName));
for (HRegionInfo hri: metas) {
final HRegion m = getMetaRegion(hri);
scanMetaRegion(m, new ScannerListener() {
@ -370,8 +370,13 @@ public class MetaUtils {
this.inTable = false;
info.getTableDesc().removeFamily(columnFamily);
updateMETARegionInfo(m, info);
FSUtils.deleteColumnFamily(fs, tabledir, info.getEncodedName(),
HStoreKey.getFamily(columnFamily));
Path tabledir = new Path(rootdir,
info.getTableDesc().getNameAsString());
Path p = Store.getStoreHomedir(tabledir, info.getEncodedName(),
columnFamily);
if (!fs.delete(p, true)) {
LOG.warn("Failed delete of " + p);
}
return false;
}
// If we got here and we have not yet encountered the table yet,

View File

@ -71,7 +71,27 @@ public class Writables {
*/
public static Writable getWritable(final byte [] bytes, final Writable w)
throws IOException {
if (bytes == null || bytes.length == 0) {
return getWritable(bytes, 0, bytes.length, w);
}
/**
* Set bytes into the passed Writable by calling its
* {@link Writable#readFields(java.io.DataInput)}.
* @param bytes
* @param offset
* @param length
* @param w An empty Writable (usually made by calling the null-arg
* constructor).
* @return The passed Writable after its readFields has been called fed
* by the passed <code>bytes</code> array or IllegalArgumentException
* if passed null or an empty <code>bytes</code> array.
* @throws IOException
* @throws IllegalArgumentException
*/
public static Writable getWritable(final byte [] bytes, final int offset,
final int length, final Writable w)
throws IOException {
if (bytes == null || length <=0) {
throw new IllegalArgumentException("Can't build a writable with empty " +
"bytes array");
}
@ -80,7 +100,7 @@ public class Writables {
}
DataInputBuffer in = new DataInputBuffer();
try {
in.reset(bytes, bytes.length);
in.reset(bytes, offset, length);
w.readFields(in);
return w;
} finally {

View File

@ -209,8 +209,9 @@ public abstract class HBaseTestCase extends TestCase {
* @param r
* @param column
* @throws IOException
* @return count of what we added.
*/
protected static void addContent(final HRegion r, final byte [] column)
protected static long addContent(final HRegion r, final byte [] column)
throws IOException {
byte [] startKey = r.getRegionInfo().getStartKey();
byte [] endKey = r.getRegionInfo().getEndKey();
@ -218,7 +219,7 @@ public abstract class HBaseTestCase extends TestCase {
if (startKeyBytes == null || startKeyBytes.length == 0) {
startKeyBytes = START_KEY_BYTES;
}
addContent(new HRegionIncommon(r), Bytes.toString(column),
return addContent(new HRegionIncommon(r), Bytes.toString(column),
startKeyBytes, endKey, -1);
}
@ -229,10 +230,11 @@ public abstract class HBaseTestCase extends TestCase {
* @param updater An instance of {@link Incommon}.
* @param column
* @throws IOException
* @return count of what we added.
*/
protected static void addContent(final Incommon updater, final String column)
protected static long addContent(final Incommon updater, final String column)
throws IOException {
addContent(updater, column, START_KEY_BYTES, null);
return addContent(updater, column, START_KEY_BYTES, null);
}
/**
@ -243,12 +245,13 @@ public abstract class HBaseTestCase extends TestCase {
* @param column
* @param startKeyBytes Where to start the rows inserted
* @param endKey Where to stop inserting rows.
* @return count of what we added.
* @throws IOException
*/
protected static void addContent(final Incommon updater, final String column,
protected static long addContent(final Incommon updater, final String column,
final byte [] startKeyBytes, final byte [] endKey)
throws IOException {
addContent(updater, column, startKeyBytes, endKey, -1);
return addContent(updater, column, startKeyBytes, endKey, -1);
}
/**
@ -260,11 +263,13 @@ public abstract class HBaseTestCase extends TestCase {
* @param startKeyBytes Where to start the rows inserted
* @param endKey Where to stop inserting rows.
* @param ts Timestamp to write the content with.
* @return count of what we added.
* @throws IOException
*/
protected static void addContent(final Incommon updater, final String column,
protected static long addContent(final Incommon updater, final String column,
final byte [] startKeyBytes, final byte [] endKey, final long ts)
throws IOException {
long count = 0;
// Add rows of three characters. The first character starts with the
// 'a' character and runs up to 'z'. Per first character, we run the
// second character over same range. And same for the third so rows
@ -287,6 +292,7 @@ public abstract class HBaseTestCase extends TestCase {
try {
batchUpdate.put(column, t);
updater.commit(batchUpdate);
count++;
} catch (RuntimeException ex) {
ex.printStackTrace();
throw ex;
@ -307,6 +313,7 @@ public abstract class HBaseTestCase extends TestCase {
}
secondCharStart = FIRST_CHAR;
}
return count;
}
/**
@ -448,13 +455,7 @@ public abstract class HBaseTestCase extends TestCase {
public static class HTableIncommon implements Incommon {
final HTable table;
private BatchUpdate batch;
private void checkBatch() {
if (batch == null) {
throw new IllegalStateException("No batch update in progress.");
}
}
/**
* @param table
*/

View File

@ -22,16 +22,17 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.math.random.RandomData;
import org.apache.commons.math.random.RandomDataImpl;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.io.MapFile;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.WritableComparable;
/**
* <p>
@ -39,12 +40,20 @@ import org.apache.commons.logging.LogFactory;
* </p>
*/
public class MapFilePerformanceEvaluation {
private static final int ROW_LENGTH = 1000;
private static final int ROW_COUNT = 1000000;
private final HBaseConfiguration conf;
private static final int ROW_LENGTH = 10;
private static final int ROW_COUNT = 100000;
static final Log LOG =
LogFactory.getLog(MapFilePerformanceEvaluation.class.getName());
/**
* @param c
*/
public MapFilePerformanceEvaluation(final HBaseConfiguration c) {
super();
this.conf = c;
}
static ImmutableBytesWritable format(final int i, ImmutableBytesWritable w) {
String v = Integer.toString(i);
@ -52,23 +61,55 @@ public class MapFilePerformanceEvaluation {
return w;
}
private void runBenchmarks() throws Exception {
Configuration conf = new Configuration();
FileSystem fs = FileSystem.get(conf);
Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
private void runBenchmarks(final String[] args) throws Exception {
final FileSystem fs = FileSystem.get(this.conf);
final Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
if (fs.exists(mf)) {
fs.delete(mf, true);
}
runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new UniformRandomSmallScan(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
PerformanceEvaluationCommons.concurrentReads(new Runnable() {
public void run() {
try {
runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
ROW_COUNT);
} catch (Exception e) {
e.printStackTrace();
}
}
});
}
private void runBenchmark(RowOrientedBenchmark benchmark, int rowCount)
@ -200,6 +241,7 @@ public class MapFilePerformanceEvaluation {
}
static class SequentialReadBenchmark extends ReadBenchmark {
ImmutableBytesWritable verify = new ImmutableBytesWritable();
public SequentialReadBenchmark(Configuration conf, FileSystem fs,
Path mf, int totalRows) {
@ -208,7 +250,10 @@ public class MapFilePerformanceEvaluation {
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
reader.next(key, value);
this.reader.next(key, value);
PerformanceEvaluationCommons.assertKey(this.key.get(),
format(i, this.verify).get());
PerformanceEvaluationCommons.assertValueSize(ROW_LENGTH, value.getSize());
}
@Override
@ -229,7 +274,9 @@ public class MapFilePerformanceEvaluation {
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
reader.get(getRandomRow(), value);
ImmutableBytesWritable k = getRandomRow();
ImmutableBytesWritable r = (ImmutableBytesWritable)reader.get(k, value);
PerformanceEvaluationCommons.assertValueSize(r.getSize(), ROW_LENGTH);
}
private ImmutableBytesWritable getRandomRow() {
@ -238,8 +285,36 @@ public class MapFilePerformanceEvaluation {
}
static class GaussianRandomReadBenchmark extends ReadBenchmark {
static class UniformRandomSmallScan extends ReadBenchmark {
private Random random = new Random();
public UniformRandomSmallScan(Configuration conf, FileSystem fs,
Path mf, int totalRows) {
super(conf, fs, mf, totalRows/10);
}
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
ImmutableBytesWritable ibw = getRandomRow();
WritableComparable<?> wc = this.reader.getClosest(ibw, this.value);
if (wc == null) {
throw new NullPointerException();
}
PerformanceEvaluationCommons.assertKey(ibw.get(),
((ImmutableBytesWritable)wc).get());
// TODO: Verify we're getting right values.
for (int ii = 0; ii < 29; ii++) {
this.reader.next(this.key, this.value);
PerformanceEvaluationCommons.assertValueSize(this.value.getSize(), ROW_LENGTH);
}
}
private ImmutableBytesWritable getRandomRow() {
return format(random.nextInt(totalRows), key);
}
}
static class GaussianRandomReadBenchmark extends ReadBenchmark {
private RandomData randomData = new RandomDataImpl();
public GaussianRandomReadBenchmark(Configuration conf, FileSystem fs,
@ -249,7 +324,9 @@ public class MapFilePerformanceEvaluation {
@Override
void doRow(@SuppressWarnings("unused") int i) throws Exception {
reader.get(getGaussianRandomRow(), value);
ImmutableBytesWritable k = getGaussianRandomRow();
ImmutableBytesWritable r = (ImmutableBytesWritable)reader.get(k, value);
PerformanceEvaluationCommons.assertValueSize(r.getSize(), ROW_LENGTH);
}
private ImmutableBytesWritable getGaussianRandomRow() {
@ -258,13 +335,13 @@ public class MapFilePerformanceEvaluation {
}
}
/**
* @param args
* @throws IOException
*/
public static void main(String[] args) throws Exception {
new MapFilePerformanceEvaluation().runBenchmarks();
new MapFilePerformanceEvaluation(new HBaseConfiguration()).
runBenchmarks(args);
}
}

View File

@ -765,7 +765,6 @@ public class PerformanceEvaluation implements HConstants {
*/
public static void main(final String[] args) {
HBaseConfiguration c = new HBaseConfiguration();
System.exit(new PerformanceEvaluation(c).
doCommandLine(args));
System.exit(new PerformanceEvaluation(c).doCommandLine(args));
}
}

View File

@ -258,4 +258,4 @@ public class TestClassMigration extends TestCase {
return s.toString().getBytes(HConstants.UTF8_ENCODING);
}
}
}
}

View File

@ -54,8 +54,8 @@ public class TestCompare extends TestCase {
// Test null keys.
HStoreKey normal = new HStoreKey("a", "b");
assertTrue(normal.compareTo(null) > 0);
assertTrue(HStoreKey.compareTo(null, null, null) == 0);
assertTrue(HStoreKey.compareTo(null, null, normal) < 0);
assertTrue(HStoreKey.compareTo(null, null) == 0);
assertTrue(HStoreKey.compareTo(null, normal) < 0);
}
/**
@ -63,6 +63,7 @@ public class TestCompare extends TestCase {
* See HBASE-832
*/
public void testHStoreKeyBorderCases() {
/** TODO!!!!
HRegionInfo info = new HRegionInfo(new HTableDescriptor("testtable"),
HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
HStoreKey rowA = new HStoreKey("testtable,www.hbase.org/,1234",
@ -92,6 +93,7 @@ public class TestCompare extends TestCase {
"", Long.MAX_VALUE, HRegionInfo.ROOT_REGIONINFO);
assertTrue(rowA.compareTo(rowB) > 0);
*/
}

View File

@ -22,7 +22,6 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.io.BatchUpdate;

View File

@ -81,11 +81,9 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
// created above into the meta
createRootAndMetaRegions();
for (HRegion region : regions) {
HRegion.addRegionToMETA(meta, region);
}
closeRootAndMeta();
}
@ -208,16 +206,11 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
*/
private HRegion createAregion(byte [] startKey, byte [] endKey)
throws IOException {
HRegion region = createNewHRegion(desc, startKey, endKey);
byte [] keyToWrite = startKey == null ? Bytes.toBytes("row_000") : startKey;
BatchUpdate bu = new BatchUpdate(keyToWrite);
bu.put(COLUMN_NAME, "test".getBytes());
region.batchUpdate(bu, null);
region.close();
region.getLog().closeAndDelete();
return region;

View File

@ -24,6 +24,8 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scanner;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
@ -88,15 +90,23 @@ public class TestScanMultipleVersions extends HBaseClusterTestCase {
public void testScanMultipleVersions() throws Exception {
// At this point we have created multiple regions and both HDFS and HBase
// are running. There are 5 cases we have to test. Each is described below.
HTable t = new HTable(conf, TABLE_NAME);
for (int i = 0; i < ROWS.length; i++) {
for (int j = 0; j < TIMESTAMPS.length; j++) {
Cell [] cells =
t.get(ROWS[i], HConstants.COLUMN_FAMILY, TIMESTAMPS[j], 1);
assertTrue(cells != null && cells.length == 1);
System.out.println("Row=" + Bytes.toString(ROWS[i]) + ", cell=" +
cells[0]);
}
}
// Case 1: scan with LATEST_TIMESTAMP. Should get two rows
int count = 0;
Scanner s = t.getScanner(HConstants.COLUMN_FAMILY_ARRAY);
try {
while (s.next() != null) {
for (RowResult rr = null; (rr = s.next()) != null;) {
System.out.println(rr.toString());
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);

View File

@ -86,14 +86,5 @@ public class TestForceSplit extends HBaseClusterTestCase {
System.out.println("Regions after split (" + m.size() + "): " + m);
// should have two regions now
assertTrue(m.size() == 2);
// and "mmi" should be the midpoint
for (HRegionInfo hri: m.keySet()) {
byte[] start = hri.getStartKey();
byte[] end = hri.getEndKey();
if (Bytes.equals(start, HConstants.EMPTY_BYTE_ARRAY))
assertTrue(Bytes.equals(end, key_mmi));
if (Bytes.equals(end, key_mmi))
assertTrue(Bytes.equals(start, HConstants.EMPTY_BYTE_ARRAY));
}
}
}

View File

@ -189,7 +189,7 @@ public class TestRegExpRowFilter extends TestCase {
// that maps to a null value.
// Testing row with columnKeys: a-e, e maps to null
colvalues.put(new byte [] {LAST_CHAR},
new Cell(HLogEdit.deleteBytes.get(), HConstants.LATEST_TIMESTAMP));
new Cell(HLogEdit.DELETED_BYTES, HConstants.LATEST_TIMESTAMP));
assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.",
filter.filterRow(colvalues));
}

View File

@ -21,19 +21,16 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.io.MapFile;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
/**
* Test compactions
@ -136,12 +133,12 @@ public class TestCompaction extends HBaseTestCase {
// they were deleted.
int count = 0;
boolean containsStartRow = false;
for (MapFile.Reader reader: this.r.stores.
get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getReaders()) {
reader.reset();
HStoreKey key = new HStoreKey();
ImmutableBytesWritable val = new ImmutableBytesWritable();
while(reader.next(key, val)) {
for (StoreFile f: this.r.stores.
get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getStorefiles().values()) {
HFileScanner scanner = f.getReader().getScanner();
scanner.seekTo();
do {
HStoreKey key = HStoreKey.create(scanner.getKey());
if (Bytes.equals(key.getRow(), STARTROW)) {
containsStartRow = true;
count++;
@ -150,13 +147,13 @@ public class TestCompaction extends HBaseTestCase {
// file.
assertFalse(Bytes.equals(key.getRow(), secondRowBytes));
}
}
} while(scanner.next());
}
assertTrue(containsStartRow);
assertTrue(count == 3);
// Do a simple TTL test.
final int ttlInSeconds = 1;
for (HStore store: this.r.stores.values()) {
for (Store store: this.r.stores.values()) {
store.ttl = ttlInSeconds * 1000;
}
Thread.sleep(ttlInSeconds * 1000);
@ -167,14 +164,15 @@ public class TestCompaction extends HBaseTestCase {
private int count() throws IOException {
int count = 0;
for (MapFile.Reader reader: this.r.stores.
get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getReaders()) {
reader.reset();
HStoreKey key = new HStoreKey();
ImmutableBytesWritable val = new ImmutableBytesWritable();
while(reader.next(key, val)) {
count++;
for (StoreFile f: this.r.stores.
get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getStorefiles().values()) {
HFileScanner scanner = f.getReader().getScanner();
if (!scanner.seekTo()) {
continue;
}
do {
count++;
} while(scanner.next());
}
return count;
}

View File

@ -61,6 +61,175 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
this.miniHdfs.getFileSystem().getHomeDirectory().toString());
}
public void testGetFullMultiMapfile() throws IOException {
HRegion region = null;
BatchUpdate batchUpdate = null;
Map<byte [], Cell> results = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
// Test ordering issue
//
byte [] row = Bytes.toBytes("row1");
// write some data
batchUpdate = new BatchUpdate(row);
batchUpdate.put(COLUMNS[0], "olderValue".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert that getFull gives us the older value
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue()));
// write a new value for the cell
batchUpdate = new BatchUpdate(row);
batchUpdate.put(COLUMNS[0], "newerValue".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert that getFull gives us the later value
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue()));
//
// Test the delete masking issue
//
byte [] row2 = Bytes.toBytes("row2");
byte [] cell1 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
byte [] cell2 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
byte [] cell3 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
// write some data at two columns
batchUpdate = new BatchUpdate(row2);
batchUpdate.put(cell1, "column0 value".getBytes());
batchUpdate.put(cell2, "column1 value".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert i get both columns
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("Should have two columns in the results map", 2, results.size());
assertEquals("column0 value", new String(results.get(cell1).getValue()));
assertEquals("column1 value", new String(results.get(cell2).getValue()));
// write a delete for the first column
batchUpdate = new BatchUpdate(row2);
batchUpdate.delete(cell1);
batchUpdate.put(cell2, "column1 new value".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert i get the second column only
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
System.out.println(Bytes.toString(results.keySet().iterator().next()));
assertEquals("Should have one column in the results map", 1, results.size());
assertNull("column0 value", results.get(cell1));
assertEquals("column1 new value", new String(results.get(cell2).getValue()));
//
// Include a delete and value from the memcache in the mix
//
batchUpdate = new BatchUpdate(row2);
batchUpdate.delete(cell2);
batchUpdate.put(cell3, "column3 value!".getBytes());
region.batchUpdate(batchUpdate, null);
// assert i get the third column only
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("Should have one column in the results map", 1, results.size());
assertNull("column0 value", results.get(cell1));
assertNull("column1 value", results.get(cell2));
assertEquals("column3 value!", new String(results.get(cell3).getValue()));
} finally {
if (region != null) {
try {
region.close();
} catch (Exception e) {
e.printStackTrace();
}
region.getLog().closeAndDelete();
}
}
}
/** For HBASE-694 */
public void testGetClosestRowBefore2() throws IOException{
HRegion region = null;
BatchUpdate batchUpdate = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
// set up some test data
String t10 = "010";
String t20 = "020";
String t30 = "030";
String t40 = "040";
batchUpdate = new BatchUpdate(t10);
batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
batchUpdate = new BatchUpdate(t30);
batchUpdate.put(COLUMNS[0], "t30 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
batchUpdate = new BatchUpdate(t40);
batchUpdate.put(COLUMNS[0], "t40 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
// try finding "035"
String t35 = "035";
Map<byte [], Cell> results =
region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
region.flushcache();
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
batchUpdate = new BatchUpdate(t20);
batchUpdate.put(COLUMNS[0], "t20 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
region.flushcache();
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
} finally {
if (region != null) {
try {
region.close();
} catch (Exception e) {
e.printStackTrace();
}
region.getLog().closeAndDelete();
}
}
}
/**
* Test for HBASE-808 and HBASE-809.
* @throws Exception
@ -349,13 +518,11 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
public void testGetClosestRowBefore() throws IOException{
HRegion region = null;
HRegionIncommon region_incommon = null;
BatchUpdate batchUpdate = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
region_incommon = new HRegionIncommon(region);
// set up some test data
String t10 = "010";
@ -438,71 +605,6 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
}
}
/** For HBASE-694 */
public void testGetClosestRowBefore2() throws IOException{
HRegion region = null;
BatchUpdate batchUpdate = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
// set up some test data
String t10 = "010";
String t20 = "020";
String t30 = "030";
String t40 = "040";
batchUpdate = new BatchUpdate(t10);
batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
batchUpdate = new BatchUpdate(t30);
batchUpdate.put(COLUMNS[0], "t30 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
batchUpdate = new BatchUpdate(t40);
batchUpdate.put(COLUMNS[0], "t40 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
// try finding "035"
String t35 = "035";
Map<byte [], Cell> results =
region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
region.flushcache();
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
batchUpdate = new BatchUpdate(t20);
batchUpdate.put(COLUMNS[0], "t20 bytes".getBytes());
region.batchUpdate(batchUpdate, null);
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
region.flushcache();
// try finding "035"
results = region.getClosestRowBefore(Bytes.toBytes(t35), COLUMNS[0]);
assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
} finally {
if (region != null) {
try {
region.close();
} catch (Exception e) {
e.printStackTrace();
}
region.getLog().closeAndDelete();
}
}
}
/**
* For HBASE-40
*/
@ -576,112 +678,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
assertNull(result.get(COLUMNS[1]));
assertNull(result.get(COLUMNS[2]));
}
public void testGetFullMultiMapfile() throws IOException {
HRegion region = null;
HRegionIncommon region_incommon = null;
BatchUpdate batchUpdate = null;
Map<byte [], Cell> results = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
region_incommon = new HRegionIncommon(region);
//
// Test ordering issue
//
byte [] row = Bytes.toBytes("row1");
// write some data
batchUpdate = new BatchUpdate(row);
batchUpdate.put(COLUMNS[0], "olderValue".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert that getFull gives us the older value
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue()));
// write a new value for the cell
batchUpdate = new BatchUpdate(row);
batchUpdate.put(COLUMNS[0], "newerValue".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert that getFull gives us the later value
results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue()));
//
// Test the delete masking issue
//
byte [] row2 = Bytes.toBytes("row2");
byte [] cell1 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
byte [] cell2 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
byte [] cell3 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
// write some data at two columns
batchUpdate = new BatchUpdate(row2);
batchUpdate.put(cell1, "column0 value".getBytes());
batchUpdate.put(cell2, "column1 value".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert i get both columns
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("Should have two columns in the results map", 2, results.size());
assertEquals("column0 value", new String(results.get(cell1).getValue()));
assertEquals("column1 value", new String(results.get(cell2).getValue()));
// write a delete for the first column
batchUpdate = new BatchUpdate(row2);
batchUpdate.delete(cell1);
batchUpdate.put(cell2, "column1 new value".getBytes());
region.batchUpdate(batchUpdate, null);
// flush
region.flushcache();
// assert i get the second column only
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("Should have one column in the results map", 1, results.size());
assertNull("column0 value", results.get(cell1));
assertEquals("column1 new value", new String(results.get(cell2).getValue()));
//
// Include a delete and value from the memcache in the mix
//
batchUpdate = new BatchUpdate(row2);
batchUpdate.delete(cell2);
batchUpdate.put(cell3, "column2 value!".getBytes());
region.batchUpdate(batchUpdate, null);
// assert i get the third column only
results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP, 1, null);
assertEquals("Should have one column in the results map", 1, results.size());
assertNull("column0 value", results.get(cell1));
assertNull("column1 value", results.get(cell2));
assertEquals("column2 value!", new String(results.get(cell3).getValue()));
} finally {
if (region != null) {
try {
region.close();
} catch (Exception e) {
e.printStackTrace();
}
region.getLog().closeAndDelete();
}
}
}
private void assertColumnsPresent(final HRegion r, final byte [] row)
throws IOException {
Map<byte [], Cell> result =

View File

@ -134,7 +134,8 @@ public class TestHLog extends HBaseTestCase implements HConstants {
assertTrue(Bytes.equals(tableName, key.getTablename()));
assertTrue(Bytes.equals(HLog.METAROW, key.getRow()));
assertTrue(Bytes.equals(HLog.METACOLUMN, val.getColumn()));
assertEquals(0, HLogEdit.completeCacheFlush.compareTo(val.getVal()));
assertEquals(0, Bytes.compareTo(HLogEdit.COMPLETE_CACHE_FLUSH,
val.getVal()));
System.out.println(key + " " + val);
}
} finally {

View File

@ -273,7 +273,7 @@ public class TestHMemcache extends TestCase {
hmemcache.add(getHSKForRow(t20), "t20 bytes".getBytes());
hmemcache.add(getHSKForRow(t30), "t30 bytes".getBytes());
// write a delete in there to see if things still work ok
hmemcache.add(getHSKForRow(t35), HLogEdit.deleteBytes.get());
hmemcache.add(getHSKForRow(t35), HLogEdit.DELETED_BYTES);
hmemcache.add(getHSKForRow(t40), "t40 bytes".getBytes());
SortedMap<HStoreKey, Long> results = null;

View File

@ -146,7 +146,7 @@ public class TestHRegion extends HBaseTestCase {
byte [] collabel = null;
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
byte [] rowlabel = Bytes.toBytes("row_" + k);
if (k % 100 == 0) LOG.info(Bytes.toString(rowlabel));
byte [] bodydata = region.get(rowlabel, CONTENTS_BASIC).getValue();
assertNotNull(bodydata);
String bodystr = new String(bodydata, HConstants.UTF8_ENCODING).trim();
@ -381,7 +381,7 @@ public class TestHRegion extends HBaseTestCase {
numInserted += 2;
}
LOG.info("Write " + (vals1.length / 2) + " rows. Elapsed time: "
LOG.info("Write " + (vals1.length / 2) + " rows (second half). Elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));
// 6. Scan from cache and disk
@ -464,19 +464,16 @@ public class TestHRegion extends HBaseTestCase {
} finally {
s.close();
}
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched,
numInserted, numFetched);
LOG.info("Scanned " + vals1.length
+ " rows from disk. Elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));
// 9. Scan with a starting point
startTime = System.currentTimeMillis();
s = r.getScanner(cols, Bytes.toBytes("row_vals1_500"),
System.currentTimeMillis(), null);
numFetched = 0;
try {
HStoreKey curKey = new HStoreKey();
@ -503,7 +500,8 @@ public class TestHRegion extends HBaseTestCase {
} finally {
s.close();
}
assertEquals("Should have fetched " + (numInserted / 2) + " values, but fetched " + numFetched, (numInserted / 2), numFetched);
assertEquals("Should have fetched " + (numInserted / 2) +
" values, but fetched " + numFetched, (numInserted / 2), numFetched);
LOG.info("Scanned " + (numFetched / 2)
+ " rows from disk with specified start point. Elapsed time: "
@ -515,30 +513,27 @@ public class TestHRegion extends HBaseTestCase {
// NOTE: This test depends on testBatchWrite succeeding
private void splitAndMerge() throws IOException {
Path oldRegionPath = r.getRegionDir();
byte [] midKey = r.compactStores();
assertNotNull(midKey);
byte [] splitRow = r.compactStores();
assertNotNull(splitRow);
long startTime = System.currentTimeMillis();
HRegion subregions[] = r.splitRegion(midKey);
HRegion subregions[] = r.splitRegion(splitRow);
if (subregions != null) {
LOG.info("Split region elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));
assertEquals("Number of subregions", subregions.length, 2);
for (int i = 0; i < subregions.length; i++) {
subregions[i] = openClosedRegion(subregions[i]);
subregions[i].compactStores();
}
// Now merge it back together
Path oldRegion1 = subregions[0].getRegionDir();
Path oldRegion2 = subregions[1].getRegionDir();
startTime = System.currentTimeMillis();
r = HRegion.mergeAdjacent(subregions[0], subregions[1]);
region = new HRegionIncommon(r);
LOG.info("Merge regions elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));
LOG.info("Merge regions elapsed time: " +
((System.currentTimeMillis() - startTime) / 1000.0));
fs.delete(oldRegion1, true);
fs.delete(oldRegion2, true);
fs.delete(oldRegionPath, true);
@ -598,8 +593,10 @@ public class TestHRegion extends HBaseTestCase {
curVals.clear();
k++;
}
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
assertEquals("Expected " + NUM_VALS + " " + Bytes.toString(CONTENTS_BASIC) +
" values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM +
" values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
LOG.info("Scanned " + NUM_VALS
+ " rows from disk. Elapsed time: "
@ -673,8 +670,8 @@ public class TestHRegion extends HBaseTestCase {
}
curVals.clear();
}
assertEquals("Inserted " + (NUM_VALS + numInserted/2) + " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
assertEquals("Inserted " + (NUM_VALS + numInserted/2) +
" values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
LOG.info("Scanned " + fetched
+ " rows from disk. Elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));

View File

@ -154,7 +154,8 @@ public class TestScanner extends HBaseTestCase {
this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
HRegionIncommon hri = new HRegionIncommon(r);
try {
addContent(hri, Bytes.toString(HConstants.COL_REGIONINFO));
LOG.info("Added: " +
addContent(hri, Bytes.toString(HConstants.COL_REGIONINFO)));
int count = count(hri, -1);
assertEquals(count, count(hri, 100));
assertEquals(count, count(hri, 0));

View File

@ -133,11 +133,12 @@ public class TestSplit extends HBaseClusterTestCase {
}
private void basicSplit(final HRegion region) throws Exception {
addContent(region, COLFAMILY_NAME3);
LOG.info("" + addContent(region, COLFAMILY_NAME3));
region.flushcache();
byte [] midkey = region.compactStores();
assertNotNull(midkey);
HRegion [] regions = split(region, midkey);
byte [] splitRow = region.compactStores();
assertNotNull(splitRow);
LOG.info("SplitRow: " + Bytes.toString(splitRow));
HRegion [] regions = split(region, splitRow);
try {
// Need to open the regions.
// TODO: Add an 'open' to HRegion... don't do open by constructing
@ -148,11 +149,11 @@ public class TestSplit extends HBaseClusterTestCase {
// Assert can get rows out of new regions. Should be able to get first
// row from first region and the midkey from second region.
assertGet(regions[0], COLFAMILY_NAME3, Bytes.toBytes(START_KEY));
assertGet(regions[1], COLFAMILY_NAME3, midkey);
assertGet(regions[1], COLFAMILY_NAME3, splitRow);
// Test I can get scanner and that it starts at right place.
assertScan(regions[0], COLFAMILY_NAME3,
Bytes.toBytes(START_KEY));
assertScan(regions[1], COLFAMILY_NAME3, midkey);
assertScan(regions[1], COLFAMILY_NAME3, splitRow);
// Now prove can't split regions that have references.
for (int i = 0; i < regions.length; i++) {
// Add so much data to this region, we create a store file that is >
@ -251,11 +252,11 @@ public class TestSplit extends HBaseClusterTestCase {
}
}
private HRegion [] split(final HRegion r, final byte [] midKey)
private HRegion [] split(final HRegion r, final byte [] splitRow)
throws IOException {
// Assert can get mid key from passed region.
assertGet(r, COLFAMILY_NAME3, midKey);
HRegion [] regions = r.splitRegion(midKey);
assertGet(r, COLFAMILY_NAME3, splitRow);
HRegion [] regions = r.splitRegion(splitRow);
assertEquals(regions.length, 2);
return regions;
}

View File

@ -83,4 +83,4 @@ public class TestTimestamp extends HBaseClusterTestCase {
HConstants.FOREVER, false));
return createNewHRegion(htd, null, null);
}
}
}

View File

@ -52,14 +52,12 @@ public class TestMergeTool extends HBaseTestCase {
this.conf.set("hbase.hstore.compactionThreshold", "2");
// Create table description
this.desc = new HTableDescriptor("TestMergeTool");
this.desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
/*
* Create the HRegionInfos for the regions.
*/
// Region 0 will contain the key range [row_0200,row_0300)
sourceRegions[0] = new HRegionInfo(this.desc, Bytes.toBytes("row_0200"),
Bytes.toBytes("row_0300"));
@ -89,10 +87,10 @@ public class TestMergeTool extends HBaseTestCase {
*/
this.rows = new byte [5][][];
this.rows[0] = Bytes.toByteArrays(new String[] { "row_0210", "row_0280" });
this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350" });
this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175" });
this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560" });
this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000" });
this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350", "row_035" });
this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175", "row_0175", "row_0175"});
this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560", "row_0560", "row_0560", "row_0560"});
this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000", "row_1000", "row_1000", "row_1000", "row_1000" });
// Start up dfs
this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);