You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/05/28 22:51:38 UTC

[05/32] git commit: ACCUMULO-2789 Add stress test with variable cell sizes

ACCUMULO-2789 Add stress test with variable cell sizes

Added a test that users can run to stress their instances. They can supply
variable cell sizes (currently row, column family and column qualifier are
supported), and the write clients will generate random data within those size
constraints. Users can also scan random tablets to add futher load onto the
system. A variable number of write and reader clients can be executed to run
the test under a mixed workload.

This test is an acid test-- it is used to verify that an instance can handle
the workload provided by it. It doesn't assert an actual state of the system,
beyond reporting what errors the write and scan clients will see.

See the included package-info.java and/or README.md files for more information.


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

Branch: refs/heads/ACCUMULO-378
Commit: 74acc30cafb8be0ee5c7cc457c5e09f0ea56a993
Parents: c16f105
Author: Bill Slacum <uj...@apache.org>
Authored: Fri May 16 14:06:15 2014 -0400
Committer: Bill Slacum <uj...@apache.org>
Committed: Thu May 22 07:20:23 2014 -0400

----------------------------------------------------------------------
 .../accumulo/test/stress/random/DataWriter.java |  50 ++++++
 .../test/stress/random/IntArgValidator.java     |  32 ++++
 .../random/RandomByteArrayMakerFunction.java    |  60 ++++++++
 .../test/stress/random/RandomByteArrays.java    |  36 +++++
 .../test/stress/random/RandomMutations.java     |  39 +++++
 .../accumulo/test/stress/random/Scan.java       | 124 +++++++++++++++
 .../accumulo/test/stress/random/ScanOpts.java   |  44 ++++++
 .../accumulo/test/stress/random/Stream.java     |  41 +++++
 .../accumulo/test/stress/random/Write.java      |  78 ++++++++++
 .../test/stress/random/WriteOptions.java        | 152 +++++++++++++++++++
 .../test/stress/random/package-info.java        |  36 +++++
 test/system/stress/README.md                    |  33 ++++
 test/system/stress/reader.sh                    |  34 +++++
 test/system/stress/writer.sh                    |  38 +++++
 14 files changed, 797 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/DataWriter.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/DataWriter.java b/test/src/main/java/org/apache/accumulo/test/stress/random/DataWriter.java
