You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2017/01/03 20:55:53 UTC

[4/7] accumulo-testing git commit: ACCUMULO-4510 Adding Randomwalk code from Accumulo

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
new file mode 100644
index 0000000..f3caf15
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
@@ -0,0 +1,131 @@
+/*
+ * 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.testing.core.randomwalk.image;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.MessageDigest;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+import org.apache.hadoop.io.Text;
+
+public class Verify extends Test {
+
+  String indexTableName;
+  String imageTableName;
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    Random rand = new Random();
+
+    int maxVerify = Integer.parseInt(props.getProperty("maxVerify"));
+    int numVerifications = rand.nextInt(maxVerify - 1) + 1;
+
+    indexTableName = state.getString("indexTableName");
+    imageTableName = state.getString("imageTableName");
+
+    Connector conn = env.getConnector();
+
+    Scanner indexScanner = conn.createScanner(indexTableName, new Authorizations());
+    Scanner imageScanner = conn.createScanner(imageTableName, new Authorizations());
+
+    String uuid = UUID.randomUUID().toString();
+
+    MessageDigest alg = MessageDigest.getInstance("SHA-1");
+    alg.update(uuid.getBytes(UTF_8));
+
+    indexScanner.setRange(new Range(new Text(alg.digest()), null));
+    indexScanner.setBatchSize(numVerifications);
+
+    Text curRow = null;
+    int count = 0;
+    for (Entry<Key,Value> entry : indexScanner) {
+
+      curRow = entry.getKey().getRow();
+      String rowToVerify = entry.getValue().toString();
+
+      verifyRow(imageScanner, rowToVerify);
+
+      count++;
+      if (count == numVerifications) {
+        break;
+      }
+    }
+
+    if (count != numVerifications && curRow != null) {
+      Text lastRow = (Text) state.get("lastIndexRow");
+      if (lastRow.compareTo(curRow) != 0) {
+        log.error("Verified only " + count + " of " + numVerifications + " - curRow " + curRow + " lastKey " + lastRow);
+      }
+    }
+
+    int verified = ((Integer) state.get("verified")).intValue() + numVerifications;
+    log.debug("Verified " + numVerifications + " - Total " + verified);
+    state.set("verified", Integer.valueOf(verified));
+  }
+
+  public void verifyRow(Scanner scanner, String row) throws Exception {
+
+    scanner.setRange(new Range(new Text(row)));
+    scanner.clearColumns();
+    scanner.fetchColumnFamily(Write.CONTENT_COLUMN_FAMILY);
+    scanner.fetchColumn(Write.META_COLUMN_FAMILY, Write.SHA1_COLUMN_QUALIFIER);
+
+    Iterator<Entry<Key,Value>> scanIter = scanner.iterator();
+
+    if (scanIter.hasNext() == false) {
+      log.error("Found row(" + row + ") in " + indexTableName + " but not " + imageTableName);
+      return;
+    }
+
+    // get image
+    Entry<Key,Value> entry = scanIter.next();
+    byte[] imageBytes = entry.getValue().get();
+
+    MessageDigest alg = MessageDigest.getInstance("SHA-1");
+    alg.update(imageBytes);
+    byte[] localHash = alg.digest();
+
+    // get stored hash
+    entry = scanIter.next();
+    byte[] storedHash = entry.getValue().get();
+
+    if (localHash.length != storedHash.length) {
+      throw new Exception("Hash lens do not match for " + row);
+    }
+
+    for (int i = 0; i < localHash.length; i++) {
+      if (localHash[i] != storedHash[i]) {
+        throw new Exception("Hashes do not match for " + row);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
new file mode 100644
index 0000000..f7a2781
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
@@ -0,0 +1,97 @@
+/*
+ * 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.testing.core.randomwalk.image;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.MessageDigest;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MultiTableBatchWriter;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+import org.apache.hadoop.io.Text;
+
+public class Write extends Test {
+
+  static final Text UUID_COLUMN_QUALIFIER = new Text("uuid");
+  static final Text COUNT_COLUMN_QUALIFIER = new Text("count");
+  static final Text SHA1_COLUMN_QUALIFIER = new Text("sha1");
+  static final Text IMAGE_COLUMN_QUALIFIER = new Text("image");
+  static final Text META_COLUMN_FAMILY = new Text("meta");
+  static final Text CONTENT_COLUMN_FAMILY = new Text("content");
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
+
+    BatchWriter imagesBW = mtbw.getBatchWriter(state.getString("imageTableName"));
+    BatchWriter indexBW = mtbw.getBatchWriter(state.getString("indexTableName"));
+
+    String uuid = UUID.randomUUID().toString();
+    Mutation m = new Mutation(new Text(uuid));
+
+    // create a fake image between 4KB and 1MB
+    int maxSize = Integer.parseInt(props.getProperty("maxSize"));
+    int minSize = Integer.parseInt(props.getProperty("minSize"));
+
+    Random rand = new Random();
+    int numBytes = rand.nextInt(maxSize - minSize) + minSize;
+    byte[] imageBytes = new byte[numBytes];
+    rand.nextBytes(imageBytes);
+    m.put(CONTENT_COLUMN_FAMILY, IMAGE_COLUMN_QUALIFIER, new Value(imageBytes));
+
+    // store size
+    m.put(META_COLUMN_FAMILY, new Text("size"), new Value(String.format("%d", numBytes).getBytes(UTF_8)));
+
+    // store hash
+    MessageDigest alg = MessageDigest.getInstance("SHA-1");
+    alg.update(imageBytes);
+    byte[] hash = alg.digest();
+    m.put(META_COLUMN_FAMILY, SHA1_COLUMN_QUALIFIER, new Value(hash));
+
+    // update write counts
+    state.set("numWrites", state.getLong("numWrites") + 1);
+    Long totalWrites = state.getLong("totalWrites") + 1;
+    state.set("totalWrites", totalWrites);
+
+    // set count
+    m.put(META_COLUMN_FAMILY, COUNT_COLUMN_QUALIFIER, new Value(String.format("%d", totalWrites).getBytes(UTF_8)));
+
+    // add mutation
+    imagesBW.addMutation(m);
+
+    // now add mutation to index
+    Text row = new Text(hash);
+    m = new Mutation(row);
+    m.put(META_COLUMN_FAMILY, UUID_COLUMN_QUALIFIER, new Value(uuid.getBytes(UTF_8)));
+
+    indexBW.addMutation(m);
+
+    Text lastRow = (Text) state.get("lastIndexRow");
+    if (lastRow.compareTo(row) < 0) {
+      state.set("lastIndexRow", new Text(row));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
new file mode 100644
index 0000000..4ac6b47
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
@@ -0,0 +1,40 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.util.Properties;
+
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class Commit extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    env.getMultiTableBatchWriter().flush();
+
+    Long numWrites = state.getLong("numWrites");
+    Long totalWrites = state.getLong("totalWrites") + numWrites;
+
+    log.debug("Committed " + numWrites + " writes.  Total writes: " + totalWrites);
+
+    state.set("totalWrites", totalWrites);
+    state.set("numWrites", Long.valueOf(0));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
new file mode 100644
index 0000000..6552161
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
@@ -0,0 +1,92 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.ToolRunner;
+
+public class CopyTable extends Test {
+
+  private final TreeSet<Text> splits;
+
+  public CopyTable() {
+    splits = new TreeSet<>();
+    for (int i = 1; i < 10; i++) {
+      splits.add(new Text(Integer.toString(i)));
+    }
+  }
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    @SuppressWarnings("unchecked")
+    List<String> tables = (List<String>) state.get("tableList");
+    if (tables.isEmpty())
+      return;
+
+    Random rand = new Random();
+    String srcTableName = tables.remove(rand.nextInt(tables.size()));
+
+    int nextId = ((Integer) state.get("nextId")).intValue();
+    String dstTableName = String.format("%s_%d", state.getString("tableNamePrefix"), nextId);
+
+    String[] args = new String[8];
+    args[0] = "-libjars";
+    args[1] = getMapReduceJars();
+    args[2] = env.getUserName();
+    args[3] = env.getPassword();
+    if (null == args[3]) {
+      args[3] = env.getKeytab();
+    }
+    args[4] = srcTableName;
+    args[5] = env.getInstance().getInstanceName();
+    args[6] = env.getConfigProperty("ZOOKEEPERS");
+    args[7] = dstTableName;
+
+    log.debug("copying " + srcTableName + " to " + dstTableName);
+
+    env.getConnector().tableOperations().create(dstTableName);
+
+    env.getConnector().tableOperations().addSplits(dstTableName, splits);
+
+    if (ToolRunner.run(CachedConfiguration.getInstance(), new CopyTool(), args) != 0) {
+      log.error("Failed to run map/red verify");
+      return;
+    }
+
+    String tableId = Tables.getNameToIdMap(env.getInstance()).get(dstTableName);
+    log.debug("copied " + srcTableName + " to " + dstTableName + " (id - " + tableId + " )");
+
+    tables.add(dstTableName);
+
+    env.getConnector().tableOperations().delete(srcTableName);
+    log.debug("dropped " + srcTableName);
+
+    nextId++;
+    state.set("nextId", Integer.valueOf(nextId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTool.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTool.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTool.java
new file mode 100644
index 0000000..02da2e0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTool.java
@@ -0,0 +1,131 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.KerberosToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Tool;
+import org.apache.log4j.Logger;
+
+public class CopyTool extends Configured implements Tool {
+  protected final Logger log = Logger.getLogger(this.getClass());
+
+  public static class SeqMapClass extends Mapper<Key,Value,Text,Mutation> {
+    @Override
+    public void map(Key key, Value val, Context output) throws IOException, InterruptedException {
+      Mutation m = new Mutation(key.getRow());
+      m.put(key.getColumnFamily(), key.getColumnQualifier(), val);
+      output.write(null, m);
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Job job = Job.getInstance(getConf(), this.getClass().getSimpleName());
+    job.setJarByClass(this.getClass());
+
+    if (job.getJar() == null) {
+      log.error("M/R requires a jar file!  Run mvn package.");
+      return 1;
+    }
+
+    ClientConfiguration clientConf = new ClientConfiguration().withInstance(args[3]).withZkHosts(args[4]);
+
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    AccumuloInputFormat.setInputTableName(job, args[2]);
+    AccumuloInputFormat.setScanAuthorizations(job, Authorizations.EMPTY);
+    AccumuloInputFormat.setZooKeeperInstance(job, clientConf);
+
+    final String principal;
+    final AuthenticationToken token;
+    if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
+      // Use the Kerberos creds to request a DelegationToken for MapReduce to use
+      // We could use the specified keytab (args[1]), but we're already logged in and don't need to, so we can just use the current user
+      KerberosToken kt = new KerberosToken();
+      try {
+        UserGroupInformation user = UserGroupInformation.getCurrentUser();
+        if (!user.hasKerberosCredentials()) {
+          throw new IllegalStateException("Expected current user to have Kerberos credentials");
+        }
+
+        // Get the principal via UGI
+        principal = user.getUserName();
+
+        // Connector w/ the Kerberos creds
+        ZooKeeperInstance inst = new ZooKeeperInstance(clientConf);
+        Connector conn = inst.getConnector(principal, kt);
+
+        // Do the explicit check to see if the user has the permission to get a delegation token
+        if (!conn.securityOperations().hasSystemPermission(conn.whoami(), SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
+          log.error(principal + " doesn't have the " + SystemPermission.OBTAIN_DELEGATION_TOKEN.name()
+              + " SystemPermission neccesary to obtain a delegation token. MapReduce tasks cannot automatically use the client's"
+              + " credentials on remote servers. Delegation tokens provide a means to run MapReduce without distributing the user's credentials.");
+          throw new IllegalStateException(conn.whoami() + " does not have permission to obtain a delegation token");
+        }
+
+        // Fetch a delegation token from Accumulo
+        token = conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+
+      } catch (Exception e) {
+        final String msg = "Failed to acquire DelegationToken for use with MapReduce";
+        log.error(msg, e);
+        throw new RuntimeException(msg, e);
+      }
+    } else {
+      // Simple principal + password
+      principal = args[0];
+      token = new PasswordToken(args[1]);
+    }
+
+    AccumuloInputFormat.setConnectorInfo(job, principal, token);
+    AccumuloOutputFormat.setConnectorInfo(job, principal, token);
+
+    job.setMapperClass(SeqMapClass.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Mutation.class);
+
+    job.setNumReduceTasks(0);
+
+    job.setOutputFormatClass(AccumuloOutputFormat.class);
+    AccumuloOutputFormat.setCreateTables(job, true);
+    AccumuloOutputFormat.setDefaultTableName(job, args[5]);
+    AccumuloOutputFormat.setZooKeeperInstance(job, clientConf);
+
+    job.waitForCompletion(true);
+    return job.isSuccessful() ? 0 : 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
new file mode 100644
index 0000000..4250003
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
@@ -0,0 +1,67 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+import org.apache.hadoop.io.Text;
+
+public class CreateTable extends Test {
+
+  private final TreeSet<Text> splits;
+
+  public CreateTable() {
+    splits = new TreeSet<>();
+    for (int i = 1; i < 10; i++) {
+      splits.add(new Text(Integer.toString(i)));
+    }
+  }
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getConnector();
+
+    int nextId = ((Integer) state.get("nextId")).intValue();
+    String tableName = String.format("%s_%d", state.getString("tableNamePrefix"), nextId);
+    try {
+      conn.tableOperations().create(tableName);
+      // Add some splits to make the server's life easier
+      conn.tableOperations().addSplits(tableName, splits);
+      String tableId = Tables.getNameToIdMap(env.getInstance()).get(tableName);
+      log.debug("created " + tableName + " (id:" + tableId + ")");
+      // Add some splits to make the server's life easier
+      conn.tableOperations().addSplits(tableName, splits);
+      log.debug("created " + splits.size() + " splits on " + tableName);
+      @SuppressWarnings("unchecked")
+      List<String> tables = (List<String>) state.get("tableList");
+      tables.add(tableName);
+    } catch (TableExistsException e) {
+      log.warn("Failed to create " + tableName + " as it already exists");
+    }
+
+    nextId++;
+    state.set("nextId", Integer.valueOf(nextId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
new file mode 100644
index 0000000..61904ca
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
@@ -0,0 +1,51 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class DropTable extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    @SuppressWarnings("unchecked")
+    List<String> tables = (List<String>) state.get("tableList");
+
+    // don't drop a table if we only have one table or less
+    if (tables.size() <= 1) {
+      return;
+    }
+
+    Random rand = new Random();
+    String tableName = tables.remove(rand.nextInt(tables.size()));
+
+    try {
+      env.getConnector().tableOperations().delete(tableName);
+      log.debug("Dropped " + tableName);
+    } catch (TableNotFoundException e) {
+      log.error("Tried to drop table " + tableName + " but could not be found!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
new file mode 100644
index 0000000..5a7a415
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
@@ -0,0 +1,74 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.net.InetAddress;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MultiTableBatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.Fixture;
+import org.apache.accumulo.testing.core.randomwalk.State;
+
+public class MultiTableFixture extends Fixture {
+
+  @Override
+  public void setUp(State state, Environment env) throws Exception {
+
+    String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
+
+    state.set("tableNamePrefix", String.format("multi_%s_%s_%d", hostname, env.getPid(), System.currentTimeMillis()));
+    state.set("nextId", Integer.valueOf(0));
+    state.set("numWrites", Long.valueOf(0));
+    state.set("totalWrites", Long.valueOf(0));
+    state.set("tableList", new CopyOnWriteArrayList<String>());
+  }
+
+  @Override
+  public void tearDown(State state, Environment env) throws Exception {
+    // We have resources we need to clean up
+    if (env.isMultiTableBatchWriterInitialized()) {
+      MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
+      try {
+        mtbw.close();
+      } catch (MutationsRejectedException e) {
+        log.error("Ignoring mutations that weren't flushed", e);
+      }
+
+      // Reset the MTBW on the state to null
+      env.resetMultiTableBatchWriter();
+    }
+
+    Connector conn = env.getConnector();
+
+    @SuppressWarnings("unchecked")
+    List<String> tables = (List<String>) state.get("tableList");
+
+    for (String tableName : tables) {
+      try {
+        conn.tableOperations().delete(tableName);
+        log.debug("Dropping table " + tableName);
+      } catch (TableNotFoundException e) {
+        log.warn("Tried to drop table " + tableName + " but could not be found!");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
new file mode 100644
index 0000000..70a6b21
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
@@ -0,0 +1,47 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class OfflineTable extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    @SuppressWarnings("unchecked")
+    List<String> tables = (List<String>) state.get("tableList");
+
+    if (tables.size() <= 0) {
+      return;
+    }
+
+    Random rand = new Random();
+    String tableName = tables.get(rand.nextInt(tables.size()));
+
+    env.getConnector().tableOperations().offline(tableName, rand.nextBoolean());
+    log.debug("Table " + tableName + " offline ");
+    env.getConnector().tableOperations().online(tableName, rand.nextBoolean());
+    log.debug("Table " + tableName + " online ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
new file mode 100644
index 0000000..3c0c792
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
@@ -0,0 +1,89 @@
+/*
+ * 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.testing.core.randomwalk.multitable;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.MessageDigest;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+import org.apache.hadoop.io.Text;
+
+public class Write extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+
+    @SuppressWarnings("unchecked")
+    List<String> tables = (List<String>) state.get("tableList");
+
+    if (tables.isEmpty()) {
+      log.debug("No tables to ingest into");
+      return;
+    }
+
+    Random rand = new Random();
+    String tableName = tables.get(rand.nextInt(tables.size()));
+
+    BatchWriter bw = null;
+    try {
+      bw = env.getMultiTableBatchWriter().getBatchWriter(tableName);
+    } catch (TableOfflineException e) {
+      log.error("Table " + tableName + " is offline!");
+      return;
+    } catch (TableNotFoundException e) {
+      log.error("Table " + tableName + " not found!");
+      return;
+    }
+
+    Text meta = new Text("meta");
+    String uuid = UUID.randomUUID().toString();
+
+    Mutation m = new Mutation(new Text(uuid));
+
+    // create a fake payload between 4KB and 16KB
+    int numBytes = rand.nextInt(12000) + 4000;
+    byte[] payloadBytes = new byte[numBytes];
+    rand.nextBytes(payloadBytes);
+    m.put(meta, new Text("payload"), new Value(payloadBytes));
+
+    // store size
+    m.put(meta, new Text("size"), new Value(String.format("%d", numBytes).getBytes(UTF_8)));
+
+    // store hash
+    MessageDigest alg = MessageDigest.getInstance("SHA-1");
+    alg.update(payloadBytes);
+    m.put(meta, new Text("sha1"), new Value(alg.digest()));
+
+    // add mutation
+    bw.addMutation(m);
+
+    state.set("numWrites", state.getLong("numWrites") + 1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
new file mode 100644
index 0000000..c075541
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
@@ -0,0 +1,101 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+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.security.SystemPermission;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class AlterSystemPerm extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getConnector();
+    WalkingSecurity ws = new WalkingSecurity(state, env);
+
+    String action = props.getProperty("task", "toggle");
+    String perm = props.getProperty("perm", "random");
+
+    String targetUser = WalkingSecurity.get(state, env).getSysUserName();
+
+    SystemPermission sysPerm;
+    if (perm.equals("random")) {
+      Random r = new Random();
+      int i = r.nextInt(SystemPermission.values().length);
+      sysPerm = SystemPermission.values()[i];
+    } else
+      sysPerm = SystemPermission.valueOf(perm);
+
+    boolean hasPerm = ws.hasSystemPermission(targetUser, sysPerm);
+
+    // toggle
+    if (!"take".equals(action) && !"give".equals(action)) {
+      if (hasPerm != conn.securityOperations().hasSystemPermission(targetUser, sysPerm))
+        throw new AccumuloException("Test framework and accumulo are out of sync!");
+      if (hasPerm)
+        action = "take";
+      else
+        action = "give";
+    }
+
+    if ("take".equals(action)) {
+      try {
+        conn.securityOperations().revokeSystemPermission(targetUser, sysPerm);
+      } catch (AccumuloSecurityException ae) {
+        switch (ae.getSecurityErrorCode()) {
+          case GRANT_INVALID:
+            if (sysPerm.equals(SystemPermission.GRANT))
+              return;
+            throw new AccumuloException("Got GRANT_INVALID when not dealing with GRANT", ae);
+          case PERMISSION_DENIED:
+            throw new AccumuloException("Test user doesn't have root", ae);
+          case USER_DOESNT_EXIST:
+            throw new AccumuloException("System user doesn't exist and they SHOULD.", ae);
+          default:
+            throw new AccumuloException("Got unexpected exception", ae);
+        }
+      }
+      ws.revokeSystemPermission(targetUser, sysPerm);
+    } else if ("give".equals(action)) {
+      try {
+        conn.securityOperations().grantSystemPermission(targetUser, sysPerm);
+      } catch (AccumuloSecurityException ae) {
+        switch (ae.getSecurityErrorCode()) {
+          case GRANT_INVALID:
+            if (sysPerm.equals(SystemPermission.GRANT))
+              return;
+            throw new AccumuloException("Got GRANT_INVALID when not dealing with GRANT", ae);
+          case PERMISSION_DENIED:
+            throw new AccumuloException("Test user doesn't have root", ae);
+          case USER_DOESNT_EXIST:
+            throw new AccumuloException("System user doesn't exist and they SHOULD.", ae);
+          default:
+            throw new AccumuloException("Got unexpected exception", ae);
+        }
+      }
+      ws.grantSystemPermission(targetUser, sysPerm);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
new file mode 100644
index 0000000..0e613d8
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
@@ -0,0 +1,74 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.net.InetAddress;
+import java.util.Properties;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class AlterTable extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+
+    String tableName = WalkingSecurity.get(state, env).getTableName();
+    String namespaceName = WalkingSecurity.get(state, env).getNamespaceName();
+
+    boolean exists = WalkingSecurity.get(state, env).getTableExists();
+    boolean hasPermission = WalkingSecurity.get(state, env).canAlterTable(WalkingSecurity.get(state, env).getSysCredentials(), tableName, namespaceName);
+    String newTableName = String.format("security_%s_%s_%d", InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_"), env.getPid(),
+        System.currentTimeMillis());
+
+    renameTable(conn, state, env, tableName, newTableName, hasPermission, exists);
+  }
+
+  public static void renameTable(Connector conn, State state, Environment env, String oldName, String newName, boolean hasPermission, boolean tableExists)
+      throws AccumuloSecurityException, AccumuloException, TableExistsException {
+    try {
+      conn.tableOperations().rename(oldName, newName);
+    } catch (AccumuloSecurityException ae) {
+      if (ae.getSecurityErrorCode().equals(SecurityErrorCode.PERMISSION_DENIED)) {
+        if (hasPermission)
+          throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+        else
+          return;
+      } else if (ae.getSecurityErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
+        if (WalkingSecurity.get(state, env).userPassTransient(conn.whoami()))
+          return;
+      }
+      throw new AccumuloException("Got unexpected ae error code", ae);
+    } catch (TableNotFoundException tnfe) {
+      if (tableExists)
+        throw new TableExistsException(null, oldName, "Got a TableNotFoundException but it should exist", tnfe);
+      else
+        return;
+    }
+    WalkingSecurity.get(state, env).setTableName(newName);
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
new file mode 100644
index 0000000..381a801
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
@@ -0,0 +1,180 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+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.impl.Credentials;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class AlterTablePerm extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    alter(state, env, props);
+  }
+
+  public static void alter(State state, Environment env, Properties props) throws Exception {
+    String action = props.getProperty("task", "toggle");
+    String perm = props.getProperty("perm", "random");
+    String sourceUserProp = props.getProperty("source", "system");
+    String targetUser = props.getProperty("target", "table");
+    boolean tabExists = WalkingSecurity.get(state, env).getTableExists();
+
+    String target;
+    if ("table".equals(targetUser))
+      target = WalkingSecurity.get(state, env).getTabUserName();
+    else
+      target = WalkingSecurity.get(state, env).getSysUserName();
+
+    boolean exists = WalkingSecurity.get(state, env).userExists(target);
+    boolean tableExists = WalkingSecurity.get(state, env).getTableExists();
+
+    TablePermission tabPerm;
+    if (perm.equals("random")) {
+      Random r = new Random();
+      int i = r.nextInt(TablePermission.values().length);
+      tabPerm = TablePermission.values()[i];
+    } else
+      tabPerm = TablePermission.valueOf(perm);
+    String tableName = WalkingSecurity.get(state, env).getTableName();
+    boolean hasPerm = WalkingSecurity.get(state, env).hasTablePermission(target, tableName, tabPerm);
+    boolean canGive;
+    String sourceUser;
+    AuthenticationToken sourceToken;
+    if ("system".equals(sourceUserProp)) {
+      sourceUser = WalkingSecurity.get(state, env).getSysUserName();
+      sourceToken = WalkingSecurity.get(state, env).getSysToken();
+    } else if ("table".equals(sourceUserProp)) {
+      sourceUser = WalkingSecurity.get(state, env).getTabUserName();
+      sourceToken = WalkingSecurity.get(state, env).getTabToken();
+    } else {
+      sourceUser = env.getUserName();
+      sourceToken = env.getToken();
+    }
+    Connector conn = env.getInstance().getConnector(sourceUser, sourceToken);
+
+    canGive = WalkingSecurity.get(state, env).canGrantTable(new Credentials(sourceUser, sourceToken).toThrift(env.getInstance()), target,
+        WalkingSecurity.get(state, env).getTableName(), WalkingSecurity.get(state, env).getNamespaceName());
+
+    // toggle
+    if (!"take".equals(action) && !"give".equals(action)) {
+      try {
+        boolean res;
+        if (hasPerm != (res = env.getConnector().securityOperations().hasTablePermission(target, tableName, tabPerm)))
+          throw new AccumuloException("Test framework and accumulo are out of sync for user " + conn.whoami() + " for perm " + tabPerm.name()
+              + " with local vs. accumulo being " + hasPerm + " " + res);
+
+        if (hasPerm)
+          action = "take";
+        else
+          action = "give";
+      } catch (AccumuloSecurityException ae) {
+        switch (ae.getSecurityErrorCode()) {
+          case USER_DOESNT_EXIST:
+            if (exists)
+              throw new AccumuloException("Framework and Accumulo are out of sync, we think user exists", ae);
+            else
+              return;
+          case TABLE_DOESNT_EXIST:
+            if (tabExists)
+              throw new AccumuloException(conn.whoami(), ae);
+            else
+              return;
+          default:
+            throw ae;
+        }
+      }
+    }
+
+    boolean trans = WalkingSecurity.get(state, env).userPassTransient(conn.whoami());
+    if ("take".equals(action)) {
+      try {
+        conn.securityOperations().revokeTablePermission(target, tableName, tabPerm);
+      } catch (AccumuloSecurityException ae) {
+        switch (ae.getSecurityErrorCode()) {
+          case GRANT_INVALID:
+            throw new AccumuloException("Got a grant invalid on non-System.GRANT option", ae);
+          case PERMISSION_DENIED:
+            if (canGive)
+              throw new AccumuloException(conn.whoami() + " failed to revoke permission to " + target + " when it should have worked", ae);
+            return;
+          case USER_DOESNT_EXIST:
+            if (exists)
+              throw new AccumuloException("Table user doesn't exist and they SHOULD.", ae);
+            return;
+          case TABLE_DOESNT_EXIST:
+            if (tableExists)
+              throw new AccumuloException("Table doesn't exist but it should", ae);
+            return;
+          case BAD_CREDENTIALS:
+            if (!trans)
+              throw new AccumuloException("Bad credentials for user " + conn.whoami());
+            return;
+          default:
+            throw new AccumuloException("Got unexpected exception", ae);
+        }
+      }
+      WalkingSecurity.get(state, env).revokeTablePermission(target, tableName, tabPerm);
+    } else if ("give".equals(action)) {
+      try {
+        conn.securityOperations().grantTablePermission(target, tableName, tabPerm);
+      } catch (AccumuloSecurityException ae) {
+        switch (ae.getSecurityErrorCode()) {
+          case GRANT_INVALID:
+            throw new AccumuloException("Got a grant invalid on non-System.GRANT option", ae);
+          case PERMISSION_DENIED:
+            if (canGive)
+              throw new AccumuloException(conn.whoami() + " failed to give permission to " + target + " when it should have worked", ae);
+            return;
+          case USER_DOESNT_EXIST:
+            if (exists)
+              throw new AccumuloException("Table user doesn't exist and they SHOULD.", ae);
+            return;
+          case TABLE_DOESNT_EXIST:
+            if (tableExists)
+              throw new AccumuloException("Table doesn't exist but it should", ae);
+            return;
+          case BAD_CREDENTIALS:
+            if (!trans)
+              throw new AccumuloException("Bad credentials for user " + conn.whoami());
+            return;
+          default:
+            throw new AccumuloException("Got unexpected exception", ae);
+        }
+      }
+      WalkingSecurity.get(state, env).grantTablePermission(target, tableName, tabPerm);
+    }
+
+    if (!exists)
+      throw new AccumuloException("User shouldn't have existed, but apparantly does");
+    if (!tableExists)
+      throw new AccumuloException("Table shouldn't have existed, but apparantly does");
+    if (!canGive)
+      throw new AccumuloException(conn.whoami() + " shouldn't have been able to grant privilege");
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
new file mode 100644
index 0000000..a9548c4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
@@ -0,0 +1,82 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+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.impl.Credentials;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class Authenticate extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    authenticate(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken(), state, env, props);
+  }
+
+  public static void authenticate(String principal, AuthenticationToken token, State state, Environment env, Properties props) throws Exception {
+    String targetProp = props.getProperty("target");
+    boolean success = Boolean.parseBoolean(props.getProperty("valid"));
+
+    Connector conn = env.getInstance().getConnector(principal, token);
+
+    String target;
+
+    if (targetProp.equals("table")) {
+      target = WalkingSecurity.get(state, env).getTabUserName();
+    } else {
+      target = WalkingSecurity.get(state, env).getSysUserName();
+    }
+    boolean exists = WalkingSecurity.get(state, env).userExists(target);
+    // Copy so if failed it doesn't mess with the password stored in state
+    byte[] password = Arrays.copyOf(WalkingSecurity.get(state, env).getUserPassword(target), WalkingSecurity.get(state, env).getUserPassword(target).length);
+    boolean hasPermission = WalkingSecurity.get(state, env).canAskAboutUser(new Credentials(principal, token).toThrift(env.getInstance()), target);
+
+    if (!success)
+      for (int i = 0; i < password.length; i++)
+        password[i]++;
+
+    boolean result;
+
+    try {
+      result = conn.securityOperations().authenticateUser(target, new PasswordToken(password));
+    } catch (AccumuloSecurityException ae) {
+      switch (ae.getSecurityErrorCode()) {
+        case PERMISSION_DENIED:
+          if (exists && hasPermission)
+            throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+          else
+            return;
+        default:
+          throw new AccumuloException("Unexpected exception!", ae);
+      }
+    }
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+    if (result != (success && exists))
+      throw new AccumuloException("Authentication " + (result ? "succeeded" : "failed") + " when it should have "
+          + ((success && exists) ? "succeeded" : "failed") + " while the user exists? " + exists);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
new file mode 100644
index 0000000..e58db32
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
@@ -0,0 +1,94 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+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.impl.Credentials;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class ChangePass extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    String target = props.getProperty("target");
+    String source = props.getProperty("source");
+
+    String principal;
+    AuthenticationToken token;
+    if (source.equals("system")) {
+      principal = WalkingSecurity.get(state, env).getSysUserName();
+      token = WalkingSecurity.get(state, env).getSysToken();
+    } else {
+      principal = WalkingSecurity.get(state, env).getTabUserName();
+      token = WalkingSecurity.get(state, env).getTabToken();
+    }
+    Connector conn = env.getInstance().getConnector(principal, token);
+
+    boolean hasPerm;
+    boolean targetExists;
+    if (target.equals("table")) {
+      target = WalkingSecurity.get(state, env).getTabUserName();
+    } else
+      target = WalkingSecurity.get(state, env).getSysUserName();
+
+    targetExists = WalkingSecurity.get(state, env).userExists(target);
+
+    hasPerm = WalkingSecurity.get(state, env).canChangePassword(new Credentials(principal, token).toThrift(env.getInstance()), target);
+
+    Random r = new Random();
+
+    byte[] newPassw = new byte[r.nextInt(50) + 1];
+    for (int i = 0; i < newPassw.length; i++)
+      newPassw[i] = (byte) ((r.nextInt(26) + 65) & 0xFF);
+
+    PasswordToken newPass = new PasswordToken(newPassw);
+    try {
+      conn.securityOperations().changeLocalUserPassword(target, newPass);
+    } catch (AccumuloSecurityException ae) {
+      switch (ae.getSecurityErrorCode()) {
+        case PERMISSION_DENIED:
+          if (hasPerm)
+            throw new AccumuloException("Change failed when it should have succeeded to change " + target + "'s password", ae);
+          return;
+        case USER_DOESNT_EXIST:
+          if (targetExists)
+            throw new AccumuloException("User " + target + " doesn't exist and they SHOULD.", ae);
+          return;
+        case BAD_CREDENTIALS:
+          if (!WalkingSecurity.get(state, env).userPassTransient(conn.whoami()))
+            throw new AccumuloException("Bad credentials for user " + conn.whoami());
+          return;
+        default:
+          throw new AccumuloException("Got unexpected exception", ae);
+      }
+    }
+    WalkingSecurity.get(state, env).changePassword(target, newPass);
+    // Waiting 1 second for password to propogate through Zk
+    Thread.sleep(1000);
+    if (!hasPerm)
+      throw new AccumuloException("Password change succeeded when it should have failed for " + source + " changing the password for " + target + ".");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
new file mode 100644
index 0000000..c0b1cd9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
@@ -0,0 +1,75 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class CreateTable extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+
+    String tableName = WalkingSecurity.get(state, env).getTableName();
+
+    boolean exists = WalkingSecurity.get(state, env).getTableExists();
+    boolean hasPermission = WalkingSecurity.get(state, env).canCreateTable(WalkingSecurity.get(state, env).getSysCredentials(), null, null);
+
+    try {
+      conn.tableOperations().create(tableName);
+    } catch (AccumuloSecurityException ae) {
+      if (ae.getSecurityErrorCode().equals(SecurityErrorCode.PERMISSION_DENIED)) {
+        if (hasPermission)
+          throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+        else {
+          // create table anyway for sake of state
+          try {
+            env.getConnector().tableOperations().create(tableName);
+            WalkingSecurity.get(state, env).initTable(tableName);
+          } catch (TableExistsException tee) {
+            if (exists)
+              return;
+            else
+              throw new AccumuloException("Test and Accumulo are out of sync");
+          }
+          return;
+        }
+      } else
+        throw new AccumuloException("Got unexpected error", ae);
+    } catch (TableExistsException tee) {
+      if (!exists)
+        throw new TableExistsException(null, tableName, "Got a TableExistsException but it shouldn't have existed", tee);
+      else
+        return;
+    }
+    WalkingSecurity.get(state, env).initTable(tableName);
+    for (TablePermission tp : TablePermission.values())
+      WalkingSecurity.get(state, env).grantTablePermission(conn.whoami(), tableName, tp);
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
new file mode 100644
index 0000000..f604928
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
@@ -0,0 +1,70 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+
+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.security.tokens.PasswordToken;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class CreateUser extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+
+    String tableUserName = WalkingSecurity.get(state, env).getTabUserName();
+
+    boolean exists = WalkingSecurity.get(state, env).userExists(tableUserName);
+    boolean hasPermission = WalkingSecurity.get(state, env).canCreateUser(WalkingSecurity.get(state, env).getSysCredentials(), tableUserName);
+    PasswordToken tabUserPass = new PasswordToken("Super Sekret Table User Password");
+    try {
+      conn.securityOperations().createLocalUser(tableUserName, tabUserPass);
+    } catch (AccumuloSecurityException ae) {
+      switch (ae.getSecurityErrorCode()) {
+        case PERMISSION_DENIED:
+          if (hasPermission)
+            throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+          else {
+            // create user anyway for sake of state
+            if (!exists) {
+              env.getConnector().securityOperations().createLocalUser(tableUserName, tabUserPass);
+              WalkingSecurity.get(state, env).createUser(tableUserName, tabUserPass);
+              Thread.sleep(1000);
+            }
+            return;
+          }
+        case USER_EXISTS:
+          if (!exists)
+            throw new AccumuloException("Got security exception when the user shouldn't have existed", ae);
+          else
+            return;
+        default:
+          throw new AccumuloException("Got unexpected exception", ae);
+      }
+    }
+    WalkingSecurity.get(state, env).createUser(tableUserName, tabUserPass);
+    Thread.sleep(1000);
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
new file mode 100644
index 0000000..235c9ba
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
@@ -0,0 +1,87 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class DropTable extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    dropTable(state, env, props);
+  }
+
+  public static void dropTable(State state, Environment env, Properties props) throws Exception {
+    String sourceUser = props.getProperty("source", "system");
+    String principal;
+    AuthenticationToken token;
+    if (sourceUser.equals("table")) {
+      principal = WalkingSecurity.get(state, env).getTabUserName();
+      token = WalkingSecurity.get(state, env).getTabToken();
+    } else {
+      principal = WalkingSecurity.get(state, env).getSysUserName();
+      token = WalkingSecurity.get(state, env).getSysToken();
+    }
+    Connector conn = env.getInstance().getConnector(principal, token);
+
+    String tableName = WalkingSecurity.get(state, env).getTableName();
+    String namespaceName = WalkingSecurity.get(state, env).getNamespaceName();
+
+    boolean exists = WalkingSecurity.get(state, env).getTableExists();
+    boolean hasPermission = WalkingSecurity.get(state, env).canDeleteTable(new Credentials(principal, token).toThrift(env.getInstance()), tableName,
+        namespaceName);
+
+    try {
+      conn.tableOperations().delete(tableName);
+    } catch (AccumuloSecurityException ae) {
+      if (ae.getSecurityErrorCode().equals(SecurityErrorCode.PERMISSION_DENIED)) {
+        if (hasPermission)
+          throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+        else {
+          // Drop anyway for sake of state
+          env.getConnector().tableOperations().delete(tableName);
+          WalkingSecurity.get(state, env).cleanTablePermissions(tableName);
+          return;
+        }
+      } else if (ae.getSecurityErrorCode().equals(SecurityErrorCode.BAD_CREDENTIALS)) {
+        if (WalkingSecurity.get(state, env).userPassTransient(conn.whoami()))
+          return;
+      }
+      throw new AccumuloException("Got unexpected ae error code", ae);
+    } catch (TableNotFoundException tnfe) {
+      if (exists)
+        throw new TableExistsException(null, tableName, "Got a TableNotFOundException but it should have existed", tnfe);
+      else
+        return;
+    }
+    WalkingSecurity.get(state, env).cleanTablePermissions(tableName);
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
new file mode 100644
index 0000000..8d6080b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
@@ -0,0 +1,68 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.util.Properties;
+
+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.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.State;
+import org.apache.accumulo.testing.core.randomwalk.Test;
+
+public class DropUser extends Test {
+
+  @Override
+  public void visit(State state, Environment env, Properties props) throws Exception {
+    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+
+    String tableUserName = WalkingSecurity.get(state, env).getTabUserName();
+
+    boolean exists = WalkingSecurity.get(state, env).userExists(tableUserName);
+    boolean hasPermission = WalkingSecurity.get(state, env).canDropUser(WalkingSecurity.get(state, env).getSysCredentials(), tableUserName);
+
+    try {
+      conn.securityOperations().dropLocalUser(tableUserName);
+    } catch (AccumuloSecurityException ae) {
+      switch (ae.getSecurityErrorCode()) {
+        case PERMISSION_DENIED:
+          if (hasPermission)
+            throw new AccumuloException("Got a security exception when I should have had permission.", ae);
+          else {
+            if (exists) {
+              env.getConnector().securityOperations().dropLocalUser(tableUserName);
+              WalkingSecurity.get(state, env).dropUser(tableUserName);
+            }
+            return;
+          }
+
+        case USER_DOESNT_EXIST:
+          if (exists)
+            throw new AccumuloException("Got user DNE exception when user should exists.", ae);
+          else
+            return;
+        default:
+          throw new AccumuloException("Got unexpected exception", ae);
+      }
+    }
+    WalkingSecurity.get(state, env).dropUser(tableUserName);
+    Thread.sleep(1000);
+    if (!hasPermission)
+      throw new AccumuloException("Didn't get Security Exception when we should have");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/ac5b271c/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
new file mode 100644
index 0000000..edfd15f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
@@ -0,0 +1,120 @@
+/*
+ * 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.testing.core.randomwalk.security;
+
+import java.net.InetAddress;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.Fixture;
+import org.apache.accumulo.testing.core.randomwalk.State;
+
+public class SecurityFixture extends Fixture {
+
+  @Override
+  public void setUp(State state, Environment env) throws Exception {
+    String secTableName, systemUserName, tableUserName, secNamespaceName;
+    // A best-effort sanity check to guard against not password-based auth
+    ClientConfiguration clientConf = ClientConfiguration.loadDefault();
+    if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
+      throw new IllegalStateException("Security module currently cannot support Kerberos/SASL instances");
+    }
+
+    Connector conn = env.getConnector();
+
+    String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
+
+    systemUserName = String.format("system_%s", hostname);
+    tableUserName = String.format("table_%s", hostname);
+    secTableName = String.format("security_%s", hostname);
+    secNamespaceName = String.format("securityNs_%s", hostname);
+
+    if (conn.tableOperations().exists(secTableName))
+      conn.tableOperations().delete(secTableName);
+    Set<String> users = conn.securityOperations().listLocalUsers();
+    if (users.contains(tableUserName))
+      conn.securityOperations().dropLocalUser(tableUserName);
+    if (users.contains(systemUserName))
+      conn.securityOperations().dropLocalUser(systemUserName);
+
+    PasswordToken sysUserPass = new PasswordToken("sysUser");
+    conn.securityOperations().createLocalUser(systemUserName, sysUserPass);
+
+    WalkingSecurity.get(state, env).setTableName(secTableName);
+    WalkingSecurity.get(state, env).setNamespaceName(secNamespaceName);
+    state.set("rootUserPass", env.getToken());
+
+    WalkingSecurity.get(state, env).setSysUserName(systemUserName);
+    WalkingSecurity.get(state, env).createUser(systemUserName, sysUserPass);
+
+    WalkingSecurity.get(state, env).changePassword(tableUserName, new PasswordToken(new byte[0]));
+
+    WalkingSecurity.get(state, env).setTabUserName(tableUserName);
+
+    for (TablePermission tp : TablePermission.values()) {
+      WalkingSecurity.get(state, env).revokeTablePermission(systemUserName, secTableName, tp);
+      WalkingSecurity.get(state, env).revokeTablePermission(tableUserName, secTableName, tp);
+    }
+    for (SystemPermission sp : SystemPermission.values()) {
+      WalkingSecurity.get(state, env).revokeSystemPermission(systemUserName, sp);
+      WalkingSecurity.get(state, env).revokeSystemPermission(tableUserName, sp);
+    }
+    WalkingSecurity.get(state, env).changeAuthorizations(tableUserName, new Authorizations());
+  }
+
+  @Override
+  public void tearDown(State state, Environment env) throws Exception {
+    log.debug("One last validate");
+    Validate.validate(state, env, log);
+    Connector conn = env.getConnector();
+
+    if (WalkingSecurity.get(state, env).getTableExists()) {
+      String secTableName = WalkingSecurity.get(state, env).getTableName();
+      log.debug("Dropping tables: " + secTableName);
+
+      conn.tableOperations().delete(secTableName);
+    }
+
+    if (WalkingSecurity.get(state, env).getNamespaceExists()) {
+      String secNamespaceName = WalkingSecurity.get(state, env).getNamespaceName();
+      log.debug("Dropping namespace: " + secNamespaceName);
+
+      conn.namespaceOperations().delete(secNamespaceName);
+    }
+
+    if (WalkingSecurity.get(state, env).userExists(WalkingSecurity.get(state, env).getTabUserName())) {
+      String tableUserName = WalkingSecurity.get(state, env).getTabUserName();
+      log.debug("Dropping user: " + tableUserName);
+
+      conn.securityOperations().dropLocalUser(tableUserName);
+    }
+    String systemUserName = WalkingSecurity.get(state, env).getSysUserName();
+    log.debug("Dropping user: " + systemUserName);
+    conn.securityOperations().dropLocalUser(systemUserName);
+    WalkingSecurity.clearInstance();
+
+    // Allow user drops to propagate, in case a new security test starts
+    Thread.sleep(2000);
+  }
+}