You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:02:10 UTC
svn commit: r1181364 - in
/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils:
DataGenerator.java HBaseUtils.java MultiThreadedReader.java
MultiThreadedWriter.java
Author: nspiegelberg
Date: Tue Oct 11 02:02:09 2011
New Revision: 1181364
URL: http://svn.apache.org/viewvc?rev=1181364&view=rev
Log:
add presplit logic to HBaseTest
Summary:
add presplit logic to HBaseTest
Test Plan:
ran HBaseTest on my test cluster and made sure the regions were growing at a
pretty uniform rate.
DiffCamp Revision: 151433
Reviewed By: pkhaitan
CC: pkhaitan, hbase@lists
Revert Plan:
OK
Modified:
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/DataGenerator.java
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/HBaseUtils.java
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedReader.java
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedWriter.java
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/DataGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/DataGenerator.java?rev=1181364&r1=1181363&r2=1181364&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/DataGenerator.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/DataGenerator.java Tue Oct 11 02:02:09 2011
@@ -21,6 +21,9 @@ package org.apache.hadoop.hbase.manual.u
import java.util.Random;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.MD5Hash;
+
public class DataGenerator {
static Random random_ = new Random();
/* one byte fill pattern */
@@ -38,12 +41,12 @@ public class DataGenerator {
int minDataSize_ = 0;
int maxDataSize_ = 0;
- static public String paddedKey(long key) {
- // left-pad key with zeroes to 10 decimal places.
- String paddedKey = String.format("%010d", key);
+ static public String md5PrefixedKey(long key) {
+ String stringKey = Long.toString(key);
+ String md5hash = MD5Hash.getMD5AsHex(Bytes.toBytes(stringKey));
- // flip the key to randomize
- return (new StringBuffer(paddedKey)).reverse().toString();
+ // flip the key to randomize
+ return md5hash + ":" + stringKey;
}
public DataGenerator(int minDataSize, int maxDataSize) {
@@ -57,7 +60,7 @@ public class DataGenerator {
// write the key first
int sizeLeft = dataSize;
- String keyAsString = DataGenerator.paddedKey(key);
+ String keyAsString = DataGenerator.md5PrefixedKey(key);
sb.append(keyAsString);
sizeLeft -= keyAsString.length();
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/HBaseUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/HBaseUtils.java?rev=1181364&r1=1181363&r2=1181364&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/HBaseUtils.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/HBaseUtils.java Tue Oct 11 02:02:09 2011
@@ -37,10 +37,19 @@ import org.apache.hadoop.hbase.client.HT
import org.apache.hadoop.hbase.util.Bytes;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.commons.lang.StringUtils;
+
public class HBaseUtils
{
private static final Log LOG = LogFactory.getLog(HBaseUtils.class);
+ private final static String MAXMD5 = "FFFFFFFF";
+ private final static int rowComparisonLength = MAXMD5.length();
+ private static int DEFAULT_REGIONS_PER_SERVER = 5;
+
public static void sleep(int millisecs) {
try {
Thread.sleep(millisecs);
@@ -61,6 +70,7 @@ public class HBaseUtils
return table;
}
+ // the table will be pre-split assuming a MD5 prefixed key space.
public static void createTableIfNotExists(HBaseConfiguration conf, byte[] tableName, byte[][] columnFamilies) {
HTableDescriptor desc = new HTableDescriptor(tableName);
for(byte[] cfName : columnFamilies) {
@@ -69,7 +79,14 @@ public class HBaseUtils
try
{
HBaseAdmin admin = new HBaseAdmin(conf);
- admin.createTable(desc);
+
+ // create a table a pre-splits regions.
+ // The number of splits is set as:
+ // region servers * regions per region server).
+ int numberOfServers = admin.getClusterStatus().getServers();
+ int totalNumberOfRegions = numberOfServers * DEFAULT_REGIONS_PER_SERVER;
+ byte[][] splits =splitKeysMD5(totalNumberOfRegions);
+ admin.createTable(desc, splits);
}
catch(MasterNotRunningException e) {
LOG.error("Master not running.");
@@ -97,4 +114,57 @@ public class HBaseUtils
return new HBaseConfiguration(c);
}
+ /**
+ * Creates splits for MD5 hashing.
+ * @param numberOfSplits
+ * @return Byte array of size (numberOfSplits-1) corresponding to the
+ * boundaries between splits.
+ */
+ private static byte[][] splitKeysMD5(int numberOfSplits) {
+ BigInteger max = new BigInteger(MAXMD5, 16);
+ BigInteger[] bigIntegerSplits = split(max, numberOfSplits);
+ byte[][] byteSplits = convertToBytes(bigIntegerSplits);
+ return byteSplits;
+ }
+
+ /**
+ * Splits the given BigInteger into numberOfSplits parts
+ * @param maxValue
+ * @param numberOfSplits
+ * @return array of BigInteger which is of size (numberOfSplits-1)
+ */
+ private static BigInteger[] split(BigInteger maxValue, int numberOfSplits) {
+ BigInteger[] splits = new BigInteger[numberOfSplits-1];
+ BigInteger sizeOfEachSplit = maxValue.divide(BigInteger.
+ valueOf(numberOfSplits));
+ for (int i = 1; i < numberOfSplits; i++) {
+ splits[i-1] = sizeOfEachSplit.multiply(BigInteger.valueOf(i));
+ }
+ return splits;
+ }
+
+ /**
+ * Returns the bytes corresponding to the BigInteger
+ * @param bigInteger
+ * @return byte corresponding to input BigInteger
+ */
+ private static byte[] convertToByte(BigInteger bigInteger) {
+ String bigIntegerString = bigInteger.toString(16);
+ bigIntegerString = StringUtils.leftPad(bigIntegerString,
+ rowComparisonLength, '0');
+ return Bytes.toBytes(bigIntegerString);
+ }
+
+ /**
+ * Returns an array of bytes corresponding to an array of BigIntegers
+ * @param bigIntegers
+ * @return bytes corresponding to the bigIntegers
+ */
+ private static byte[][] convertToBytes(BigInteger[] bigIntegers) {
+ byte[][] returnBytes = new byte[bigIntegers.length][];
+ for (int i = 0; i < bigIntegers.length; i++) {
+ returnBytes[i] = convertToByte(bigIntegers[i]);
+ }
+ return returnBytes;
+ }
}
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedReader.java?rev=1181364&r1=1181363&r2=1181364&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedReader.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedReader.java Tue Oct 11 02:02:09 2011
@@ -121,7 +121,7 @@ public class MultiThreadedReader extends
} else {
curKey = startKey_ + Math.abs(reader_.random_.nextLong())%(endKey_ - startKey_);
}
- get = new Get(DataGenerator.paddedKey(curKey).getBytes());
+ get = new Get(DataGenerator.md5PrefixedKey(curKey).getBytes());
get.addFamily(columnFamily_);
// get.addColumn(columnFamily_, Bytes.toBytes("0"));
}
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedWriter.java?rev=1181364&r1=1181363&r2=1181364&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedWriter.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils/MultiThreadedWriter.java Tue Oct 11 02:02:09 2011
@@ -150,7 +150,7 @@ public class MultiThreadedWriter extends
}
public static byte[] longToByteArrayKey(long rowKey) {
- return DataGenerator.paddedKey(rowKey).getBytes();
+ return DataGenerator.md5PrefixedKey(rowKey).getBytes();
}
public void insert(long rowKey, long col) {
@@ -179,7 +179,7 @@ public class MultiThreadedWriter extends
if(startCol >= endCol) {
return;
}
- Put put = new Put(DataGenerator.paddedKey(rowKey).getBytes());
+ Put put = new Put(DataGenerator.md5PrefixedKey(rowKey).getBytes());
byte[] columnQualifier;
byte[] value;
for(long i = startCol; i < endCol; ++i) {