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 19:44:04 UTC
svn commit: r1181954 -
/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
Author: nspiegelberg
Date: Tue Oct 11 17:44:04 2011
New Revision: 1181954
URL: http://svn.apache.org/viewvc?rev=1181954&view=rev
Log:
Fix & improve TestHFilePerformance.
Summary:
Due to bug TestHFilePerformance was adding keys not in lexographical order.
The sanity check would not work because key byte[] array was reused, so
lastKeyBuffer was usually pointing to the same buffer.
The bug is fixed in this diff:
Moreover:
- better generation of values; instead of simply repeating them, some parts are
redundant and other generated randomly
- generate dataset once only and reuse it in different tests
- exclude generating data from performance meassure
- remove unused arguments
Test Plan: Run TestHFilePerformance unit test.
Reviewers: pkhemani, kranganathan, mbautin
Reviewed By: pkhemani
CC: gqchen, hbase@lists, pkhemani, jacek
Differential Revision: 312035
Revert Plan: This commit does not affect HBase itself. Only unit test is
affected.
Task ID: 687599
Modified:
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java?rev=1181954&r1=1181953&r2=1181954&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java Tue Oct 11 17:44:04 2011
@@ -21,6 +21,9 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
import java.util.Random;
import junit.framework.TestCase;
@@ -31,11 +34,36 @@ import org.apache.hadoop.fs.FSDataOutput
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.GzipCodec;
+class RawKeyValue implements Comparable<RawKeyValue> {
+ private final byte[] key;
+ private final byte[] value;
+
+ public RawKeyValue(byte[] key, byte[] value) {
+ super();
+ this.key = key.clone();
+ this.value = value.clone();
+ }
+
+ public byte[] getKey() {
+ return key;
+ }
+
+ public byte[] getValue() {
+ return value;
+ }
+
+ @Override
+ public int compareTo(RawKeyValue o) {
+ return Bytes.compareTo(key, o.key);
+ }
+}
+
/**
* Set of long-running tests to measure performance of HFile.
* <p>
@@ -103,13 +131,28 @@ public class TestHFilePerformance extend
private static class KeyValueGenerator {
Random keyRandomizer;
Random valueRandomizer;
- long randomValueRatio = 3; // 1 out of randomValueRatio generated values will be random.
- long valueSequence = 0 ;
+ int randomPartsRatio = 10; // 1 out of randomValueRatio generated values will be random.
+ long valueSequence = 0;
+
+ byte[][] randomParts;
+ final int randomPartsCount = 64;
+ final int randomPartsMinLength = 8;
+ final int randomPartsMaxLength = 64;
+ final int randomPartsDiffLength = 1 + randomPartsMaxLength -
+ randomPartsMinLength;
KeyValueGenerator() {
keyRandomizer = new Random(0L); //TODO with seed zero
valueRandomizer = new Random(1L); //TODO with seed one
+ randomParts = new byte[randomPartsCount][];
+
+ for (int i = 0 ; i < randomParts.length ; ++i) {
+ int length = valueRandomizer.nextInt(randomPartsDiffLength) +
+ randomPartsMinLength;
+ randomParts[i] = new byte[length];
+ valueRandomizer.nextBytes(randomParts[i]);
+ }
}
// Key is always random now.
@@ -118,8 +161,22 @@ public class TestHFilePerformance extend
}
void getValue(byte[] value) {
- if (valueSequence % randomValueRatio == 0)
- valueRandomizer.nextBytes(value);
+ for (int pos = 0 ; pos < value.length ; ) {
+ byte[] part;
+ if (valueRandomizer.nextInt(randomPartsRatio) + 1 ==
+ randomPartsRatio) {
+ int length = valueRandomizer.nextInt(randomPartsDiffLength) +
+ randomPartsMinLength;
+ part = new byte[length];
+ valueRandomizer.nextBytes(part);
+ } else {
+ int partId = valueRandomizer.nextInt(randomParts.length);
+ part = randomParts[partId];
+ }
+ int copyLength = Math.min(part.length, value.length - pos);
+ System.arraycopy(part, 0, value, pos, copyLength);
+ pos += copyLength;
+ }
valueSequence++;
}
}
@@ -136,19 +193,13 @@ public class TestHFilePerformance extend
* @throws IOException
*/
//TODO writeMethod: implement multiple ways of writing e.g. A) known length (no chunk) B) using a buffer and streaming (for many chunks).
- public void timeWrite(String fileType, int keyLength, int valueLength,
- String codecName, long rows, String writeMethod, int minBlockSize)
+ public void timeWrite(String fileType, List<RawKeyValue> testKeyValues,
+ String codecName, String writeMethod, int minBlockSize)
throws IOException {
System.out.println("File Type: " + fileType);
System.out.println("Writing " + fileType + " with codecName: " + codecName);
long totalBytesWritten = 0;
-
- //Using separate randomizer for key/value with seeds matching Sequence File.
- byte[] key = new byte[keyLength];
- byte[] value = new byte[valueLength];
- KeyValueGenerator generator = new KeyValueGenerator();
-
startTime();
Path path = new Path(ROOT_DIR, fileType + ".Performance");
@@ -161,12 +212,10 @@ public class TestHFilePerformance extend
minBlockSize, codecName, null);
// Writing value in one shot.
- for (long l=0 ; l<rows ; l++ ) {
- generator.getKey(key);
- generator.getValue(value);
- writer.append(key, value);
- totalBytesWritten += key.length;
- totalBytesWritten += value.length;
+ for (RawKeyValue rawKV : testKeyValues) {
+ writer.append(rawKV.getKey(), rawKV.getValue());
+ totalBytesWritten += rawKV.getKey().length;
+ totalBytesWritten += rawKV.getValue().length;
}
writer.close();
} else if ("SequenceFile".equals(fileType)){
@@ -190,14 +239,11 @@ public class TestHFilePerformance extend
BytesWritable keyBsw;
BytesWritable valBsw;
- for (long l=0 ; l<rows ; l++ ) {
-
- generator.getKey(key);
- keyBsw = new BytesWritable(key);
+ for (RawKeyValue rawKV : testKeyValues) {
+ keyBsw = new BytesWritable(rawKV.getKey());
totalBytesWritten += keyBsw.getSize();
- generator.getValue(value);
- valBsw = new BytesWritable(value);
+ valBsw = new BytesWritable(rawKV.getValue());
writer.append(keyBsw, valBsw);
totalBytesWritten += valBsw.getSize();
}
@@ -220,7 +266,22 @@ public class TestHFilePerformance extend
printlnWithTimestamp(" total = " + fs.getFileStatus(path).getLen() + "B");
}
- public void timeReading(String fileType, int keyLength, int valueLength,
+ private List<RawKeyValue> generateDataset(int keyLength, int valueLength,
+ long rows, KeyValueGenerator generator) {
+ byte[] key = new byte[keyLength];
+ byte[] value = new byte[valueLength];
+ List<RawKeyValue> testKeyValues = new ArrayList<RawKeyValue>((int) rows);
+ for (long l=0 ; l<rows ; l++ ) {
+ generator.getKey(key);
+ generator.getValue(value);
+ testKeyValues.add(new RawKeyValue(
+ key, value));
+ }
+ Collections.sort(testKeyValues);
+ return testKeyValues;
+ }
+
+ public void timeReading(String fileType,
long rows, int method) throws IOException {
System.out.println("Reading file of type: " + fileType);
Path path = new Path(ROOT_DIR, fileType + ".Performance");
@@ -309,20 +370,25 @@ public class TestHFilePerformance extend
int minBlockSize = 10*1024*1024; // 10MB
int rows = 10000;
+ //Using separate randomizer for key/value with seeds matching Sequence File.
+ KeyValueGenerator generator = new KeyValueGenerator();
+ List<RawKeyValue> testKeyValues = generateDataset(keyLength, valueLength,
+ rows, generator);
+
System.out.println("****************************** Sequence File *****************************");
- timeWrite("SequenceFile", keyLength, valueLength, "none", rows, null, minBlockSize);
+ timeWrite("SequenceFile", testKeyValues, "none", null, minBlockSize);
System.out.println("\n+++++++\n");
- timeReading("SequenceFile", keyLength, valueLength, rows, -1);
+ timeReading("SequenceFile", rows, -1);
System.out.println("");
System.out.println("----------------------");
System.out.println("");
/* DISABLED LZO
- timeWrite("SequenceFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
+ timeWrite("SequenceFile", testKeyValues, "lzo", null, minBlockSize);
System.out.println("\n+++++++\n");
- timeReading("SequenceFile", keyLength, valueLength, rows, -1);
+ timeReading("SequenceFile", rows, -1);
System.out.println("");
System.out.println("----------------------");
@@ -331,10 +397,9 @@ public class TestHFilePerformance extend
/* Sequence file can only use native hadoop libs gzipping so commenting out.
*/
try {
- timeWrite("SequenceFile", keyLength, valueLength, "gz", rows, null,
- minBlockSize);
+ timeWrite("SequenceFile", testKeyValues, "gz", null, minBlockSize);
System.out.println("\n+++++++\n");
- timeReading("SequenceFile", keyLength, valueLength, rows, -1);
+ timeReading("SequenceFile", rows, -1);
} catch (IllegalArgumentException e) {
System.out.println("Skipping sequencefile gz: " + e.getMessage());
}
@@ -343,15 +408,15 @@ public class TestHFilePerformance extend
System.out.println("\n\n\n");
System.out.println("****************************** HFile *****************************");
- timeWrite("HFile", keyLength, valueLength, "none", rows, null, minBlockSize);
+ timeWrite("HFile", testKeyValues, "none", null, minBlockSize);
System.out.println("\n+++++++\n");
- timeReading("HFile", keyLength, valueLength, rows, 0 );
+ timeReading("HFile", rows, 0 );
System.out.println("");
System.out.println("----------------------");
System.out.println("");
/* DISABLED LZO
- timeWrite("HFile", keyLength, valueLength, "lzo", rows, null, minBlockSize);
+ timeWrite("HFile", testKeyValues, "lzo", null, minBlockSize);
System.out.println("\n+++++++\n");
timeReading("HFile", keyLength, valueLength, rows, 0 );
System.out.println("\n+++++++\n");
@@ -363,18 +428,16 @@ public class TestHFilePerformance extend
System.out.println("----------------------");
System.out.println("");
*/
- timeWrite("HFile", keyLength, valueLength, "gz", rows, null, minBlockSize);
+ timeWrite("HFile", testKeyValues, "gz", null, minBlockSize);
System.out.println("\n+++++++\n");
- timeReading("HFile", keyLength, valueLength, rows, 0 );
+ timeReading("HFile", rows, 0 );
System.out.println("\n\n\n\nNotes: ");
System.out.println(" * Timing includes open/closing of files.");
System.out.println(" * Timing includes reading both Key and Value");
System.out.println(" * Data is generated as random bytes. Other methods e.g. using " +
"dictionary with care for distributation of words is under development.");
- System.out.println(" * Timing of write currently, includes random value/key generations. " +
- "Which is the same for Sequence File and HFile. Another possibility is to generate " +
- "test data beforehand");
+ System.out.println(" * Timing of write excludes random value/key generations. ");
System.out.println(" * We need to mitigate cache effect on benchmark. We can apply several " +
"ideas, for next step we do a large dummy read between benchmark read to dismantle " +
"caching of data. Renaming of file may be helpful. We can have a loop that reads with" +