You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2018/05/08 02:27:37 UTC

[6/8] hbase git commit: HBASE-20505 PE should support multi column family read and write cases

HBASE-20505 PE should support multi column family read and write cases


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/ef9fd29c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ef9fd29c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ef9fd29c

Branch: refs/heads/branch-1.2
Commit: ef9fd29cad44437d3db185931c50598621bc125a
Parents: b3a162b
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon Apr 30 12:28:26 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon May 7 18:49:47 2018 -0700

----------------------------------------------------------------------
 .../hbase/IntegrationTestRegionReplicaPerf.java |   3 +-
 .../hadoop/hbase/PerformanceEvaluation.java     | 268 ++++++++++++-------
 .../hbase/mapreduce/TestHFileOutputFormat.java  |   8 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |   8 +-
 4 files changed, 176 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ef9fd29c/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
index d05e039..8a6afdb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
@@ -61,6 +61,7 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
 
   private static final Log LOG = LogFactory.getLog(IntegrationTestRegionReplicaPerf.class);
 
+  private static final byte[] FAMILY_NAME = Bytes.toBytes("info");
   private static final String SLEEP_TIME_KEY = "sleeptime";
   // short default interval because tests don't run very long.
   private static final String SLEEP_TIME_DEFAULT = "" + (10 * 1000l);
@@ -236,7 +237,7 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
 
   @Override
   protected Set<String> getColumnFamilies() {
-    return Sets.newHashSet(Bytes.toString(PerformanceEvaluation.FAMILY_NAME));
+    return Sets.newHashSet(Bytes.toString(FAMILY_NAME));
   }
 
   /** Compute the mean of the given {@code stat} from a timing results. */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef9fd29c/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 4a7c514..ceb5cf0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -98,8 +98,8 @@ import org.apache.hadoop.util.ToolRunner;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import com.yammer.metrics.core.Histogram;
-import com.yammer.metrics.stats.UniformSample;
 import com.yammer.metrics.stats.Snapshot;
+import com.yammer.metrics.stats.UniformSample;
 
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
@@ -132,8 +132,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
   }
 
   public static final String TABLE_NAME = "TestTable";
-  public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
-  public static final byte [] COLUMN_ZERO = Bytes.toBytes("" + 0);
+  public static final String FAMILY_NAME_BASE = "info";
+  public static final byte[] FAMILY_ZERO = Bytes.toBytes("info0");
+  public static final byte[] COLUMN_ZERO = Bytes.toBytes("" + 0);
   public static final int DEFAULT_VALUE_LENGTH = 1000;
   public static final int ROW_LENGTH = 26;
 
@@ -368,22 +369,25 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * Create an HTableDescriptor from provided TestOptions.
    */
   protected static HTableDescriptor getTableDescriptor(TestOptions opts) {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(opts.tableName));
-    HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME);
-    family.setDataBlockEncoding(opts.blockEncoding);
-    family.setCompressionType(opts.compression);
-    family.setBloomFilterType(opts.bloomType);
-    if (opts.inMemoryCF) {
-      family.setInMemory(true);
-    }
-    desc.addFamily(family);
+    HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(opts.tableName));
+    for (int family = 0; family < opts.families; family++) {
+      byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+      HColumnDescriptor familyDesc = new HColumnDescriptor(familyName);
+      familyDesc.setDataBlockEncoding(opts.blockEncoding);
+      familyDesc.setCompressionType(opts.compression);
+      familyDesc.setBloomFilterType(opts.bloomType);
+      if (opts.inMemoryCF) {
+        familyDesc.setInMemory(true);
+      }
+      tableDesc.addFamily(familyDesc);
+    }
     if (opts.replicas != DEFAULT_OPTS.replicas) {
-      desc.setRegionReplication(opts.replicas);
+      tableDesc.setRegionReplication(opts.replicas);
     }
