You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2015/06/04 20:53:12 UTC

[31/43] accumulo git commit: ACCUMULO-3871 move ITs into distro jar, stop building test jar

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
new file mode 100644
index 0000000..9455456
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
@@ -0,0 +1,148 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.BatchScanner;
+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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+public class MetadataIT extends AccumuloClusterHarness {
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+  }
+
+  @Override
+  public int defaultTimeoutSeconds() {
+    return 2 * 60;
+  }
+
+  @Test
+  public void testFlushAndCompact() throws Exception {
+    Connector c = getConnector();
+    String tableNames[] = getUniqueNames(2);
+
+    // create a table to write some data to metadata table
+    c.tableOperations().create(tableNames[0]);
+
+    Scanner rootScanner = c.createScanner(RootTable.NAME, Authorizations.EMPTY);
+    rootScanner.setRange(MetadataSchema.TabletsSection.getRange());
+    rootScanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+
+    Set<String> files1 = new HashSet<String>();
+    for (Entry<Key,Value> entry : rootScanner)
+      files1.add(entry.getKey().getColumnQualifier().toString());
+
+    c.tableOperations().create(tableNames[1]);
+    c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+
+    Set<String> files2 = new HashSet<String>();
+    for (Entry<Key,Value> entry : rootScanner)
+      files2.add(entry.getKey().getColumnQualifier().toString());
+
+    // flush of metadata table should change file set in root table
+    Assert.assertTrue(files2.size() > 0);
+    Assert.assertNotEquals(files1, files2);
+
+    c.tableOperations().compact(MetadataTable.NAME, null, null, false, true);
+
+    Set<String> files3 = new HashSet<String>();
+    for (Entry<Key,Value> entry : rootScanner)
+      files3.add(entry.getKey().getColumnQualifier().toString());
+
+    // compaction of metadata table should change file set in root table
+    Assert.assertNotEquals(files2, files3);
+  }
+
+  @Test
+  public void mergeMeta() throws Exception {
+    Connector c = getConnector();
+    String[] names = getUniqueNames(5);
+    SortedSet<Text> splits = new TreeSet<Text>();
+    for (String id : "1 2 3 4 5".split(" ")) {
+      splits.add(new Text(id));
+    }
+    c.tableOperations().addSplits(MetadataTable.NAME, splits);
+    for (String tableName : names) {
+      c.tableOperations().create(tableName);
+    }
+    c.tableOperations().merge(MetadataTable.NAME, null, null);
+    Scanner s = c.createScanner(RootTable.NAME, Authorizations.EMPTY);
+    s.setRange(MetadataSchema.DeletesSection.getRange());
+    while (Iterators.size(s.iterator()) == 0) {
+      UtilWaitThread.sleep(100);
+    }
+    assertEquals(0, c.tableOperations().listSplits(MetadataTable.NAME).size());
+  }
+
+  @Test
+  public void batchScanTest() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+
+    // batch scan regular metadata table
+    BatchScanner s = c.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 1);
+    s.setRanges(Collections.singleton(new Range()));
+    int count = 0;
+    for (Entry<Key,Value> e : s) {
+      if (e != null)
+        count++;
+    }
+    s.close();
+    assertTrue(count > 0);
+
+    // batch scan root metadata table
+    s = c.createBatchScanner(RootTable.NAME, Authorizations.EMPTY, 1);
+    s.setRanges(Collections.singleton(new Range()));
+    count = 0;
+    for (Entry<Key,Value> e : s) {
+      if (e != null)
+        count++;
+    }
+    s.close();
+    assertTrue(count > 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
new file mode 100644
index 0000000..086dd1a
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
@@ -0,0 +1,116 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.trace.Tracer;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class MetadataMaxFilesIT extends ConfigurableMacBase {
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    Map<String,String> siteConfig = new HashMap<String,String>();
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1");
+    siteConfig.put(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), "10");
+    cfg.setSiteConfig(siteConfig);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 4 * 60;
+  }
+
+  @Test
+  public void test() throws Exception {
+    Connector c = getConnector();
+    SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 0; i < 1000; i++) {
+      splits.add(new Text(String.format("%03d", i)));
+    }
+    c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10000");
+    for (int i = 0; i < 5; i++) {
+      String tableName = "table" + i;
+      log.info("Creating " + tableName);
+      c.tableOperations().create(tableName);
+      log.info("adding splits");
+      c.tableOperations().addSplits(tableName, splits);
+      log.info("flushing");
+      c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+      c.tableOperations().flush(RootTable.NAME, null, null, true);
+    }
+    UtilWaitThread.sleep(20 * 1000);
+    log.info("shutting down");
+    assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
+    cluster.stop();
+    log.info("starting up");
+    cluster.start();
+
+    UtilWaitThread.sleep(30 * 1000);
+
+    while (true) {
+      MasterMonitorInfo stats = null;
+      Credentials creds = new Credentials("root", new PasswordToken(ROOT_PASSWORD));
+      Client client = null;
+      try {
+        ClientContext context = new ClientContext(c.getInstance(), creds, getClientConfig());
+        client = MasterClient.getConnectionWithRetry(context);
+        stats = client.getMasterStats(Tracer.traceInfo(), context.rpcCreds());
+      } finally {
+        if (client != null)
+          MasterClient.close(client);
+      }
+      int tablets = 0;
+      for (TabletServerStatus tserver : stats.tServerInfo) {
+        for (Entry<String,TableInfo> entry : tserver.tableMap.entrySet()) {
+          if (entry.getKey().startsWith("!") || entry.getKey().startsWith("+"))
+            continue;
+          tablets += entry.getValue().onlineTablets;
+        }
+      }
+      if (tablets == 5005)
+        break;
+      UtilWaitThread.sleep(1000);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
new file mode 100644
index 0000000..ab2c791
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
@@ -0,0 +1,56 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class MetadataSplitIT extends ConfigurableMacBase {
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "100ms"));
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 2 * 60;
+  }
+
+  @Test
+  public void test() throws Exception {
+    Connector c = getConnector();
+    assertEquals(1, c.tableOperations().listSplits(MetadataTable.NAME).size());
+    c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(), "500");
+    for (int i = 0; i < 10; i++) {
+      c.tableOperations().create("table" + i);
+      c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+    }
+    UtilWaitThread.sleep(10 * 1000);
+    assertTrue(c.tableOperations().listSplits(MetadataTable.NAME).size() > 2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
new file mode 100644
index 0000000..c59c52e
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
@@ -0,0 +1,121 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.net.URL;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.monitor.Monitor;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.commons.io.FileUtils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MonitorLoggingIT extends ConfigurableMacBase {
+  private static final Logger log = LoggerFactory.getLogger(MonitorLoggingIT.class);
+
+  @Override
+  public void beforeClusterStart(MiniAccumuloConfigImpl cfg) throws Exception {
+    cfg.setNumTservers(1);
+    File confDir = cfg.getConfDir();
+    try {
+      FileUtils.copyFileToDirectory(new File(MonitorLoggingIT.class.getResource("/conf/generic_logger.xml").toURI()), confDir);
+      FileUtils.copyFileToDirectory(new File(MonitorLoggingIT.class.getResource("/conf/monitor_logger.xml").toURI()), confDir);
+    } catch (Exception e) {
+      log.error("Failed to copy Log4J XML files to conf dir", e);
+    }
+  }
+
+  private static final int NUM_LOCATION_PASSES = 5;
+  private static final int LOCATION_DELAY_SECS = 5;
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 30 + ((NUM_LOCATION_PASSES + 2) * LOCATION_DELAY_SECS);
+  }
+
+  @Test
+  public void logToMonitor() throws Exception {
+    // Start the monitor.
+    log.debug("Starting Monitor");
+    Process monitor = cluster.exec(Monitor.class);
+
+    // Get monitor location to ensure it is running.
+    String monitorLocation = null;
+    for (int i = 0; i < NUM_LOCATION_PASSES; i++) {
+      Thread.sleep(LOCATION_DELAY_SECS * 1000);
+      try {
+        monitorLocation = getMonitor();
+        break;
+      } catch (KeeperException e) {
+        log.debug("Monitor not up yet, trying again in " + LOCATION_DELAY_SECS + " secs");
+      }
+    }
+    assertNotNull("Monitor failed to start within " + (LOCATION_DELAY_SECS * NUM_LOCATION_PASSES) + " secs", monitorLocation);
+    log.debug("Monitor running at " + monitorLocation);
+
+    // The tserver has to observe that the log-forwarding address
+    // changed in ZooKeeper. If we cause the error before the tserver
+    // updates, we'll never see the error on the monitor.
+    Thread.sleep(10000);
+
+    // Attempt a scan with an invalid iterator to force a log message in the monitor.
+    try {
+      Connector c = getConnector();
+      Scanner s = c.createScanner("accumulo.root", new Authorizations());
+      IteratorSetting cfg = new IteratorSetting(100, "incorrect", "java.lang.String");
+      s.addScanIterator(cfg);
+      s.iterator().next();
+    } catch (RuntimeException e) {
+      // expected, the iterator was bad
+    }
+
+    String result = "";
+    while (true) {
+      Thread.sleep(LOCATION_DELAY_SECS * 1000); // extra precaution to ensure monitor has opportunity to log
+
+      // Verify messages were received at the monitor.
+      URL url = new URL("http://" + monitorLocation + "/log");
+      log.debug("Fetching web page " + url);
+      result = FunctionalTestUtils.readAll(url.openStream());
+      if (result.contains("<pre class='logevent'>")) {
+        break;
+      }
+      log.debug("No messages found, waiting a little longer...");
+    }
+
+    assertTrue("No log messages found", result.contains("<pre class='logevent'>"));
+
+    // Shutdown cleanly.
+    log.debug("Stopping mini accumulo cluster");
+    Process shutdown = cluster.exec(Admin.class, "stopAll");
+    shutdown.waitFor();
+    assertTrue(shutdown.exitValue() == 0);
+    log.debug("success!");
+    monitor.destroy();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
new file mode 100644
index 0000000..7283c4d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
@@ -0,0 +1,132 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.net.URL;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Map;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSession;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.X509TrustManager;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.MonitorUtil;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Check SSL for the Monitor
+ *
+ */
+public class MonitorSslIT extends ConfigurableMacBase {
+  @BeforeClass
+  public static void initHttps() throws NoSuchAlgorithmException, KeyManagementException {
+    SSLContext ctx = SSLContext.getInstance("SSL");
+    TrustManager[] tm = new TrustManager[] {new TestTrustManager()};
+    ctx.init(new KeyManager[0], tm, new SecureRandom());
+    SSLContext.setDefault(ctx);
+    HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory());
+    HttpsURLConnection.setDefaultHostnameVerifier(new TestHostnameVerifier());
+  }
+
+  private static class TestTrustManager implements X509TrustManager {
+    @Override
+    public void checkClientTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public X509Certificate[] getAcceptedIssuers() {
+      return null;
+    }
+  }
+
+  private static class TestHostnameVerifier implements HostnameVerifier {
+    @Override
+    public boolean verify(String hostname, SSLSession session) {
+      return true;
+    }
+  }
+
+  @Override
+  public int defaultTimeoutSeconds() {
+    return 6 * 60;
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    super.configure(cfg, hadoopCoreSite);
+    File baseDir = createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName());
+    configureForSsl(cfg, getSslDir(baseDir));
+    Map<String,String> siteConfig = cfg.getSiteConfig();
+    siteConfig.put(Property.MONITOR_SSL_KEYSTORE.getKey(), siteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey()));
+    siteConfig.put(Property.MONITOR_SSL_KEYSTOREPASS.getKey(), siteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey()));
+    if (siteConfig.containsKey(Property.RPC_SSL_KEYSTORE_TYPE.getKey())) {
+      siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(), siteConfig.get(Property.RPC_SSL_KEYSTORE_TYPE.getKey()));
+    } else {
+      siteConfig.put(Property.MONITOR_SSL_KEYSTORETYPE.getKey(), Property.RPC_SSL_KEYSTORE_TYPE.getDefaultValue());
+    }
+    siteConfig.put(Property.MONITOR_SSL_TRUSTSTORE.getKey(), siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey()));
+    siteConfig.put(Property.MONITOR_SSL_TRUSTSTOREPASS.getKey(), siteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey()));
+    if (siteConfig.containsKey(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey())) {
+      siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(), siteConfig.get(Property.RPC_SSL_TRUSTSTORE_TYPE.getKey()));
+    } else {
+      siteConfig.put(Property.MONITOR_SSL_TRUSTSTORETYPE.getKey(), Property.RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue());
+    }
+    cfg.setSiteConfig(siteConfig);
+  }
+
+  @Test
+  public void test() throws Exception {
+    log.debug("Starting Monitor");
+    cluster.getClusterControl().startAllServers(ServerType.MONITOR);
+    String monitorLocation = null;
+    while (null == monitorLocation) {
+      try {
+        monitorLocation = MonitorUtil.getLocation(getConnector().getInstance());
+      } catch (Exception e) {
+        // ignored
+      }
+      if (null == monitorLocation) {
+        log.debug("Could not fetch monitor HTTP address from zookeeper");
+        Thread.sleep(2000);
+      }
+    }
+    URL url = new URL("https://" + monitorLocation);
+    log.debug("Fetching web page " + url);
+    String result = FunctionalTestUtils.readAll(url.openStream());
+    assertTrue(result.length() > 100);
+    assertTrue(result.indexOf("Accumulo Overview") >= 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
new file mode 100644
index 0000000..9175379
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
@@ -0,0 +1,613 @@
+/*
+ * 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.functional;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Random;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.tserver.NativeMap;
+import org.apache.hadoop.io.Text;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class NativeMapIT {
+
+  private Key nk(int r) {
+    return new Key(new Text(String.format("r%09d", r)));
+  }
+
+  private Key nk(int r, int cf, int cq, int cv, int ts, boolean deleted) {
+    Key k = new Key(new Text(String.format("r%09d", r)), new Text(String.format("cf%09d", cf)), new Text(String.format("cq%09d", cq)), new Text(String.format(
+        "cv%09d", cv)), ts);
+
+    k.setDeleted(deleted);
+
+    return k;
+  }
+
+  private Value nv(int v) {
+    return new Value(String.format("r%09d", v).getBytes(UTF_8));
+  }
+
+  public static File nativeMapLocation() {
+    File projectDir = new File(System.getProperty("user.dir")).getParentFile();
+    File nativeMapDir = new File(projectDir, "server/native/target/accumulo-native-" + Constants.VERSION + "/accumulo-native-" + Constants.VERSION);
+    return nativeMapDir;
+  }
+
+  @BeforeClass
+  public static void setUp() {
+    NativeMap.loadNativeLib(Collections.singletonList(nativeMapLocation()));
+  }
+
+  private void verifyIterator(int start, int end, int valueOffset, Iterator<Entry<Key,Value>> iter) {
+    for (int i = start; i <= end; i++) {
+      assertTrue(iter.hasNext());
+      Entry<Key,Value> entry = iter.next();
+      assertEquals(nk(i), entry.getKey());
+      assertEquals(nv(i + valueOffset), entry.getValue());
+    }
+
+    assertFalse(iter.hasNext());
+  }
+
+  private void insertAndVerify(NativeMap nm, int start, int end, int valueOffset) {
+    for (int i = start; i <= end; i++) {
+      nm.put(nk(i), nv(i + valueOffset));
+    }
+
+    for (int i = start; i <= end; i++) {
+      Value v = nm.get(nk(i));
+      assertNotNull(v);
+      assertEquals(nv(i + valueOffset), v);
+
+      Iterator<Entry<Key,Value>> iter2 = nm.iterator(nk(i));
+      assertTrue(iter2.hasNext());
+      Entry<Key,Value> entry = iter2.next();
+      assertEquals(nk(i), entry.getKey());
+      assertEquals(nv(i + valueOffset), entry.getValue());
+    }
+
+    assertNull(nm.get(nk(start - 1)));
+
+    assertNull(nm.get(nk(end + 1)));
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+    verifyIterator(start, end, valueOffset, iter);
+
+    for (int i = start; i <= end; i++) {
+      iter = nm.iterator(nk(i));
+      verifyIterator(i, end, valueOffset, iter);
+
+      // lookup nonexistant key that falls after existing key
+      iter = nm.iterator(nk(i, 1, 1, 1, 1, false));
+      verifyIterator(i + 1, end, valueOffset, iter);
+    }
+
+    assertEquals(end - start + 1, nm.size());
+  }
+
+  private void insertAndVerifyExhaustive(NativeMap nm, int num, int run) {
+    for (int i = 0; i < num; i++) {
+      for (int j = 0; j < num; j++) {
+        for (int k = 0; k < num; k++) {
+          for (int l = 0; l < num; l++) {
+            for (int ts = 0; ts < num; ts++) {
+              Key key = nk(i, j, k, l, ts, true);
+              Value value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + true + "_" + run).getBytes(UTF_8));
+
+              nm.put(key, value);
+
+              key = nk(i, j, k, l, ts, false);
+              value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + false + "_" + run).getBytes(UTF_8));
+
+              nm.put(key, value);
+            }
+          }
+        }
+      }
+    }
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+
+    for (int i = 0; i < num; i++) {
+      for (int j = 0; j < num; j++) {
+        for (int k = 0; k < num; k++) {
+          for (int l = 0; l < num; l++) {
+            for (int ts = num - 1; ts >= 0; ts--) {
+              Key key = nk(i, j, k, l, ts, true);
+              Value value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + true + "_" + run).getBytes(UTF_8));
+
+              assertTrue(iter.hasNext());
+              Entry<Key,Value> entry = iter.next();
+              assertEquals(key, entry.getKey());
+              assertEquals(value, entry.getValue());
+
+              key = nk(i, j, k, l, ts, false);
+              value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + false + "_" + run).getBytes(UTF_8));
+
+              assertTrue(iter.hasNext());
+              entry = iter.next();
+              assertEquals(key, entry.getKey());
+              assertEquals(value, entry.getValue());
+            }
+          }
+        }
+      }
+    }
+
+    assertFalse(iter.hasNext());
+
+    for (int i = 0; i < num; i++) {
+      for (int j = 0; j < num; j++) {
+        for (int k = 0; k < num; k++) {
+          for (int l = 0; l < num; l++) {
+            for (int ts = 0; ts < num; ts++) {
+              Key key = nk(i, j, k, l, ts, true);
+              Value value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + true + "_" + run).getBytes(UTF_8));
+
+              assertEquals(value, nm.get(key));
+
+              Iterator<Entry<Key,Value>> iter2 = nm.iterator(key);
+              assertTrue(iter2.hasNext());
+              Entry<Key,Value> entry = iter2.next();
+              assertEquals(key, entry.getKey());
+              assertEquals(value, entry.getValue());
+
+              key = nk(i, j, k, l, ts, false);
+              value = new Value((i + "_" + j + "_" + k + "_" + l + "_" + ts + "_" + false + "_" + run).getBytes(UTF_8));
+
+              assertEquals(value, nm.get(key));
+
+              Iterator<Entry<Key,Value>> iter3 = nm.iterator(key);
+              assertTrue(iter3.hasNext());
+              Entry<Key,Value> entry2 = iter3.next();
+              assertEquals(key, entry2.getKey());
+              assertEquals(value, entry2.getValue());
+            }
+          }
+        }
+      }
+    }
+
+    assertEquals(num * num * num * num * num * 2, nm.size());
+  }
+
+  @Test
+  public void test1() {
+    NativeMap nm = new NativeMap();
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+    assertFalse(iter.hasNext());
+    nm.delete();
+  }
+
+  @Test
+  public void test2() {
+    NativeMap nm = new NativeMap();
+
+    insertAndVerify(nm, 1, 10, 0);
+    insertAndVerify(nm, 1, 10, 1);
+    insertAndVerify(nm, 1, 10, 2);
+
+    nm.delete();
+  }
+
+  @Test
+  public void test4() {
+    NativeMap nm = new NativeMap();
+
+    insertAndVerifyExhaustive(nm, 3, 0);
+    insertAndVerifyExhaustive(nm, 3, 1);
+
+    nm.delete();
+  }
+
+  @Test
+  public void test5() {
+    NativeMap nm = new NativeMap();
+
+    insertAndVerify(nm, 1, 10, 0);
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+    iter.next();
+
+    nm.delete();
+
+    try {
+      nm.put(nk(1), nv(1));
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+    try {
+      nm.get(nk(1));
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+    try {
+      nm.iterator();
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+    try {
+      nm.iterator(nk(1));
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+    try {
+      nm.size();
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+    try {
+      iter.next();
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+
+  }
+
+  @Test
+  public void test7() {
+    NativeMap nm = new NativeMap();
+
+    insertAndVerify(nm, 1, 10, 0);
+
+    nm.delete();
+
+    try {
+      nm.delete();
+      assertTrue(false);
+    } catch (IllegalStateException e) {
+
+    }
+  }
+
+  @Test
+  public void test8() {
+    // test verifies that native map sorts keys sharing some common prefix properly
+
+    NativeMap nm = new NativeMap();
+
+    TreeMap<Key,Value> tm = new TreeMap<Key,Value>();
+
+    tm.put(new Key(new Text("fo")), new Value(new byte[] {'0'}));
+    tm.put(new Key(new Text("foo")), new Value(new byte[] {'1'}));
+    tm.put(new Key(new Text("foo1")), new Value(new byte[] {'2'}));
+    tm.put(new Key(new Text("foo2")), new Value(new byte[] {'3'}));
+
+    for (Entry<Key,Value> entry : tm.entrySet()) {
+      nm.put(entry.getKey(), entry.getValue());
+    }
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+
+    for (Entry<Key,Value> entry : tm.entrySet()) {
+      assertTrue(iter.hasNext());
+      Entry<Key,Value> entry2 = iter.next();
+
+      assertEquals(entry.getKey(), entry2.getKey());
+      assertEquals(entry.getValue(), entry2.getValue());
+    }
+
+    assertFalse(iter.hasNext());
+
+    nm.delete();
+  }
+
+  @Test
+  public void test9() {
+    NativeMap nm = new NativeMap();
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+
+    try {
+      iter.next();
+      assertTrue(false);
+    } catch (NoSuchElementException e) {
+
+    }
+
+    insertAndVerify(nm, 1, 1, 0);
+
+    iter = nm.iterator();
+    iter.next();
+
+    try {
+      iter.next();
+      assertTrue(false);
+    } catch (NoSuchElementException e) {
+
+    }
+
+    nm.delete();
+  }
+
+  @Test
+  public void test10() {
+    int start = 1;
+    int end = 10000;
+
+    NativeMap nm = new NativeMap();
+    for (int i = start; i <= end; i++) {
+      nm.put(nk(i), nv(i));
+    }
+
+    long mem1 = nm.getMemoryUsed();
+
+    for (int i = start; i <= end; i++) {
+      nm.put(nk(i), nv(i));
+    }
+
+    long mem2 = nm.getMemoryUsed();
+
+    if (mem1 != mem2) {
+      throw new RuntimeException("Memory changed after inserting duplicate data " + mem1 + " " + mem2);
+    }
+
+    for (int i = start; i <= end; i++) {
+      nm.put(nk(i), nv(i));
+    }
+
+    long mem3 = nm.getMemoryUsed();
+
+    if (mem1 != mem3) {
+      throw new RuntimeException("Memory changed after inserting duplicate data " + mem1 + " " + mem3);
+    }
+
+    byte bigrow[] = new byte[1000000];
+    byte bigvalue[] = new byte[bigrow.length];
+
+    for (int i = 0; i < bigrow.length; i++) {
+      bigrow[i] = (byte) (0xff & (i % 256));
+      bigvalue[i] = bigrow[i];
+    }
+
+    nm.put(new Key(new Text(bigrow)), new Value(bigvalue));
+
+    long mem4 = nm.getMemoryUsed();
+
+    Value val = nm.get(new Key(new Text(bigrow)));
+    if (val == null || !val.equals(new Value(bigvalue))) {
+      throw new RuntimeException("Did not get expected big value");
+    }
+
+    nm.put(new Key(new Text(bigrow)), new Value(bigvalue));
+
+    long mem5 = nm.getMemoryUsed();
+
+    if (mem4 != mem5) {
+      throw new RuntimeException("Memory changed after inserting duplicate data " + mem4 + " " + mem5);
+    }
+
+    val = nm.get(new Key(new Text(bigrow)));
+    if (val == null || !val.equals(new Value(bigvalue))) {
+      throw new RuntimeException("Did not get expected big value");
+    }
+
+    nm.delete();
+  }
+
+  // random length random field
+  private static byte[] rlrf(Random r, int maxLen) {
+    int len = r.nextInt(maxLen);
+
+    byte f[] = new byte[len];
+    r.nextBytes(f);
+
+    return f;
+  }
+
+  @Test
+  public void test11() {
+    NativeMap nm = new NativeMap();
+
+    // insert things with varying field sizes and value sizes
+
+    // generate random data
+    Random r = new Random(75);
+
+    ArrayList<Pair<Key,Value>> testData = new ArrayList<Pair<Key,Value>>();
+
+    for (int i = 0; i < 100000; i++) {
+
+      Key k = new Key(rlrf(r, 97), rlrf(r, 13), rlrf(r, 31), rlrf(r, 11), (r.nextLong() & 0x7fffffffffffffffl), false, false);
+      Value v = new Value(rlrf(r, 511));
+
+      testData.add(new Pair<Key,Value>(k, v));
+    }
+
+    // insert unsorted data
+    for (Pair<Key,Value> pair : testData) {
+      nm.put(pair.getFirst(), pair.getSecond());
+    }
+
+    for (int i = 0; i < 2; i++) {
+
+      // sort data
+      Collections.sort(testData, new Comparator<Pair<Key,Value>>() {
+        @Override
+        public int compare(Pair<Key,Value> o1, Pair<Key,Value> o2) {
+          return o1.getFirst().compareTo(o2.getFirst());
+        }
+      });
+
+      // verify
+      Iterator<Entry<Key,Value>> iter1 = nm.iterator();
+      Iterator<Pair<Key,Value>> iter2 = testData.iterator();
+
+      while (iter1.hasNext() && iter2.hasNext()) {
+        Entry<Key,Value> e = iter1.next();
+        Pair<Key,Value> p = iter2.next();
+
+        if (!e.getKey().equals(p.getFirst()))
+          throw new RuntimeException("Keys not equal");
+
+        if (!e.getValue().equals(p.getSecond()))
+          throw new RuntimeException("Values not equal");
+      }
+
+      if (iter1.hasNext())
+        throw new RuntimeException("Not all of native map consumed");
+
+      if (iter2.hasNext())
+        throw new RuntimeException("Not all of test data consumed");
+
+      System.out.println("test 11 nm mem " + nm.getMemoryUsed());
+
+      // insert data again w/ different value
+      Collections.shuffle(testData, r);
+      // insert unsorted data
+      for (Pair<Key,Value> pair : testData) {
+        pair.getSecond().set(rlrf(r, 511));
+        nm.put(pair.getFirst(), pair.getSecond());
+      }
+    }
+
+    nm.delete();
+  }
+
+  @Test
+  public void testBinary() {
+    NativeMap nm = new NativeMap();
+
+    byte emptyBytes[] = new byte[0];
+
+    for (int i = 0; i < 256; i++) {
+      for (int j = 0; j < 256; j++) {
+        byte row[] = new byte[] {'r', (byte) (0xff & i), (byte) (0xff & j)};
+        byte data[] = new byte[] {'v', (byte) (0xff & i), (byte) (0xff & j)};
+
+        Key k = new Key(row, emptyBytes, emptyBytes, emptyBytes, 1);
+        Value v = new Value(data);
+
+        nm.put(k, v);
+      }
+    }
+
+    Iterator<Entry<Key,Value>> iter = nm.iterator();
+    for (int i = 0; i < 256; i++) {
+      for (int j = 0; j < 256; j++) {
+        byte row[] = new byte[] {'r', (byte) (0xff & i), (byte) (0xff & j)};
+        byte data[] = new byte[] {'v', (byte) (0xff & i), (byte) (0xff & j)};
+
+        Key k = new Key(row, emptyBytes, emptyBytes, emptyBytes, 1);
+        Value v = new Value(data);
+
+        assertTrue(iter.hasNext());
+        Entry<Key,Value> entry = iter.next();
+
+        assertEquals(k, entry.getKey());
+        assertEquals(v, entry.getValue());
+
+      }
+    }
+
+    assertFalse(iter.hasNext());
+
+    for (int i = 0; i < 256; i++) {
+      for (int j = 0; j < 256; j++) {
+        byte row[] = new byte[] {'r', (byte) (0xff & i), (byte) (0xff & j)};
+        byte data[] = new byte[] {'v', (byte) (0xff & i), (byte) (0xff & j)};
+
+        Key k = new Key(row, emptyBytes, emptyBytes, emptyBytes, 1);
+        Value v = new Value(data);
+
+        Value v2 = nm.get(k);
+
+        assertEquals(v, v2);
+      }
+    }
+
+    nm.delete();
+  }
+
+  @Test
+  public void testEmpty() {
+    NativeMap nm = new NativeMap();
+
+    assertTrue(nm.size() == 0);
+    assertTrue(nm.getMemoryUsed() == 0);
+
+    nm.delete();
+  }
+
+  @Test
+  public void testConcurrentIter() throws IOException {
+    NativeMap nm = new NativeMap();
+
+    nm.put(nk(0), nv(0));
+    nm.put(nk(1), nv(1));
+    nm.put(nk(3), nv(3));
+
+    SortedKeyValueIterator<Key,Value> iter = nm.skvIterator();
+
+    // modify map after iter created
+    nm.put(nk(2), nv(2));
+
+    assertTrue(iter.hasTop());
+    assertEquals(iter.getTopKey(), nk(0));
+    iter.next();
+
+    assertTrue(iter.hasTop());
+    assertEquals(iter.getTopKey(), nk(1));
+    iter.next();
+
+    assertTrue(iter.hasTop());
+    assertEquals(iter.getTopKey(), nk(2));
+    iter.next();
+
+    assertTrue(iter.hasTop());
+    assertEquals(iter.getTopKey(), nk(3));
+    iter.next();
+
+    assertFalse(iter.hasTop());
+
+    nm.delete();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/01ae5b85/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
new file mode 100644
index 0000000..8700891
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -0,0 +1,707 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+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.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+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.metadata.MetadataTable;
+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.harness.AccumuloClusterHarness;
+import org.apache.hadoop.io.Text;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// This test verifies the default permissions so a clean instance must be used. A shared instance might
+// not be representative of a fresh installation.
+public class PermissionsIT extends AccumuloClusterHarness {
+  private static final Logger log = LoggerFactory.getLogger(PermissionsIT.class);
+
+  @Override
+  public int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Before
+  public void limitToMini() throws Exception {
+    Assume.assumeTrue(ClusterType.MINI == getClusterType());
+    Connector c = getConnector();
+    Set<String> users = c.securityOperations().listLocalUsers();
+    ClusterUser user = getUser(0);
+    if (users.contains(user.getPrincipal())) {
+      c.securityOperations().dropLocalUser(user.getPrincipal());
+    }
+  }
+
+  private void loginAs(ClusterUser user) throws IOException {
+    // Force a re-login as the provided user
+    user.getToken();
+  }
+
+  @Test
+  public void systemPermissionsTest() throws Exception {
+    ClusterUser testUser = getUser(0), rootUser = getAdminUser();
+
+    // verify that the test is being run by root
+    Connector c = getConnector();
+    verifyHasOnlyTheseSystemPermissions(c, c.whoami(), SystemPermission.values());
+
+    // create the test user
+    String principal = testUser.getPrincipal();
+    AuthenticationToken token = testUser.getToken();
+    PasswordToken passwordToken = null;
+    if (token instanceof PasswordToken) {
+      passwordToken = (PasswordToken) token;
+    }
+    loginAs(rootUser);
+    c.securityOperations().createLocalUser(principal, passwordToken);
+    loginAs(testUser);
+    Connector test_user_conn = c.getInstance().getConnector(principal, token);
+    loginAs(rootUser);
+    verifyHasNoSystemPermissions(c, principal, SystemPermission.values());
+
+    // test each permission
+    for (SystemPermission perm : SystemPermission.values()) {
+      log.debug("Verifying the " + perm + " permission");
+
+      // test permission before and after granting it
+      String tableNamePrefix = getUniqueNames(1)[0];
+      testMissingSystemPermission(tableNamePrefix, c, rootUser, test_user_conn, testUser, perm);
+      loginAs(rootUser);
+      c.securityOperations().grantSystemPermission(principal, perm);
+      verifyHasOnlyTheseSystemPermissions(c, principal, perm);
+      testGrantedSystemPermission(tableNamePrefix, c, rootUser, test_user_conn, testUser, perm);
+      loginAs(rootUser);
+      c.securityOperations().revokeSystemPermission(principal, perm);
+      verifyHasNoSystemPermissions(c, principal, perm);
+    }
+  }
+
+  static Map<String,String> map(Iterable<Entry<String,String>> i) {
+    Map<String,String> result = new HashMap<String,String>();
+    for (Entry<String,String> e : i) {
+      result.put(e.getKey(), e.getValue());
+    }
+    return result;
+  }
+
+  private void testMissingSystemPermission(String tableNamePrefix, Connector root_conn, ClusterUser rootUser, Connector test_user_conn, ClusterUser testUser,
+      SystemPermission perm) throws Exception {
+    String tableName, user, password = "password", namespace;
+    boolean passwordBased = testUser.getPassword() != null;
+    log.debug("Confirming that the lack of the " + perm + " permission properly restricts the user");
+
+    // test permission prior to granting it
+    switch (perm) {
+      case CREATE_TABLE:
+        tableName = tableNamePrefix + "__CREATE_TABLE_WITHOUT_PERM_TEST__";
+        try {
+          loginAs(testUser);
+          test_user_conn.tableOperations().create(tableName);
+          throw new IllegalStateException("Should NOT be able to create a table");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.tableOperations().list().contains(tableName))
+            throw e;
+        }
+        break;
+      case DROP_TABLE:
+        tableName = tableNamePrefix + "__DROP_TABLE_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.tableOperations().create(tableName);
+        try {
+          loginAs(testUser);
+          test_user_conn.tableOperations().delete(tableName);
+          throw new IllegalStateException("Should NOT be able to delete a table");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableOperations().list().contains(tableName))
+            throw e;
+        }
+        break;
+      case ALTER_TABLE:
+        tableName = tableNamePrefix + "__ALTER_TABLE_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.tableOperations().create(tableName);
+        try {
+          loginAs(testUser);
+          test_user_conn.tableOperations().setProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+          throw new IllegalStateException("Should NOT be able to set a table property");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || map(root_conn.tableOperations().getProperties(tableName)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        loginAs(rootUser);
+        root_conn.tableOperations().setProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        try {
+          loginAs(testUser);
+          test_user_conn.tableOperations().removeProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey());
+          throw new IllegalStateException("Should NOT be able to remove a table property");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || !map(root_conn.tableOperations().getProperties(tableName)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        String table2 = tableName + "2";
+        try {
+          loginAs(testUser);
+          test_user_conn.tableOperations().rename(tableName, table2);
+          throw new IllegalStateException("Should NOT be able to rename a table");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableOperations().list().contains(tableName)
+              || root_conn.tableOperations().list().contains(table2))
+            throw e;
+        }
+        break;
+      case CREATE_USER:
+        user = "__CREATE_USER_WITHOUT_PERM_TEST__";
+        try {
+          loginAs(testUser);
+          test_user_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+          throw new IllegalStateException("Should NOT be able to create a user");
+        } catch (AccumuloSecurityException e) {
+          AuthenticationToken userToken = testUser.getToken();
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || (userToken instanceof PasswordToken && root_conn.securityOperations().authenticateUser(user, userToken)))
+            throw e;
+        }
+        break;
+      case DROP_USER:
+        user = "__DROP_USER_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+        try {
+          loginAs(testUser);
+          test_user_conn.securityOperations().dropLocalUser(user);
+          throw new IllegalStateException("Should NOT be able to delete a user");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.securityOperations().listLocalUsers().contains(user)) {
+            log.info("Failed to authenticate as " + user);
+            throw e;
+          }
+        }
+        break;
+      case ALTER_USER:
+        user = "__ALTER_USER_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+        try {
+          loginAs(testUser);
+          test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
+          throw new IllegalStateException("Should NOT be able to alter a user");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.securityOperations().getUserAuthorizations(user).isEmpty())
+            throw e;
+        }
+        break;
+      case SYSTEM:
+        // test for system permission would go here
+        break;
+      case CREATE_NAMESPACE:
+        namespace = "__CREATE_NAMESPACE_WITHOUT_PERM_TEST__";
+        try {
+          loginAs(testUser);
+          test_user_conn.namespaceOperations().create(namespace);
+          throw new IllegalStateException("Should NOT be able to create a namespace");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.namespaceOperations().list().contains(namespace))
+            throw e;
+        }
+        break;
+      case DROP_NAMESPACE:
+        namespace = "__DROP_NAMESPACE_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.namespaceOperations().create(namespace);
+        try {
+          loginAs(testUser);
+          test_user_conn.namespaceOperations().delete(namespace);
+          throw new IllegalStateException("Should NOT be able to delete a namespace");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.namespaceOperations().list().contains(namespace))
+            throw e;
+        }
+        break;
+      case ALTER_NAMESPACE:
+        namespace = "__ALTER_NAMESPACE_WITHOUT_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.namespaceOperations().create(namespace);
+        try {
+          loginAs(testUser);
+          test_user_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+          throw new IllegalStateException("Should NOT be able to set a namespace property");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || map(root_conn.namespaceOperations().getProperties(namespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        loginAs(rootUser);
+        root_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        try {
+          loginAs(testUser);
+          test_user_conn.namespaceOperations().removeProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+          throw new IllegalStateException("Should NOT be able to remove a namespace property");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || !map(root_conn.namespaceOperations().getProperties(namespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        String namespace2 = namespace + "2";
+        try {
+          loginAs(testUser);
+          test_user_conn.namespaceOperations().rename(namespace, namespace2);
+          throw new IllegalStateException("Should NOT be able to rename a namespace");
+        } catch (AccumuloSecurityException e) {
+          loginAs(rootUser);
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.namespaceOperations().list().contains(namespace)
+              || root_conn.namespaceOperations().list().contains(namespace2))
+            throw e;
+        }
+        break;
+      case OBTAIN_DELEGATION_TOKEN:
+        ClientConfiguration clientConf = cluster.getClientConfig();
+        if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
+          // TODO Try to obtain a delegation token without the permission
+        }
+        break;
+      case GRANT:
+        loginAs(testUser);
+        try {
+          test_user_conn.securityOperations().grantSystemPermission(testUser.getPrincipal(), SystemPermission.GRANT);
+          throw new IllegalStateException("Should NOT be able to grant System.GRANT to yourself");
+        } catch (AccumuloSecurityException e) {
+          // Expected
+          loginAs(rootUser);
+          assertFalse(root_conn.securityOperations().hasSystemPermission(testUser.getPrincipal(), SystemPermission.GRANT));
+        }
+        break;
+      default:
+        throw new IllegalArgumentException("Unrecognized System Permission: " + perm);
+    }
+  }
+
+  private void testGrantedSystemPermission(String tableNamePrefix, Connector root_conn, ClusterUser rootUser, Connector test_user_conn, ClusterUser testUser,
+      SystemPermission perm) throws Exception {
+    String tableName, user, password = "password", namespace;
+    boolean passwordBased = testUser.getPassword() != null;
+    log.debug("Confirming that the presence of the " + perm + " permission properly permits the user");
+
+    // test permission after granting it
+    switch (perm) {
+      case CREATE_TABLE:
+        tableName = tableNamePrefix + "__CREATE_TABLE_WITH_PERM_TEST__";
+        loginAs(testUser);
+        test_user_conn.tableOperations().create(tableName);
+        loginAs(rootUser);
+        if (!root_conn.tableOperations().list().contains(tableName))
+          throw new IllegalStateException("Should be able to create a table");
+        break;
+      case DROP_TABLE:
+        tableName = tableNamePrefix + "__DROP_TABLE_WITH_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.tableOperations().create(tableName);
+        loginAs(testUser);
+        test_user_conn.tableOperations().delete(tableName);
+        loginAs(rootUser);
+        if (root_conn.tableOperations().list().contains(tableName))
+          throw new IllegalStateException("Should be able to delete a table");
+        break;
+      case ALTER_TABLE:
+        tableName = tableNamePrefix + "__ALTER_TABLE_WITH_PERM_TEST__";
+        String table2 = tableName + "2";
+        loginAs(rootUser);
+        root_conn.tableOperations().create(tableName);
+        loginAs(testUser);
+        test_user_conn.tableOperations().setProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        loginAs(rootUser);
+        Map<String,String> properties = map(root_conn.tableOperations().getProperties(tableName));
+        if (!properties.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to set a table property");
+        loginAs(testUser);
+        test_user_conn.tableOperations().removeProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey());
+        loginAs(rootUser);
+        properties = map(root_conn.tableOperations().getProperties(tableName));
+        if (properties.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to remove a table property");
+        loginAs(testUser);
+        test_user_conn.tableOperations().rename(tableName, table2);
+        loginAs(rootUser);
+        if (root_conn.tableOperations().list().contains(tableName) || !root_conn.tableOperations().list().contains(table2))
+          throw new IllegalStateException("Should be able to rename a table");
+        break;
+      case CREATE_USER:
+        user = "__CREATE_USER_WITH_PERM_TEST__";
+        loginAs(testUser);
+        test_user_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+        loginAs(rootUser);
+        if (passwordBased && !root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
+          throw new IllegalStateException("Should be able to create a user");
+        break;
+      case DROP_USER:
+        user = "__DROP_USER_WITH_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+        loginAs(testUser);
+        test_user_conn.securityOperations().dropLocalUser(user);
+        loginAs(rootUser);
+        if (passwordBased && root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
+          throw new IllegalStateException("Should be able to delete a user");
+        break;
+      case ALTER_USER:
+        user = "__ALTER_USER_WITH_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.securityOperations().createLocalUser(user, (passwordBased ? new PasswordToken(password) : null));
+        loginAs(testUser);
+        test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
+        loginAs(rootUser);
+        if (root_conn.securityOperations().getUserAuthorizations(user).isEmpty())
+          throw new IllegalStateException("Should be able to alter a user");
+        break;
+      case SYSTEM:
+        // test for system permission would go here
+        break;
+      case CREATE_NAMESPACE:
+        namespace = "__CREATE_NAMESPACE_WITH_PERM_TEST__";
+        loginAs(testUser);
+        test_user_conn.namespaceOperations().create(namespace);
+        loginAs(rootUser);
+        if (!root_conn.namespaceOperations().list().contains(namespace))
+          throw new IllegalStateException("Should be able to create a namespace");
+        break;
+      case DROP_NAMESPACE:
+        namespace = "__DROP_NAMESPACE_WITH_PERM_TEST__";
+        loginAs(rootUser);
+        root_conn.namespaceOperations().create(namespace);
+        loginAs(testUser);
+        test_user_conn.namespaceOperations().delete(namespace);
+        loginAs(rootUser);
+        if (root_conn.namespaceOperations().list().contains(namespace))
+          throw new IllegalStateException("Should be able to delete a namespace");
+        break;
+      case ALTER_NAMESPACE:
+        namespace = "__ALTER_NAMESPACE_WITH_PERM_TEST__";
+        String namespace2 = namespace + "2";
+        loginAs(rootUser);
+        root_conn.namespaceOperations().create(namespace);
+        loginAs(testUser);
+        test_user_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        loginAs(rootUser);
+        Map<String,String> propies = map(root_conn.namespaceOperations().getProperties(namespace));
+        if (!propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to set a table property");
+        loginAs(testUser);
+        test_user_conn.namespaceOperations().removeProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+        loginAs(rootUser);
+        propies = map(root_conn.namespaceOperations().getProperties(namespace));
+        if (propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to remove a table property");
+        loginAs(testUser);
+        test_user_conn.namespaceOperations().rename(namespace, namespace2);
+        loginAs(rootUser);
+        if (root_conn.namespaceOperations().list().contains(namespace) || !root_conn.namespaceOperations().list().contains(namespace2))
+          throw new IllegalStateException("Should be able to rename a table");
+        break;
+      case OBTAIN_DELEGATION_TOKEN:
+        ClientConfiguration clientConf = cluster.getClientConfig();
+        if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
+          // TODO Try to obtain a delegation token with the permission
+        }
+        break;
+      case GRANT:
+        loginAs(rootUser);
+        root_conn.securityOperations().grantSystemPermission(testUser.getPrincipal(), SystemPermission.GRANT);
+        loginAs(testUser);
+        test_user_conn.securityOperations().grantSystemPermission(testUser.getPrincipal(), SystemPermission.CREATE_TABLE);
+        loginAs(rootUser);
+        assertTrue("Test user should have CREATE_TABLE",
+            root_conn.securityOperations().hasSystemPermission(testUser.getPrincipal(), SystemPermission.CREATE_TABLE));
+        assertTrue("Test user should have GRANT", root_conn.securityOperations().hasSystemPermission(testUser.getPrincipal(), SystemPermission.GRANT));
+        root_conn.securityOperations().revokeSystemPermission(testUser.getPrincipal(), SystemPermission.CREATE_TABLE);
+        break;
+      default:
+        throw new IllegalArgumentException("Unrecognized System Permission: " + perm);
+    }
+  }
+
+  private void verifyHasOnlyTheseSystemPermissions(Connector root_conn, String user, SystemPermission... perms) throws AccumuloException,
+      AccumuloSecurityException {
+    List<SystemPermission> permList = Arrays.asList(perms);
+    for (SystemPermission p : SystemPermission.values()) {
+      if (permList.contains(p)) {
+        // should have these
+        if (!root_conn.securityOperations().hasSystemPermission(user, p))
+          throw new IllegalStateException(user + " SHOULD have system permission " + p);
+      } else {
+        // should not have these
+        if (root_conn.securityOperations().hasSystemPermission(user, p))
+          throw new IllegalStateException(user + " SHOULD NOT have system permission " + p);
+      }
+    }
+  }
+
+  private void verifyHasNoSystemPermissions(Connector root_conn, String user, SystemPermission... perms) throws AccumuloException, AccumuloSecurityException {
+    for (SystemPermission p : perms)
+      if (root_conn.securityOperations().hasSystemPermission(user, p))
+        throw new IllegalStateException(user + " SHOULD NOT have system permission " + p);
+  }
+
+  @Test
+  public void tablePermissionTest() throws Exception {
+    // create the test user
+    ClusterUser testUser = getUser(0), rootUser = getAdminUser();
+
+    String principal = testUser.getPrincipal();
+    AuthenticationToken token = testUser.getToken();
+    PasswordToken passwordToken = null;
+    if (token instanceof PasswordToken) {
+      passwordToken = (PasswordToken) token;
+    }
+    loginAs(rootUser);
+    Connector c = getConnector();
+    c.securityOperations().createLocalUser(principal, passwordToken);
+    loginAs(testUser);
+    Connector test_user_conn = c.getInstance().getConnector(principal, token);
+
+    // check for read-only access to metadata table
+    loginAs(rootUser);
+    verifyHasOnlyTheseTablePermissions(c, c.whoami(), MetadataTable.NAME, TablePermission.READ, TablePermission.ALTER_TABLE);
+    verifyHasOnlyTheseTablePermissions(c, principal, MetadataTable.NAME, TablePermission.READ);
+    String tableName = getUniqueNames(1)[0] + "__TABLE_PERMISSION_TEST__";
+
+    // test each permission
+    for (TablePermission perm : TablePermission.values()) {
+      log.debug("Verifying the " + perm + " permission");
+
+      // test permission before and after granting it
+      createTestTable(c, principal, tableName);
+      loginAs(testUser);
+      testMissingTablePermission(test_user_conn, testUser, perm, tableName);
+      loginAs(rootUser);
+      c.securityOperations().grantTablePermission(principal, tableName, perm);
+      verifyHasOnlyTheseTablePermissions(c, principal, tableName, perm);
+      loginAs(testUser);
+      testGrantedTablePermission(test_user_conn, testUser, perm, tableName);
+
+      loginAs(rootUser);
+      createTestTable(c, principal, tableName);
+      c.securityOperations().revokeTablePermission(principal, tableName, perm);
+      verifyHasNoTablePermissions(c, principal, tableName, perm);
+    }
+  }
+
+  private void createTestTable(Connector c, String testUser, String tableName) throws Exception, MutationsRejectedException {
+    if (!c.tableOperations().exists(tableName)) {
+      // create the test table
+      c.tableOperations().create(tableName);
+      // put in some initial data
+      BatchWriter writer = c.createBatchWriter(tableName, new BatchWriterConfig());
+      Mutation m = new Mutation(new Text("row"));
+      m.put(new Text("cf"), new Text("cq"), new Value("val".getBytes()));
+      writer.addMutation(m);
+      writer.close();
+
+      // verify proper permissions for creator and test user
+      verifyHasOnlyTheseTablePermissions(c, c.whoami(), tableName, TablePermission.values());
+      verifyHasNoTablePermissions(c, testUser, tableName, TablePermission.values());
+
+    }
+  }
+
+  private void testMissingTablePermission(Connector test_user_conn, ClusterUser testUser, TablePermission perm, String tableName) throws Exception {
+    Scanner scanner;
+    BatchWriter writer;
+    Mutation m;
+    log.debug("Confirming that the lack of the " + perm + " permission properly restricts the user");
+
+    // test permission prior to granting it
+    switch (perm) {
+      case READ:
+        try {
+          scanner = test_user_conn.createScanner(tableName, Authorizations.EMPTY);
+          int i = 0;
+          for (Entry<Key,Value> entry : scanner)
+            i += 1 + entry.getKey().getRowData().length();
+          if (i != 0)
+            throw new IllegalStateException("Should NOT be able to read from the table");
+        } catch (RuntimeException e) {
+          AccumuloSecurityException se = (AccumuloSecurityException) e.getCause();
+          if (se.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
+            throw se;
+        }
+        break;
+      case WRITE:
+        try {
+          writer = test_user_conn.createBatchWriter(tableName, new BatchWriterConfig());
+          m = new Mutation(new Text("row"));
+          m.put(new Text("a"), new Text("b"), new Value("c".getBytes()));
+          writer.addMutation(m);
+          try {
+            writer.close();
+          } catch (MutationsRejectedException e1) {
+            if (e1.getSecurityErrorCodes().size() > 0)
+              throw new AccumuloSecurityException(test_user_conn.whoami(), org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode.PERMISSION_DENIED, e1);
+          }
+          throw new IllegalStateException("Should NOT be able to write to a table");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
+            throw e;
+        }
+        break;
+      case BULK_IMPORT:
+        // test for bulk import permission would go here
+        break;
+      case ALTER_TABLE:
+        Map<String,Set<Text>> groups = new HashMap<String,Set<Text>>();
+        groups.put("tgroup", new HashSet<Text>(Arrays.asList(new Text("t1"), new Text("t2"))));
+        try {
+          test_user_conn.tableOperations().setLocalityGroups(tableName, groups);
+          throw new IllegalStateException("User should not be able to set locality groups");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
+            throw e;
+        }
+        break;
+      case DROP_TABLE:
+        try {
+          test_user_conn.tableOperations().delete(tableName);
+          throw new IllegalStateException("User should not be able delete the table");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
+            throw e;
+        }
+        break;
+      case GRANT:
+        try {
+          test_user_conn.securityOperations().grantTablePermission(getAdminPrincipal(), tableName, TablePermission.GRANT);
+          throw new IllegalStateException("User should not be able grant permissions");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED)
+            throw e;
+        }
+        break;
+      default:
+        throw new IllegalArgumentException("Unrecognized table Permission: " + perm);
+    }
+  }
+
+  private void testGrantedTablePermission(Connector test_user_conn, ClusterUser normalUser, TablePermission perm, String tableName) throws AccumuloException,
+      TableExistsException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException {
+    Scanner scanner;
+    BatchWriter writer;
+    Mutation m;
+    log.debug("Confirming that the presence of the " + perm + " permission properly permits the user");
+
+    // test permission after granting it
+    switch (perm) {
+      case READ:
+        scanner = test_user_conn.createScanner(tableName, Authorizations.EMPTY);
+        Iterator<Entry<Key,Value>> iter = scanner.iterator();
+        while (iter.hasNext())
+          iter.next();
+        break;
+      case WRITE:
+        writer = test_user_conn.createBatchWriter(tableName, new BatchWriterConfig());
+        m = new Mutation(new Text("row"));
+        m.put(new Text("a"), new Text("b"), new Value("c".getBytes()));
+        writer.addMutation(m);
+        writer.close();
+        break;
+      case BULK_IMPORT:
+        // test for bulk import permission would go here
+        break;
+      case ALTER_TABLE:
+        Map<String,Set<Text>> groups = new HashMap<String,Set<Text>>();
+        groups.put("tgroup", new HashSet<Text>(Arrays.asList(new Text("t1"), new Text("t2"))));
+        break;
+      case DROP_TABLE:
+        test_user_conn.tableOperations().delete(tableName);
+        break;
+      case GRANT:
+        test_user_conn.securityOperations().grantTablePermission(getAdminPrincipal(), tableName, TablePermission.GRANT);
+        break;
+      default:
+        throw new IllegalArgumentException("Unrecognized table Permission: " + perm);
+    }
+  }
+
+  private void verifyHasOnlyTheseTablePermissions(Connector root_conn, String user, String table, TablePermission... perms) throws AccumuloException,
+      AccumuloSecurityException {
+    List<TablePermission> permList = Arrays.asList(perms);
+    for (TablePermission p : TablePermission.values()) {
+      if (permList.contains(p)) {
+        // should have these
+        if (!root_conn.securityOperations().hasTablePermission(user, table, p))
+          throw new IllegalStateException(user + " SHOULD have table permission " + p + " for table " + table);
+      } else {
+        // should not have these
+        if (root_conn.securityOperations().hasTablePermission(user, table, p))
+          throw new IllegalStateException(user + " SHOULD NOT have table permission " + p + " for table " + table);
+      }
+    }
+  }
+
+  private void verifyHasNoTablePermissions(Connector root_conn, String user, String table, TablePermission... perms) throws AccumuloException,
+      AccumuloSecurityException {
+    for (TablePermission p : perms)
+      if (root_conn.securityOperations().hasTablePermission(user, table, p))
+        throw new IllegalStateException(user + " SHOULD NOT have table permission " + p + " for table " + table);
+  }
+}