You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/07/12 02:26:35 UTC

svn commit: r1360489 - /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/

Author: mbautin
Date: Thu Jul 12 00:26:35 2012
New Revision: 1360489

URL: http://svn.apache.org/viewvc?rev=1360489&view=rev
Log:
[HBASE-6215] Per-request profiling loadtest options

Author: aurickq

Summary: Ability to specify percent of RPC calls to use profiling in loadtest.

Test Plan: MR loadtest

Reviewers: kranganathan, liyintang

Reviewed By: kranganathan

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D510111

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetGenerator.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetOperation.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/MixedWorkloadGenerator.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutGenerator.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutOperation.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutReGenerator.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/SeparateWorkloadGenerator.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/VersionWorkloadGenerator.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetGenerator.java Thu Jul 12 00:26:35 2012
@@ -17,6 +17,7 @@ public class GetGenerator implements Ope
   private byte[] columnFamily;
   private Random random;
   private double verifyFraction;
+  private double profilingFraction;
   private int maxVersions;
   private long minTime;
   private long maxTime;
@@ -31,8 +32,9 @@ public class GetGenerator implements Ope
    * @param verifyFraction the fraction in [0,1] of operations to verify
    */
   public GetGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      double verifyFraction) {
-    this(columnFamily, keysWritten, verifyFraction, 1, 0, Long.MAX_VALUE);
+      double verifyFraction, double profilingFraction) {
+    this(columnFamily, keysWritten, verifyFraction,
+        profilingFraction, 1, 0, Long.MAX_VALUE);
   }
 
   /**
@@ -47,9 +49,10 @@ public class GetGenerator implements Ope
    * @param timeDelta the maximum allowed age of a version, in milliseconds
    */
   public GetGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      double verifyFraction, int maxVersions, long timeDelta) {
-    this(columnFamily, keysWritten, verifyFraction, maxVersions, 0, 0,
-        timeDelta);
+      double verifyFraction, double profilingFraction,
+      int maxVersions, long timeDelta) {
+    this(columnFamily, keysWritten, verifyFraction, profilingFraction,
+        maxVersions, 0, 0, timeDelta);
   }
 
   /**
@@ -65,21 +68,23 @@ public class GetGenerator implements Ope
    * @param maxTime the latest allowable timestamp on a version
    */
   public GetGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      double verifyFraction, int maxVersions, long minTime, long maxTime) {
-    this(columnFamily, keysWritten, verifyFraction, maxVersions, minTime,
-        maxTime, 0);
+      double verifyFraction, double profilingFraction, 
+      int maxVersions, long minTime, long maxTime) {
+    this(columnFamily, keysWritten, verifyFraction, profilingFraction,
+        maxVersions, minTime, maxTime, 0);
   }
 
   /**
    * Private constructor, used by public constructors to set all properties.
    */
   private GetGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      double verifyFraction, int maxVersions, long minTime, long maxTime,
-      long timeDelta) {
+      double verifyFraction, double profilingFraction, int maxVersions, 
+      long minTime, long maxTime, long timeDelta) {
     this.keysWritten = keysWritten;
     this.columnFamily = columnFamily;
     this.random = new Random();
     this.verifyFraction = verifyFraction;
+    this.profilingFraction = profilingFraction;
     this.maxVersions = maxVersions;
     this.minTime = minTime;
     this.maxTime = maxTime;
@@ -109,7 +114,7 @@ public class GetGenerator implements Ope
         e.printStackTrace();
       }
       boolean verify = random.nextDouble() < verifyFraction;
-      return new GetOperation(key, get, verify ? dataGenerator : null);
+      return new GetOperation(key, get, verify ? dataGenerator : null, profilingFraction);
     } catch (KeyCounter.NoKeysException e) {
       return null;
     }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetOperation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetOperation.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetOperation.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/GetOperation.java Thu Jul 12 00:26:35 2012
@@ -1,6 +1,7 @@
 package org.apache.hadoop.hbase.mapreduce.loadtest;
 
 import java.io.IOException;