-    if (opts.splitPolicy != DEFAULT_OPTS.splitPolicy) {
-      desc.setRegionSplitPolicyClassName(opts.splitPolicy);
+    if (opts.splitPolicy != null && !opts.splitPolicy.equals(DEFAULT_OPTS.splitPolicy)) {
+      tableDesc.setRegionSplitPolicyClassName(opts.splitPolicy);
     }
-    return desc;
+    return tableDesc;
   }
 
   /**
@@ -615,6 +619,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     int valueSize = DEFAULT_VALUE_LENGTH;
     int period = (this.perClientRunRows / 10) == 0? perClientRunRows: perClientRunRows / 10;
     int columns = 1;
+    int families = 1;
     int caching = 30;
     boolean addColumns = true;
     long bufferSize = 2l * 1024l * 1024l;
@@ -659,6 +664,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.randomSleep = that.randomSleep;
       this.addColumns = that.addColumns;
       this.columns = that.columns;
+      this.families = that.families;
       this.caching = that.caching;
       this.bufferSize = that.bufferSize;
     }
@@ -679,6 +685,14 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.columns = columns;
     }
 
+    public int getFamilies() {
+      return this.families;
+    }
+
+    public void setFamilies(final int families) {
+      this.families = families;
+    }
+
     public boolean isValueZipf() {
       return valueZipf;
     }
@@ -1214,13 +1228,16 @@ public class PerformanceEvaluation extends Configured implements Tool {
       Scan scan = new Scan(getRandomRow(this.rand, opts.totalRows));
       scan.setCaching(opts.caching);
       FilterList list = new FilterList();
-      if (opts.addColumns) {
-        for (int column = 0; column < opts.columns; column++) {
-          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-          scan.addColumn(FAMILY_NAME, qualifier);
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        if (opts.addColumns) {
+          for (int column = 0; column < opts.columns; column++) {
+            byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+            scan.addColumn(familyName, qualifier);
+          }
+        } else {
+          scan.addFamily(familyName);
         }
-      } else {
-        scan.addFamily(FAMILY_NAME);
       }
       if (opts.filterAll) {
         list.addFilter(new FilterAllFilter());
@@ -1251,18 +1268,22 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
       Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
-      scan.setCaching(opts.caching);
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        if (opts.addColumns) {
+          for (int column = 0; column < opts.columns; column++) {
+            byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+            scan.addColumn(familyName, qualifier);
+          }
+        } else {
+          scan.addFamily(familyName);
+        }
+      }
       if (opts.filterAll) {
         scan.setFilter(new FilterAllFilter());
       }
-      if (opts.addColumns) {
-        for (int column = 0; column < opts.columns; column++) {
-          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-          scan.addColumn(FAMILY_NAME, qualifier);
-        }
-      } else {
-        scan.addFamily(FAMILY_NAME);
-      }
+      scan.setCaching(opts.caching);
+      scan.setScanMetricsEnabled(true);
       Result r = null;
       int count = 0;
       ResultScanner s = this.table.getScanner(scan);
@@ -1358,13 +1379,16 @@ public class PerformanceEvaluation extends Configured implements Tool {
         Thread.sleep(rd.nextInt(opts.randomSleep));
       }
       Get get = new Get(getRandomRow(this.rand, opts.totalRows));
-      if (opts.addColumns) {
-        for (int column = 0; column < opts.columns; column++) {
-          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-          get.addColumn(FAMILY_NAME, qualifier);
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        if (opts.addColumns) {
+          for (int column = 0; column < opts.columns; column++) {
+            byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+            get.addColumn(familyName, qualifier);
+          }
+        } else {
+          get.addFamily(familyName);
         }
-      } else {
-        get.addFamily(FAMILY_NAME);
       }
       if (opts.filterAll) {
         get.setFilter(new FilterAllFilter());
@@ -1408,23 +1432,26 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       byte[] row = getRandomRow(this.rand, opts.totalRows);
       Put put = new Put(row);
-      for (int column = 0; column < opts.columns; column++) {
-        byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-        byte[] value = generateData(this.rand, getValueLength(this.rand));
-        if (opts.useTags) {
-          byte[] tag = generateData(this.rand, TAG_LENGTH);
-          Tag[] tags = new Tag[opts.noOfTags];
-          for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
-            tags[n] = t;
-          }
-          KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        for (int column = 0; column < opts.columns; column++) {
+          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+          byte[] value = generateData(this.rand, getValueLength(this.rand));
+          if (opts.useTags) {
+            byte[] tag = generateData(this.rand, TAG_LENGTH);
+            Tag[] tags = new Tag[opts.noOfTags];
+            for (int n = 0; n < opts.noOfTags; n++) {
+              Tag t = new Tag((byte) n, tag);
+              tags[n] = t;
+            }
+            KeyValue kv = new KeyValue(row, familyName, qualifier, HConstants.LATEST_TIMESTAMP,
               value, tags);
-          put.add(kv);
-          updateValueSize(kv.getValueLength());
-        } else {
-          put.add(FAMILY_NAME, qualifier, value);
-          updateValueSize(value.length);
+            put.add(kv);
+            updateValueSize(kv.getValueLength());
+          } else {
+            put.add(familyName, qualifier, value);
+            updateValueSize(value.length);
+          }
         }
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
@@ -1455,19 +1482,22 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       if (this.testScanner == null) {
         Scan scan = new Scan(format(opts.startRow));
-        scan.setCaching(opts.caching);
-        if (opts.addColumns) {
-          for (int column = 0; column < opts.columns; column++) {
-            byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-            scan.addColumn(FAMILY_NAME, qualifier);
+        for (int family = 0; family < opts.families; family++) {
+          byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+          if (opts.addColumns) {
+            for (int column = 0; column < opts.columns; column++) {
+              byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+              scan.addColumn(familyName, qualifier);
+            }
+          } else {
+            scan.addFamily(familyName);
           }
-        } else {
-          scan.addFamily(FAMILY_NAME);
         }
         if (opts.filterAll) {
           scan.setFilter(new FilterAllFilter());
         }
-       this.testScanner = table.getScanner(scan);
+        scan.setCaching(opts.caching);
+        this.testScanner = table.getScanner(scan);
       }
       Result r = testScanner.next();
       updateValueSize(r);
@@ -1512,7 +1542,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void testRow(final int i) throws IOException {
       Increment increment = new Increment(format(i));
-      increment.addColumn(FAMILY_NAME, getQualifier(), 1l);
+      // unlike checkAndXXX tests, which make most sense to do on a single value,
+      // if multiple families are specified for an increment test we assume it is
+      // meant to raise the work factor
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        increment.addColumn(familyName, getQualifier(), 1l);
+      }
       updateValueSize(this.table.increment(increment));
     }
   }
@@ -1526,7 +1562,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       byte [] bytes = format(i);
       Append append = new Append(bytes);
-      append.add(FAMILY_NAME, getQualifier(), bytes);
+      // unlike checkAndXXX tests, which make most sense to do on a single value,
+      // if multiple families are specified for an increment test we assume it is
+      // meant to raise the work factor
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        append.add(familyName, getQualifier(), bytes);
+      }
       updateValueSize(this.table.append(append));
     }
   }
@@ -1538,14 +1580,15 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void testRow(final int i) throws IOException {
-      byte [] bytes = format(i);
+      final byte [] bytes = format(i);
+      // checkAndXXX tests operate on only a single value
       // Put a known value so when we go to check it, it is there.
       Put put = new Put(bytes);
-      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      put.addColumn(FAMILY_ZERO, getQualifier(), bytes);
       this.table.put(put);
       RowMutations mutations = new RowMutations(bytes);
       mutations.add(put);
-      this.table.checkAndMutate(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes,
+      this.table.checkAndMutate(bytes, FAMILY_ZERO, getQualifier(), CompareOp.EQUAL, bytes,
           mutations);
     }
   }
@@ -1557,12 +1600,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void testRow(final int i) throws IOException {
-      byte [] bytes = format(i);
+      final byte [] bytes = format(i);
+      // checkAndXXX tests operate on only a single value
       // Put a known value so when we go to check it, it is there.
       Put put = new Put(bytes);
-      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      put.addColumn(FAMILY_ZERO, getQualifier(), bytes);
       this.table.put(put);
-      this.table.checkAndPut(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes, put);
+      this.table.checkAndPut(bytes, FAMILY_ZERO, getQualifier(), CompareOp.EQUAL, bytes, put);
     }
   }
 
@@ -1573,14 +1617,15 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void testRow(final int i) throws IOException {
-      byte [] bytes = format(i);
+      final byte [] bytes = format(i);
+      // checkAndXXX tests operate on only a single value
       // Put a known value so when we go to check it, it is there.
       Put put = new Put(bytes);
-      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      put.addColumn(FAMILY_ZERO, getQualifier(), bytes);
       this.table.put(put);
       Delete delete = new Delete(put.getRow());
-      delete.addColumn(FAMILY_NAME, getQualifier());
-      this.table.checkAndDelete(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes, delete);
+      delete.addColumn(FAMILY_ZERO, getQualifier());
+      this.table.checkAndDelete(bytes, FAMILY_ZERO, getQualifier(), CompareOp.EQUAL, bytes, delete);
     }
   }
 
@@ -1592,10 +1637,15 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void testRow(final int i) throws IOException {
       Get get = new Get(format(i));
-      if (opts.addColumns) {
-        for (int column = 0; column < opts.columns; column++) {
-          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-          get.addColumn(FAMILY_NAME, qualifier);
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        if (opts.addColumns) {
+          for (int column = 0; column < opts.columns; column++) {
+            byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+            get.addColumn(familyName, qualifier);
+          }
+        } else {
+          get.addFamily(familyName);
         }
       }
       if (opts.filterAll) {
@@ -1614,23 +1664,26 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       byte[] row = format(i);
       Put put = new Put(row);
-      for (int column = 0; column < opts.columns; column++) {
-        byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-        byte[] value = generateData(this.rand, getValueLength(this.rand));
-        if (opts.useTags) {
-          byte[] tag = generateData(this.rand, TAG_LENGTH);
-          Tag[] tags = new Tag[opts.noOfTags];
-          for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
-            tags[n] = t;
-          }
-          KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
+      for (int family = 0; family < opts.families; family++) {
+        byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
+        for (int column = 0; column < opts.columns; column++) {
+          byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+          byte[] value = generateData(this.rand, getValueLength(this.rand));
+          if (opts.useTags) {
+            byte[] tag = generateData(this.rand, TAG_LENGTH);
+            Tag[] tags = new Tag[opts.noOfTags];
+            for (int n = 0; n < opts.noOfTags; n++) {
+              Tag t = new Tag((byte) n, tag);
+              tags[n] = t;
+            }
+            KeyValue kv = new KeyValue(row, familyName, qualifier, HConstants.LATEST_TIMESTAMP,
               value, tags);
-          put.add(kv);
-          updateValueSize(kv.getValueLength());
-        } else {
-          put.add(FAMILY_NAME, qualifier, value);
-          updateValueSize(value.length);
+            put.add(kv);
+            updateValueSize(kv.getValueLength());
+          } else {
+            put.add(familyName, qualifier, value);
+            updateValueSize(value.length);
+          }
         }
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
@@ -1665,12 +1718,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     protected Scan constructScan(byte[] valuePrefix) throws IOException {
       FilterList list = new FilterList();
-      Filter filter = new SingleColumnValueFilter(
-          FAMILY_NAME, COLUMN_ZERO, CompareFilter.CompareOp.EQUAL,
-          new BinaryComparator(valuePrefix)
-      );
+      Filter filter = new SingleColumnValueFilter(FAMILY_ZERO, COLUMN_ZERO,
+        CompareFilter.CompareOp.EQUAL, new BinaryComparator(valuePrefix));
       list.addFilter(filter);
-      if(opts.filterAll) {
+      if (opts.filterAll) {
         list.addFilter(new FilterAllFilter());
       }
       Scan scan = new Scan();
@@ -1678,10 +1729,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
       if (opts.addColumns) {
         for (int column = 0; column < opts.columns; column++) {
           byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
-          scan.addColumn(FAMILY_NAME, qualifier);
+          scan.addColumn(FAMILY_ZERO, qualifier);
         }
       } else {
-        scan.addFamily(FAMILY_NAME);
+        scan.addFamily(FAMILY_ZERO);
       }
       scan.setFilter(list);
       return scan;
@@ -1694,9 +1745,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * @param timeMs Time taken in milliseconds.
    * @return String value with label, ie '123.76 MB/s'
    */
