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:42:39 UTC

svn commit: r1181929 - in /hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/utils: DataGenerator.java MultiThreadedReader.java

Author: nspiegelberg
Date: Tue Oct 11 17:42:39 2011
New Revision: 1181929

URL: http://svn.apache.org/viewvc?rev=1181929&view=rev
Log:
Modify HBase Load Tester to verify entire data.

Summary:
The HBase Load Tester in the read mode would only verify the
prefix of the data instead of the entire data for a particular row. This
has been modified so as to make the value of any cell in the row to be a
deterministic function of the row key itself. Hence the data put into
the row is randomized with a seed being the row key, but since the seed
is the row key we can always deterministically know what its value
should be when we the row.

Test Plan: 1) Run it for HBase running on a 5 node dev cluster.
Reviewed By: kannan
Reviewers: kannan, nspiegelberg
CC: hbase@lists, pritam, kannan, nspiegelberg
Revert Plan:
Tags:

- begin *PUBLIC* platform impact section -
Bugzilla: #
- end platform impact -

Differential Revision: 300961

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/MultiThreadedReader.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=1181929&r1=1181928&r2=1181929&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 17:42:39 2011
@@ -26,17 +26,6 @@ import org.apache.hadoop.hbase.util.MD5H
 
 public class DataGenerator {
   static Random random_ = new Random();
-  /* one byte fill pattern */
-  public static final String fill1B_     = "-";
-  /* 64 byte fill pattern */
-  public static final String fill64B_    = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789  ";
-  /* alternate 64 byte fill pattern */
-  public static final String fill64BAlt_ = "AaBbCcDdEeFfGgHhIiJjKkLlMmNnOoPpQqRrSsTtUuVvWwXxYyZz0123456789+-";
-  /* 1K fill pattern */
-  public static final String fill1K_     = fill64BAlt_+fill64BAlt_+fill64BAlt_+fill64BAlt_+
-                                           fill64BAlt_+fill64BAlt_+fill64BAlt_+fill64BAlt_+
-                                           fill64BAlt_+fill64BAlt_+fill64BAlt_+fill64BAlt_+
-                                           fill64BAlt_+fill64BAlt_+fill64BAlt_+fill64BAlt_;
 
   int minDataSize_ = 0;
   int maxDataSize_ = 0;
@@ -54,38 +43,23 @@ public class DataGenerator {
     maxDataSize_ = maxDataSize;
   }
 
+  private static byte[] getDataForKey(String rowKey, int dataSize) {
+    // Need a different local random object since multiple threads might invoke
+    // this method at the same time.
+    Random random = new Random(rowKey.hashCode());
+    byte[] rbytes = new byte[dataSize];
+    random.nextBytes(rbytes);
+    return rbytes;
+  }
+
   public byte[] getDataInSize(long key) {
+    String rowKey = DataGenerator.md5PrefixedKey(key);
     int dataSize = minDataSize_ + random_.nextInt(Math.abs(maxDataSize_ - minDataSize_));
-    StringBuilder sb = new StringBuilder();
-
-    // write the key first
-    int sizeLeft = dataSize;
-    String keyAsString = DataGenerator.md5PrefixedKey(key);
-    sb.append(keyAsString);
-    sizeLeft -= keyAsString.length();
-
-    for(int i = 0; i < sizeLeft/1024; ++i)
-    {
-      sb.append(fill1K_);
-    }
-    sizeLeft = sizeLeft % 1024;
-    for(int i = 0; i < sizeLeft/64; ++i)
-    {
-      sb.append(fill64B_);
-    }
-    sizeLeft = sizeLeft % 64;
-    for(int i = 0; i < dataSize%64; ++i)
-    {
-      sb.append(fill1B_);
-    }
-
-    return sb.toString().getBytes();
+    return getDataForKey(rowKey, dataSize);
   }
 
-  public static boolean verify(String rowKey, String actionId, String data) {
-    if(!data.startsWith(rowKey)) {
-      return false;
-    }
-    return true;
+  public static boolean verify(String rowKey, String actionId, byte[] data) {
+    byte[] expectedData = getDataForKey(rowKey, data.length);
+    return (Bytes.equals(expectedData, data));
   }
 }

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=1181929&r1=1181928&r2=1181929&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 17:42:39 2011
@@ -176,10 +176,9 @@ public class MultiThreadedReader extends
             List<KeyValue> keyValues = result.list();
             for(KeyValue kv : keyValues) {
               String actionId = new String(kv.getQualifier());
-              String data = new String(kv.getValue());
 
               // if something does not look right report it
-              if(!DataGenerator.verify(rowKey, actionId, data)) {
+              if (!DataGenerator.verify(rowKey, actionId, kv.getValue())) {
                 reader_.numErrors_.addAndGet(1);
                 LOG.error("Error checking data for key = " + rowKey + ", actionId = " + actionId);
               }