+import java.util.Random;
 
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
@@ -16,6 +17,8 @@ public class GetOperation extends Operat
   private final Get get;
   private Result result;
   private DataGenerator dataGenerator;
+  private double profilingFraction;
+  private Random random;
 
   public Operation.Type getType() {
     return Operation.Type.GET;
@@ -29,14 +32,18 @@ public class GetOperation extends Operat
    * @param dataGenerator the DataGenerator to use to verify the result, or null
    *        if the result should not be verified
    */
-  public GetOperation(long key, Get get, DataGenerator dataGenerator) {
+  public GetOperation(long key, Get get, DataGenerator dataGenerator,
+      double profilingFraction) {
     this.key = key;
     this.get = get;
     this.result = null;
     this.dataGenerator = dataGenerator;
+    this.profilingFraction = profilingFraction;
+    this.random = new Random ();
   }
 
   public void perform(HTable table) throws IOException {
+    table.setProfiling(random.nextDouble() < profilingFraction);
     result = table.get(get);
   }
 

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/MixedWorkloadGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/MixedWorkloadGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/MixedWorkloadGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/MixedWorkloadGenerator.java Thu Jul 12 00:26:35 2012
@@ -20,11 +20,12 @@ public class MixedWorkloadGenerator exte
   private int insertWeight = 1;
   private int getWeight = 1;
   private double getVerificationFraction = 0.05;
+  private double getProfilingFraction = LoadTest.DEFAULT_PROFILING_FRACTION;
 
   public List<List<Workload>> generateWorkloads(int numWorkloads, String args) {
     if (args != null) {
       String[] splits = args.split(":");
-      if (splits.length != 5) {
+      if (splits.length != 6) {
         throw new IllegalArgumentException("Wrong number of argument splits");
       }
       opsPerSecond = Integer.parseInt(splits[0]);
@@ -32,6 +33,7 @@ public class MixedWorkloadGenerator exte
       insertWeight = Integer.parseInt(splits[2]);
       getWeight = Integer.parseInt(splits[3]);
       getVerificationFraction = Double.parseDouble(splits[4]);
+      getProfilingFraction = Double.parseDouble(splits[5]);
     }
 
     List<List<Workload>> workloads = new ArrayList<List<Workload>>(numWorkloads);
@@ -39,7 +41,8 @@ public class MixedWorkloadGenerator exte
       List<Workload> clientWorkloads = new ArrayList<Workload>();
       long startKey = Long.MAX_VALUE / numWorkloads * i;
       clientWorkloads.add(new MixedWorkload(startKey, opsPerSecond, numThreads,
-          insertWeight, getWeight, getVerificationFraction));
+          insertWeight, getWeight, getVerificationFraction,
+          getProfilingFraction));
       workloads.add(clientWorkloads);
     }
     return workloads;
@@ -60,23 +63,28 @@ public class MixedWorkloadGenerator exte
     private int insertWeight;
     private int getWeight;
     private double getVerificationFraction;
+    private double getProfilingFraction;
 
     public MixedWorkload(long startKey, int opsPerSecond, int numThreads,
-        int insertWeight, int getWeight, double getVerificationFraction) {
+        int insertWeight, int getWeight, double getVerificationFraction,
+        double getProfilingFraction) {
       this.startKey = startKey;
       this.opsPerSecond = opsPerSecond;
       this.numThreads = numThreads;
       this.insertWeight = insertWeight;
       this.getWeight = getWeight;
       this.getVerificationFraction = getVerificationFraction;
+      this.getProfilingFraction = getProfilingFraction;
     }
 
     public OperationGenerator constructGenerator() {
       KeyCounter keysWritten = new KeyCounter(startKey);
       PutGenerator insertGenerator =
-          new PutGenerator(columnFamily, keysWritten, startKey, true);
+          new PutGenerator(columnFamily, keysWritten, startKey, true,
+              getProfilingFraction);
       GetGenerator getGenerator =
-          new GetGenerator(columnFamily, keysWritten, getVerificationFraction);
+          new GetGenerator(columnFamily, keysWritten, getVerificationFraction,
+              getProfilingFraction);
 
       CompositeOperationGenerator compositeGenerator =
           new CompositeOperationGenerator();

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutGenerator.java Thu Jul 12 00:26:35 2012
@@ -17,6 +17,7 @@ public class PutGenerator implements Ope
   protected final AtomicLong nextKey;
   protected final byte[] columnFamily;
   protected final KeyCounter keysWritten;
+  protected double profilingFraction = LoadTest.DEFAULT_PROFILING_FRACTION;
 
   // If queue is null, then bulk inserts will be generated, otherwise, inserts
   // of single KeyValues will be generated.
@@ -31,7 +32,7 @@ public class PutGenerator implements Ope
    * @param bulkInsert if true, operations will each insert multiple key-values
    */
   public PutGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      long startKey, boolean bulkInsert) {
+      long startKey, boolean bulkInsert, double profilingFraction) {
     nextKey = new AtomicLong(startKey);
     this.columnFamily = columnFamily;
     this.keysWritten = keysWritten;
@@ -40,6 +41,7 @@ public class PutGenerator implements Ope
     } else {
       this.queue = new LinkedBlockingQueue<PutOperation>();
     }
+    this.profilingFraction = profilingFraction;
   }
 
   public Operation nextOperation(DataGenerator dataGenerator) {
@@ -79,7 +81,7 @@ public class PutGenerator implements Ope
   protected PutOperation getBulkPut(DataGenerator dataGenerator, long key) {
     Put put = dataGenerator.constructBulkPut(key, columnFamily);
     if (put != null) {
-      return new PutOperation(key, put, keysWritten);
+      return new PutOperation(key, put, keysWritten, profilingFraction);
     } else {
       // Key was defined to be skipped, mark it as complete so it can be read.
       keysWritten.markKey(key, true);
@@ -93,9 +95,10 @@ public class PutGenerator implements Ope
       AtomicInteger remainingParts = new AtomicInteger(puts.size());
       for (int i = 1; i < puts.size(); i++) {
         queue.offer(new PutOperation(key, puts.get(i), keysWritten,
-            remainingParts));
+            remainingParts, profilingFraction));
       }
-      return new PutOperation(key, puts.get(0), keysWritten, remainingParts);
+      return new PutOperation(key, puts.get(0), keysWritten, remainingParts, 
+          profilingFraction);
     } else {
       keysWritten.markKey(key, true);
       return null;

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutOperation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutOperation.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutOperation.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutOperation.java Thu Jul 12 00:26:35 2012
@@ -1,6 +1,7 @@
 package org.apache.hadoop.hbase.mapreduce.loadtest;
 
 import java.io.IOException;
+import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.client.HTable;
@@ -15,6 +16,8 @@ public class PutOperation extends Operat
   private final long key;
   private boolean success;
   private final KeyCounter writtenKeys;
+  private final double profilingFraction;
+  private Random random;
 
   // Multi-part operations need to coordinate between themselves so that only
   // one of the parts makes certain updates. Related parts share an atomic
@@ -29,8 +32,9 @@ public class PutOperation extends Operat
    * @param put
    * @param writtenKeys the sink of keys to be updated after execution
    */
-  public PutOperation(long key, Put put, KeyCounter writtenKeys) {
-    this(key, put, writtenKeys, null);
+  public PutOperation(long key, Put put, KeyCounter writtenKeys, 
+      double profilingFraction) {
+    this(key, put, writtenKeys, null, profilingFraction);
   }
 
   /**
@@ -42,14 +46,15 @@ public class PutOperation extends Operat
    * @param writtenKeys the sink of keys to be updated after execution
    */
   public PutOperation(long key, Put put, KeyCounter writtenKeys,
-      AtomicInteger partsRemaining) {
+      AtomicInteger partsRemaining, double profilingFraction) {
     this.put = put;
     this.key = key;
     this.writtenKeys = writtenKeys;
     this.success = false;
-
+    this.profilingFraction = profilingFraction;
     this.partsRemaining = partsRemaining;
     this.wasLast = false;
+    this.random = new Random();
   }
 
   public Operation.Type getType() {
@@ -59,6 +64,7 @@ public class PutOperation extends Operat
   }
 
   public void perform(HTable table) throws IOException {
+    table.setProfiling(random.nextDouble() < profilingFraction);
     table.put(put);
     success = true;
     if (partsRemaining != null) {

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutReGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutReGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutReGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/PutReGenerator.java Thu Jul 12 00:26:35 2012
@@ -16,8 +16,8 @@ public class PutReGenerator extends PutG
    * @param bulkInsert if true, operations will each insert multiple key-values
    */
   public PutReGenerator(byte[] columnFamily, KeyCounter keysWritten,
-      boolean bulkInsert) {
-    super(columnFamily, keysWritten, 0, bulkInsert);
+      boolean bulkInsert, double profilingFraction) {
+    super(columnFamily, keysWritten, 0, bulkInsert, profilingFraction);
   }
 
   /**

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/SeparateWorkloadGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/SeparateWorkloadGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/SeparateWorkloadGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/SeparateWorkloadGenerator.java Thu Jul 12 00:26:35 2012
@@ -20,12 +20,13 @@ public class SeparateWorkloadGenerator e
   private int getThreads = 20;
   private int getOpsPerSecond = getThreads * 1000000;
   private double getVerificationFraction = 0.05;
+  private double getProfilingFraction = 0.05;
 
   public List<List<Workload>> generateWorkloads(int numWorkloads, String args) {
 
     if (args != null) {
       String[] splits = args.split(":");
-      if (splits.length != 5) {
+      if (splits.length != 6) {
         throw new IllegalArgumentException("Wrong number of argument splits");
       }
       insertOpsPerSecond = Integer.parseInt(splits[0]);
@@ -33,6 +34,7 @@ public class SeparateWorkloadGenerator e
       getOpsPerSecond = Integer.parseInt(splits[2]);
       getThreads = Integer.parseInt(splits[3]);
       getVerificationFraction = Double.parseDouble(splits[4]);
+      getProfilingFraction = Double.parseDouble(splits[5]);
     }
 
     List<List<Workload>> workloads =
@@ -42,9 +44,9 @@ public class SeparateWorkloadGenerator e
       long startKey = Long.MAX_VALUE / numWorkloads * i;
       KeyCounter keysWritten = new KeyCounter(startKey);
       clientWorkloads.add(new GetWorkload(getOpsPerSecond, getThreads,
-          getVerificationFraction, keysWritten));
+          getVerificationFraction, getProfilingFraction, keysWritten));
       clientWorkloads.add(new InsertWorkload(startKey, insertOpsPerSecond,
-          insertThreads, keysWritten));
+          insertThreads, getProfilingFraction, keysWritten));
       workloads.add(clientWorkloads);
     }
     return workloads;
@@ -62,19 +64,22 @@ public class SeparateWorkloadGenerator e
     private int opsPerSecond;
     private int numThreads;
     private double getVerificationFraction;
+    private double getProfilingFraction = LoadTest.DEFAULT_PROFILING_FRACTION;
     private KeyCounter keysWritten;
 
     public GetWorkload(int opsPerSecond, int numThreads,
-        double getVerificationFraction, KeyCounter keysWritten) {
+        double getVerificationFraction, double getProfilingFraction,
+        KeyCounter keysWritten) {
       this.opsPerSecond = opsPerSecond;
       this.numThreads = numThreads;
       this.getVerificationFraction = getVerificationFraction;
+      this.getProfilingFraction = getProfilingFraction;
       this.keysWritten = keysWritten;
     }
 
     public OperationGenerator constructGenerator() {
       return new GetGenerator(columnFamily, keysWritten,
-          getVerificationFraction);
+          getVerificationFraction, getProfilingFraction);
     }
 
     public int getNumThreads() {
@@ -96,18 +101,21 @@ public class SeparateWorkloadGenerator e
     private long startKey;
     private int opsPerSecond;
     private int numThreads;
+    private double getProfilingFraction;
     private KeyCounter keysWritten;
 
     public InsertWorkload(long startKey, int opsPerSecond, int numThreads,
-        KeyCounter keysWritten) {
+        double getProfilingFraction, KeyCounter keysWritten) {
       this.startKey = startKey;
       this.opsPerSecond = opsPerSecond;
       this.numThreads = numThreads;
+      this.getProfilingFraction = getProfilingFraction;
       this.keysWritten = keysWritten;
     }
 
     public OperationGenerator constructGenerator() {
-      return new PutGenerator(columnFamily, keysWritten, startKey, true);
+      return new PutGenerator(columnFamily, keysWritten, startKey, true,
+          getProfilingFraction);
     }
 
     public int getNumThreads() {

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/VersionWorkloadGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/VersionWorkloadGenerator.java?rev=1360489&r1=1360488&r2=1360489&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/VersionWorkloadGenerator.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/mapreduce/loadtest/VersionWorkloadGenerator.java Thu Jul 12 00:26:35 2012
@@ -22,12 +22,13 @@ public class VersionWorkloadGenerator ex
   private int reinsertWeight = 1;
   private int getWeight = 1;
   private double getVerificationFraction = 0.05;
+  private double getProfilingFraction = LoadTest.DEFAULT_PROFILING_FRACTION;
 
   public List<List<Workload>> generateWorkloads(int numWorkloads, String args) {
 
     if (args != null) {
       String[] splits = args.split(":");
-      if (splits.length != 6) {
+      if (splits.length != 7) {
         throw new IllegalArgumentException("Wrong number of argument splits");
       }
       opsPerSecond = Integer.parseInt(splits[0]);
@@ -36,6 +37,7 @@ public class VersionWorkloadGenerator ex
       reinsertWeight = Integer.parseInt(splits[3]);
       getWeight = Integer.parseInt(splits[4]);
       getVerificationFraction = Double.parseDouble(splits[5]);
+      getProfilingFraction = Double.parseDouble(splits[6]);
     }
 
     List<List<Workload>> workloads =
@@ -45,7 +47,7 @@ public class VersionWorkloadGenerator ex
       long startKey = Long.MAX_VALUE / numWorkloads * i;
       clientWorkloads.add(new MessagesWorkload(startKey, opsPerSecond,
           numThreads, insertWeight, reinsertWeight, getWeight,
-          getVerificationFraction));
+          getVerificationFraction, getProfilingFraction));
       workloads.add(clientWorkloads);
     }
     return workloads;
@@ -71,10 +73,11 @@ public class VersionWorkloadGenerator ex
     private int reinsertWeight;
     private int getWeight;
     private double getVerificationFraction;
+    private double getProfilingFraction;
 
     public MessagesWorkload(long startKey, int opsPerSecond, int numThreads,
         int insertWeight, int getWeight, int reinsertWeight,
-        double getVerificationFraction) {
+        double getVerificationFraction, double getProfilingFraction) {
       this.startKey = startKey;
       this.opsPerSecond = opsPerSecond;
       this.numThreads = numThreads;
@@ -82,17 +85,20 @@ public class VersionWorkloadGenerator ex
       this.reinsertWeight = reinsertWeight;
       this.getWeight = getWeight;
       this.getVerificationFraction = getVerificationFraction;
+      this.getProfilingFraction = getProfilingFraction;
     }
 
     public OperationGenerator constructGenerator() {
       KeyCounter keysWritten = new KeyCounter(startKey);
       PutGenerator insertGenerator =
-          new PutGenerator(columnFamily, keysWritten, startKey, true);
+          new PutGenerator(columnFamily, keysWritten, startKey, true, 
+              getProfilingFraction);
       PutReGenerator insertReGenerator =
-          new PutReGenerator(columnFamily, keysWritten, true);
+          new PutReGenerator(columnFamily, keysWritten, true,
+              getProfilingFraction);
       GetGenerator getGenerator =
-          new GetGenerator(columnFamily, keysWritten, getVerificationFraction,
-              Integer.MAX_VALUE, 3600000);
+          new GetGenerator(columnFamily, keysWritten, getVerificationFraction, 
+              getProfilingFraction, Integer.MAX_VALUE, 3600000);
 
       CompositeOperationGenerator compositeGenerator =
           new CompositeOperationGenerator();