-  private static String calculateMbps(int rows, long timeMs, final int valueSize, int columns) {
+  private static String calculateMbps(int rows, long timeMs, final int valueSize, int families,
+      int columns) {
     BigDecimal rowSize = BigDecimal.valueOf(ROW_LENGTH +
-      ((valueSize + FAMILY_NAME.length + COLUMN_ZERO.length) * columns));
+      ((valueSize + (FAMILY_NAME_BASE.length()+1) + COLUMN_ZERO.length) * columns) * families);
     BigDecimal mbps = BigDecimal.valueOf(rows).multiply(rowSize, CXT)
       .divide(BigDecimal.valueOf(timeMs), CXT).multiply(MS_PER_SEC, CXT)
       .divide(BYTES_PER_MB, CXT);
@@ -1785,7 +1837,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     status.setStatus("Finished " + cmd + " in " + totalElapsedTime +
       "ms at offset " + opts.startRow + " for " + opts.perClientRunRows + " rows" +
       " (" + calculateMbps((int)(opts.perClientRunRows * opts.sampleRate), totalElapsedTime,
-          getAverageValueLength(opts), opts.columns) + ")");
+          getAverageValueLength(opts), opts.families, opts.columns) + ")");
 
     return new RunResult(totalElapsedTime, t.getLatency());
   }