new file mode 100644
index 0000000..33a3984
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/DataWriter.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+
+public class DataWriter extends Stream<Void> {
+  private final BatchWriter writer;
+  private final RandomMutations mutations;
+  
+  public DataWriter(BatchWriter writer, RandomMutations mutations) {
+    this.writer = writer;
+    this.mutations = mutations;
+  }
+  
+  @Override
+  public Void next() {
+    try {
+      writer.addMutation(mutations.next());
+    } catch (MutationsRejectedException e) {
+      throw new RuntimeException(e);
+    }
+    return null;
+  }
+  
+  @Override
+  public void finalize() {
+    try {
+      this.writer.close();
+    } catch (MutationsRejectedException e) {
+      System.err.println("Error closing batch writer.");
+      e.printStackTrace();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/IntArgValidator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/IntArgValidator.java b/test/src/main/java/org/apache/accumulo/test/stress/random/IntArgValidator.java
new file mode 100644
index 0000000..6ba6ca9
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/IntArgValidator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.test.stress.random;
+
+import com.beust.jcommander.IValueValidator;
+import com.beust.jcommander.ParameterException;
+import com.google.common.base.Preconditions;
+
+public class IntArgValidator implements IValueValidator<Integer> {
+  
+  @Override
+  public void validate(String name, Integer value) throws ParameterException {
+    Preconditions.checkNotNull(value);
+    Preconditions.checkArgument(value > 0);
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrayMakerFunction.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrayMakerFunction.java b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrayMakerFunction.java
new file mode 100644
index 0000000..bfe0be8
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrayMakerFunction.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import java.util.Random;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class is one RBAMF. Don't mess with it, or it'll mess with you.
+ */
+public class RandomByteArrayMakerFunction {
+  private final Random random;
+  
+  public RandomByteArrayMakerFunction() {
+    this(0);
+  }
+  
+  public RandomByteArrayMakerFunction(int seed) {
+    random = new Random(seed);
+  }
+  
+  public byte[] make(int size) {
+    byte[] b = new byte[size];
+    random.nextBytes(b);
+    return b;
+  }
+  
+  public byte[] makeWithRandomSize(int max) {
+    return makeWithRandomSize(1, random.nextInt(max));
+  }
+  
+  public byte[] makeWithRandomSize(int min, int max) {
+    Preconditions.checkArgument(min > 0, "Min must be positive.");
+    Preconditions.checkArgument(max >= min, "Max must be greater than or equal to min.");
+    if (min == max) {
+      return make(min);
+    } else {
+      final int spread = max - min;
+      final int random_value = random.nextInt(spread);
+      // we pick a random number that's between 0 and (max - min), then add
+      // min as an offset to get a random number that's [min, max)
+      return make(random_value + min);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrays.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrays.java b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrays.java
new file mode 100644
index 0000000..62f7cb2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomByteArrays.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+/**
+ * An iterator that will create random byte arrays as it is looped over.
+ */
+public class RandomByteArrays extends Stream<byte[]> {
+  private final RandomByteArrayMakerFunction rbamf;
+  private final int min_array_size, max_array_size;
+  
+  public RandomByteArrays(RandomByteArrayMakerFunction rbamf, int min_array_size,
+      int max_array_size) {
+    this.rbamf = rbamf;
+    this.min_array_size = min_array_size;
+    this.max_array_size = max_array_size;
+  }
+  
+  public byte[] next() {
+    return rbamf.makeWithRandomSize(min_array_size, max_array_size);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/RandomMutations.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/RandomMutations.java b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomMutations.java
new file mode 100644
index 0000000..a22222b
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/RandomMutations.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import org.apache.accumulo.core.data.Mutation;
+
+public class RandomMutations extends Stream<Mutation> {
+  private final RandomByteArrays rows, column_families, column_qualifiers, values;
+  
+  public RandomMutations(RandomByteArrays rows, RandomByteArrays column_families,
+      RandomByteArrays column_qualifiers, RandomByteArrays values) {
+    this.rows = rows;
+    this.column_families = column_families;
+    this.column_qualifiers = column_qualifiers;
+    this.values = values;
+  }
+  
+  // TODO should we care about timestamps?
+  @Override
+  public Mutation next() {
+    Mutation m = new Mutation(rows.next());
+    m.put(column_families.next(), column_qualifiers.next(), values.next());
+    return m;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/Scan.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/Scan.java b/test/src/main/java/org/apache/accumulo/test/stress/random/Scan.java
new file mode 100644
index 0000000..00a1cec
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/Scan.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.collect.Lists;
+
+public class Scan {
+  
+  public static void main(String[] args) throws Exception {
+    ScanOpts opts = new ScanOpts();
+    opts.parseArgs(Scan.class.getName(), args);
+    
+    Connector connector = opts.getConnector();
+    Scanner scanner = connector.createScanner(opts.getTableName(), new Authorizations());
+    
+    if (opts.isolate) {
+      scanner.enableIsolation();
+    }
+    
+    Random tablet_index_generator = new Random(opts.scan_seed);
+    
+    LoopControl scanning_condition = opts.continuous ? new ContinuousLoopControl() :
+                                                       new IterativeLoopControl(opts.scan_iterations);
+    
+    while(scanning_condition.keepScanning()) {
+      Range range = pickRange(connector.tableOperations(), opts.getTableName(),
+          tablet_index_generator);
+      scanner.setRange(range);
+      try {
+        consume(scanner);
+      } catch (Exception e) {
+        System.err.println(
+            String.format(
+                "Exception while scanning range %s. Check the state of Accumulo for errors.",
+                range));
+        throw e;
+      }
+    }
+  }
+  
+  public static void consume(Iterable<?> iterable) {
+    Iterator<?> itr = iterable.iterator();
+    while (itr.hasNext()) {
+      itr.next();
+    }
+  }
+  
+  public static Range pickRange(TableOperations tops, String table, Random r)
+      throws TableNotFoundException, AccumuloSecurityException, AccumuloException {
+    ArrayList<Text> splits = Lists.newArrayList(tops.listSplits(table));
+    if (splits.isEmpty()) {
+      return new Range();
+    } else {
+      int index = r.nextInt(splits.size());
+      Text endRow = splits.get(index);
+      Text startRow = index == 0 ? null : splits.get(index - 1);
+      return new Range(startRow, false, endRow, true);
+    }
+  }
+  
+  /*
+   * These interfaces + implementations are used to determine
+   * how many times the scanner should look up a random tablet
+   * and scan it.
+   */
+  static interface LoopControl {
+    public boolean keepScanning();
+  }
+  
+  // Does a finite number of iterations
+  static class IterativeLoopControl implements LoopControl {
+    private final int max;
+    private int current;
+    
+    public IterativeLoopControl(int max) {
+      this.max = max;
+      this.current = 0;
+    }
+    
+    public boolean keepScanning() {
+      if(current < max) {
+        ++current;
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+  
+  // Does an infinite number of iterations
+  static class ContinuousLoopControl implements LoopControl {
+    public boolean keepScanning() {
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/ScanOpts.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/ScanOpts.java b/test/src/main/java/org/apache/accumulo/test/stress/random/ScanOpts.java
new file mode 100644
index 0000000..22024d1
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/ScanOpts.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import org.apache.accumulo.core.cli.ClientOnDefaultTable;
+
+import com.beust.jcommander.Parameter;
+
+class ScanOpts extends ClientOnDefaultTable {
+  @Parameter(names = "--isolate",
+      description = "true to turn on scan isolation, false to turn off. default is false.")
+  boolean isolate = false;
+  
+  @Parameter(names = "--num-iterations", description = "number of scan iterations")
+  int scan_iterations = 1024;
+  
+  @Parameter(names = "--continuous", description = "continuously scan the table. note that this overrides --num-iterations")
+  boolean continuous;
+  
+  @Parameter(names = "--scan-seed", description = "seed for randomly choosing tablets to scan")
+  int scan_seed = 1337;
+  
+  public ScanOpts() {
+    this(WriteOptions.DEFAULT_TABLE);
+  }
+  
+  public ScanOpts(String table) {
+    super(table);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/Stream.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/Stream.java b/test/src/main/java/org/apache/accumulo/test/stress/random/Stream.java
new file mode 100644
index 0000000..2e022e1
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/Stream.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import java.util.Iterator;
+
+/**
+ * Base class to model an infinite stream of data. A stream implements an iterator whose {{@link #hasNext()} method will always return true.
+ * 
+ * @param <T>
+ */
+public abstract class Stream<T> implements Iterator<T> {
+  
+  @Override
+  public final boolean hasNext() {
+    return true;
+  }
+  
+  @Override
+  public abstract T next();
+  
+  @Override
+  public final void remove() {
+    throw new UnsupportedOperationException();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/Write.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/Write.java b/test/src/main/java/org/apache/accumulo/test/stress/random/Write.java
new file mode 100644
index 0000000..2c817c3
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/Write.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+
+public class Write {
+  
+  public static void main(String[] args) throws Exception {
+    WriteOptions opts = new WriteOptions();
+    BatchWriterOpts batch_writer_opts = new BatchWriterOpts();
+    opts.parseArgs(Write.class.getName(), args, batch_writer_opts);
+    
+    opts.check();
+    
+    Connector c = opts.getConnector();
+    
+    if(opts.clear_table && c.tableOperations().exists(opts.getTableName())) {
+      try {
+      c.tableOperations().delete(opts.getTableName());
+      } catch(TableNotFoundException e) {
+        System.err.println("Couldn't delete the table because it doesn't exist any more.");
+      }
+    }
+    
+    if(!c.tableOperations().exists(opts.getTableName())) {
+      try {
+        c.tableOperations().create(opts.getTableName());
+      } catch (TableExistsException e) {
+        System.err.println("Couldn't create table ourselves, but that's ok. Continuing.");
+      }
+    }
+  
+    DataWriter dw = new DataWriter(c.createBatchWriter(opts.getTableName(), batch_writer_opts.getBatchWriterConfig()),
+        new RandomMutations(
+            //rows
+            new RandomByteArrays(
+                new RandomByteArrayMakerFunction(opts.row_seed),
+                opts.rowMin(),
+                opts.rowMax()),
+            //cfs
+            new RandomByteArrays(
+                new RandomByteArrayMakerFunction(opts.cf_seed),
+                opts.cfMin(),
+                opts.cfMax()),
+            //cqs
+            new RandomByteArrays(
+                new RandomByteArrayMakerFunction(opts.cq_seed),
+                opts.cqMin(),
+                opts.cqMax()),
+            //vals
+            new RandomByteArrays(
+                new RandomByteArrayMakerFunction(opts.value_seed),
+                opts.valueMin(),
+                opts.valueMax())));
+    
+    while(true) {
+      dw.next();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/WriteOptions.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/WriteOptions.java b/test/src/main/java/org/apache/accumulo/test/stress/random/WriteOptions.java
new file mode 100644
index 0000000..b246ab7
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/WriteOptions.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+import org.apache.accumulo.core.cli.ClientOnDefaultTable;
+
+import com.beust.jcommander.Parameter;
+
+class WriteOptions extends ClientOnDefaultTable {
+  static final String DEFAULT_TABLE = "stress_test";
+  static final int DEFAULT_MIN = 1, DEFAULT_MAX = 128, DEFAULT_SPREAD = DEFAULT_MAX - DEFAULT_MIN;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--min-row-size", description = "minimum row size")
+  Integer row_min;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--max-row-size", description = "maximum row size")
+  Integer row_max;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--min-cf-size", description = "minimum column family size")
+  Integer cf_min;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--max-cf-size", description = "maximum column family size")
+  Integer cf_max;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--min-cq-size", description = "minimum column qualifier size")
+  Integer cq_min;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--max-cq-size", description = "maximum column qualifier size")
+  Integer cq_max;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--min-value-size", description = "minimum value size")
+  Integer value_min;
+  
+  @Parameter(validateValueWith = IntArgValidator.class, names = "--max-value-size", description = "maximum value size")
+  Integer value_max;
+  
+  @Parameter(names = "--clear-table", description = "clears the table before ingesting")
+  boolean clear_table;
+  
+  @Parameter(names = "--row-seed", description = "seed for generating rows")
+  int row_seed = 87;
+  
+  @Parameter(names = "--cf-seed", description = "seed for generating column families")
+  int cf_seed = 7;
+  
+  @Parameter(names = "--cq-seed", description = "seed for generating column qualifiers")
+  int cq_seed = 43;
+  
+  @Parameter(names = "--value-seed", description = "seed for generating values")
+  int value_seed = 99;
+  
+  public WriteOptions(String table) {
+    super(table);
+  }
+  
+  public WriteOptions() {
+    this(DEFAULT_TABLE);
+  }
+  
+  private static int minOrDefault(Integer ref) {
+    return ref == null ? DEFAULT_MIN : ref;
+  }
+  
+  private static int calculateMax(Integer min_ref, Integer max_ref) {
+    if (max_ref == null) {
+      if (min_ref == null) {
+        return DEFAULT_MAX;
+      } else {
+        return min_ref + DEFAULT_SPREAD;
+      }
+    } else {
+      return max_ref;
+    }
+  }
+  
+  public void check() {
+    checkPair("ROW", row_min, row_max);
+    checkPair("COLUMN FAMILY", cf_min, cf_max);
+    checkPair("COLUMN QUALIFIER", cq_min, cq_max);
+    checkPair("VALUE", value_min, value_max);
+  }
+  
+  public void checkPair(String label, Integer min_ref, Integer max_ref) {
+    // we've already asserted that the numbers will either be
+    // 1) null
+    // 2) positive
+    // need to verify that they're coherent here
+    
+    if(min_ref == null && max_ref != null) {
+      // we don't support just specifying a max yet
+      throw new IllegalArgumentException(
+          String.format("[%s] Maximum value supplied, but no minimum. Must supply a minimum with a maximum value.",
+                        label));
+    } else if(min_ref != null && max_ref != null) {
+      // if a user supplied lower and upper bounds, we need to verify
+      // that min <= max
+      if(min_ref.compareTo(max_ref) > 0) {
+        throw new IllegalArgumentException(
+            String.format("[%s] Min value (%d) is greater than max value (%d)",
+                          label,
+                          min_ref,
+                          max_ref));
+      }
+    }
+  }
+  
+  public int rowMin() {
+    return minOrDefault(row_min);
+  }
+  
+  public int rowMax() {
+    return calculateMax(row_min, row_max);
+  }
+  
+  public int cfMin() {
+    return minOrDefault(cf_min);
+  }
+  
+  public int cfMax() {
+    return calculateMax(cf_min, cf_max);
+  }
+  
+  public int cqMin() {
+    return minOrDefault(cq_min);
+  }
+  
+  public int cqMax() {
+    return calculateMax(cf_min, cf_max);
+  }
+  
+  public int valueMin() {
+    return minOrDefault(value_min);
+  }
+  
+  public int valueMax() {
+    return calculateMax(value_min, value_max);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/src/main/java/org/apache/accumulo/test/stress/random/package-info.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/stress/random/package-info.java b/test/src/main/java/org/apache/accumulo/test/stress/random/package-info.java
new file mode 100644
index 0000000..4a489dc
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/stress/random/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.stress.random;
+
+/**
+ * This package contains utility classes designed to test Accumulo when large cells are being written. This is an attempt to observe the behavior Accumulo
+ * displays when compacting and reading these cells.
+ * 
+ * There are two components to this package: {@link org.apache.accumulo.test.stress.random.Write} and {@link org.apache.accumulo.test.stress.random.Scan}.
+ * 
+ * The {@link org.apache.accumulo.test.stress.random.Write} provides facilities for writing random sized cells. Users can configure minimum and maximum
+ * sized portions of a cell. The portions users can configure are the row, column family, column qualifier and value. Note that the sizes are uniformly
+ * distributed between the minimum and maximum values. See {@link org.apache.accumulo.test.stress.random.WriteOptions} for available options and default sizing
+ * information.
+ * 
+ * The Scan provides users with the ability to query tables generated by the Write. It will pick a tablet at random and scan the entire range. The
+ * amount of times this process is done is user configurable. By default, it happens 1,024 times. Users can also specify whether or not the scan should be
+ * isolated or not.
+ * 
+ * There is no shared state intended by either of these services. This allows multiple clients to be run in parallel, either on the same host or distributed
+ * across hosts.
+ */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/system/stress/README.md
----------------------------------------------------------------------
diff --git a/test/system/stress/README.md b/test/system/stress/README.md
new file mode 100644
index 0000000..1a0bbae
--- /dev/null
+++ b/test/system/stress/README.md
@@ -0,0 +1,33 @@
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at 
+ 
+    http://www.apache.org/licenses/LICENSE-2.0
+ 
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+
+Accumulo Stress Test
+--------------------
+
+This is an attempt to observe the behavior Accumulo displays when compacting and reading these cells. There are two components to this package: `org.apache.accumulo.test.stress.random.Write` and `org.apache.accumulo.test.stress.random.Scan`.
+
+Since this test is meant to stress the system, there is no pre-defined test period and no success criteria that we can programmatically check. During a successful test period, the Accumulo cluster should remain stable and functional. Possible failure conditions include component failures, such as tablet servers running out of memory.
+
+The `org.apache.accumulo.test.stress.random.Write` utility provides facilities for writing random sized cells. Users can configure minimum and maximum sized portions of a cell. The portions users can configure are the row, column family, column qualifier and value. Note that the sizes are uniformly distributed between the minimum and maximum values
+
+The `org.apache.accumulo.test.stress.random.Scan` utility provides users with the ability to query tables generated by the Write. It will pick a tablet at random and scan the entire range. The amount of times this process is performed is user configurable. By default, it happens 1,024 times. Users can also specify whether or not the scan should be isolated or not.
+
+There is no shared state intended by either of these services. This allows multiple clients to be run in parallel, either on the same host or distributedacross hosts.
+
+See the provided `test/system/stress/writer.sh` and `test/system/stress/reader.sh` scripts for example configuration options and how to launch the processes.
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/system/stress/reader.sh
----------------------------------------------------------------------
diff --git a/test/system/stress/reader.sh b/test/system/stress/reader.sh
new file mode 100755
index 0000000..8f10fa2
--- /dev/null
+++ b/test/system/stress/reader.sh
@@ -0,0 +1,34 @@
+#! /usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Edit the credentials to match your system
+USERPASS='-u root -p secret'
+INSTANCE='-z localhost -i inst'
+
+# This is the seed for the range picking logic used by the scanner.
+SCAN_SEED='--scan-seed 1337'
+
+# Controls the number of random tablets the scanner will read sequentially
+#SCAN_ITERATIONS='--num-iterations 1024'
+
+# Alternatively, we can just continously scan
+CONTINUOUS_SCAN='--continuous'
+
+# Controls whether or not the scan will be using an isolated scanner. Add this to the execution 
+#SCAN_ISOLATION='--isolate'
+
+../../../bin/accumulo org.apache.accumulo.test.stress.random.Scan $INSTANCE $USERPASS $SCAN_SEED $CONTINUOUS_SCAN
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/74acc30c/test/system/stress/writer.sh
----------------------------------------------------------------------
diff --git a/test/system/stress/writer.sh b/test/system/stress/writer.sh
new file mode 100755
index 0000000..6054ab1
--- /dev/null
+++ b/test/system/stress/writer.sh
@@ -0,0 +1,38 @@
+#! /usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Edit the credentials to match your system
+USERPASS='-u root -p secret'
+INSTANCE='-z localhost -i inst'
+
+# Edit these to change the range of each cell component. The size is in bytes.
+ROW_RANGE='--min-row-size 128 --max-row-size 128'
+CF_RANGE='--min-cf-size 128 --max-cf-size 128'
+CQ_RANGE='--min-cq-size 128 --max-cq-size 128'
+VALUE_RANGE='--min-value-size 1024 --max-value-size 2048'
+
+# These are the seeds for the random number generates used to generate each cell component.
+ROW_SEED='--row-seed 1'
+CF_SEED='--cf-seed 2'
+CQ_SEED='--cq-seed 3'
+VALUE_SEED='--value-seed 4'
+
+# Let's reset the table, for good measure
+../../../bin/accumulo shell $USERPASS -e 'deletetable -f stress_test'
+../../../bin/accumulo shell $USERPASS -e 'createtable stress_test'
+
+../../../bin/accumulo org.apache.accumulo.test.stress.random.Write $INSTANCE $USERPASS $ROW_RANGE $CF_RANGE $CQ_RANGE $VALUE_RANGE $ROW_SEED $CF_SEED $CQ_SEED $VALUE_SEED