@@ -1877,6 +1929,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println(" splitPolicy     Specify a custom RegionSplitPolicy for the table.");
     System.err.println(" randomSleep     Do a random sleep before each get between 0 and entered value. Defaults: 0");
     System.err.println(" columns         Columns to write per row. Default: 1");
+    System.err.println(" families        Specify number of column families for the table. Default: 1");
     System.err.println(" caching         Scan caching to use. Default: 30");
     System.err.println(" bufferSize      Set the value of client side buffering. Default: 2MB");
     System.err.println();
@@ -2104,6 +2157,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
         continue;
       }
 
+      final String families = "--families=";
+      if (cmd.startsWith(families)) {
+        opts.families = Integer.parseInt(cmd.substring(families.length()));
+        continue;
+      }
+
       final String caching = "--caching=";
       if (cmd.startsWith(caching)) {
         opts.caching = Integer.parseInt(cmd.substring(caching.length()));
@@ -2146,7 +2205,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
   }
 
   static int getRowsPerGB(final TestOptions opts) {
-    return ONE_GB / ((opts.valueRandom? opts.valueSize/2: opts.valueSize) * opts.getColumns());
+    return ONE_GB / ((opts.valueRandom? opts.valueSize/2: opts.valueSize) * opts.getColumns() *
+        opts.getFamilies());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef9fd29c/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
index abe8d5c..4d75dba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
@@ -104,9 +104,11 @@ public class TestHFileOutputFormat  {
       withTimeout(this.getClass()).withLookingForStuckThread(true).build();
   private final static int ROWSPERSPLIT = 1024;
 
-  private static final byte[][] FAMILIES
-    = { Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-A"))
-      , Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-B"))};
+  private static final byte[] FAMILY_NAME = Bytes.toBytes("info");
+  private static final byte[][] FAMILIES = {
+      Bytes.add(FAMILY_NAME, Bytes.toBytes("-A")),
+      Bytes.add(FAMILY_NAME, Bytes.toBytes("-B"))
+    };
   private static final TableName TABLE_NAME =
       TableName.valueOf("TestTable");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef9fd29c/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index f1beae1..95c753a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -105,9 +105,11 @@ public class TestHFileOutputFormat2  {
       withTimeout(this.getClass()).withLookingForStuckThread(true).build();
   private final static int ROWSPERSPLIT = 1024;
 
-  private static final byte[][] FAMILIES
-    = { Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-A"))
-      , Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-B"))};
+  private static final byte[] FAMILY_NAME = Bytes.toBytes("info");
+  private static final byte[][] FAMILIES = {
+      Bytes.add(FAMILY_NAME, Bytes.toBytes("-A")),
+      Bytes.add(FAMILY_NAME, Bytes.toBytes("-B"))
+    };
   private static final TableName TABLE_NAME =
       TableName.valueOf("TestTable");