You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/07/31 17:13:51 UTC

[14/50] [abbrv] Reversing ACCUMULO-1438.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e46360cf/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java b/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
new file mode 100644
index 0000000..2cb3ae2
--- /dev/null
+++ b/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
@@ -0,0 +1,364 @@
+/*
+ * 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.server.mini;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.TimerTask;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.logger.LogService;
+import org.apache.accumulo.server.master.Master;
+import org.apache.accumulo.server.tabletserver.TabletServer;
+import org.apache.accumulo.server.util.Initialize;
+import org.apache.accumulo.server.util.PortUtils;
+import org.apache.accumulo.server.util.time.SimpleTimer;
+import org.apache.accumulo.start.Main;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+
+/**
+ * A utility class that will create Zookeeper and Accumulo processes that write all of their data to a single local directory. This class makes it easy to test
+ * code against a real Accumulo instance. Its much more accurate for testing than MockAccumulo, but much slower than MockAccumulo.
+ * 
+ * @since 1.5.0, 1.4.4
+ */
+public class MiniAccumuloCluster {
+  
+  private static final String INSTANCE_SECRET = "DONTTELL";
+  private static final String INSTANCE_NAME = "miniInstance";
+  
+  private static class LogWriter extends Thread {
+    private BufferedReader in;
+    private BufferedWriter out;
+    
+    /**
+     * @throws java.io.IOException
+     */
+    public LogWriter(InputStream stream, File logFile) throws IOException {
+      this.setDaemon(true);
+      this.in = new BufferedReader(new InputStreamReader(stream));
+      out = new BufferedWriter(new FileWriter(logFile));
+      
+      SimpleTimer.getInstance().schedule(new TimerTask() {
+        @Override
+        public void run() {
+          try {
+            flush();
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+        }
+      }, 1000, 1000);
+    }
+    
+    public synchronized void flush() throws IOException {
+      if (out != null)
+        out.flush();
+    }
+    
+    @Override
+    public void run() {
+      String line;
+      
+      try {
+        while ((line = in.readLine()) != null) {
+          out.append(line);
+          out.append("\n");
+        }
+        
+        synchronized (this) {
+          out.close();
+          out = null;
+          in.close();
+        }
+        
+      } catch (IOException e) {
+      }
+    }
+  }
+  
+  private File libDir;
+  private File confDir;
+  private File zooKeeperDir;
+  private File accumuloDir;
+  private File zooCfgFile;
+  private File logDir;
+  private File walogDir;
+  
+  private Process zooKeeperProcess;
+  private Process masterProcess;
+  private Process loggerProcess;
+  
+  private int zooKeeperPort;
+  
+  private List<LogWriter> logWriters = new ArrayList<LogWriter>();
+  
+  private MiniAccumuloConfig config;
+  private Process[] tabletServerProcesses;
+  
+  Process exec(Class<? extends Object> clazz, String... args) throws IOException {
+    String javaHome = System.getProperty("java.home");
+    String javaBin = javaHome + File.separator + "bin" + File.separator + "java";
+    String classpath = System.getProperty("java.class.path");
+    
+    classpath = confDir.getAbsolutePath() + File.pathSeparator + classpath;
+    
+    String className = clazz.getCanonicalName();
+    
+    ArrayList<String> argList = new ArrayList<String>();
+    
+    argList.addAll(Arrays.asList(javaBin, "-cp", classpath, "-Xmx128m", "-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75",
+        Main.class.getName(), className));
+    
+    argList.addAll(Arrays.asList(args));
+    
+    ProcessBuilder builder = new ProcessBuilder(argList);
+    
+    builder.environment().put("ACCUMULO_HOME", config.getDir().getAbsolutePath());
+    builder.environment().put("ACCUMULO_LOG_DIR", logDir.getAbsolutePath());
+    
+    // if we're running under accumulo.start, we forward these env vars
+    String env = System.getenv("HADOOP_PREFIX");
+    if (env != null)
+      builder.environment().put("HADOOP_PREFIX", env);
+    env = System.getenv("ZOOKEEPER_HOME");
+    if (env != null)
+      builder.environment().put("ZOOKEEPER_HOME", env);
+    
+    Process process = builder.start();
+    
+    LogWriter lw;
+    lw = new LogWriter(process.getErrorStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".err"));
+    logWriters.add(lw);
+    lw.start();
+    lw = new LogWriter(process.getInputStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".out"));
+    logWriters.add(lw);
+    lw.start();
+    
+    return process;
+  }
+  
+  private void appendProp(FileWriter fileWriter, Property key, String value, Map<String,String> siteConfig) throws IOException {
+    appendProp(fileWriter, key.getKey(), value, siteConfig);
+  }
+  
+  private void appendProp(FileWriter fileWriter, String key, String value, Map<String,String> siteConfig) throws IOException {
+    if (!siteConfig.containsKey(key))
+      fileWriter.append("<property><name>" + key + "</name><value>" + value + "</value></property>\n");
+  }
+  
+  /**
+   * @param dir
+   *          An empty or nonexistant temp directoy that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava,
+   *          and Junit provide methods for creating temporary directories.
+   * @param rootPassword
+   *          Initial root password for instance.
+   * @throws IOException
+   */
+  public MiniAccumuloCluster(File dir, String rootPassword) throws IOException {
+    this(new MiniAccumuloConfig(dir, rootPassword));
+  }
+  
+  /**
+   * @param config
+   *          initial configuration
+   * @throws IOException
+   */
+  
+  public MiniAccumuloCluster(MiniAccumuloConfig config) throws IOException {
+    
+    if (config.getDir().exists() && !config.getDir().isDirectory())
+      throw new IllegalArgumentException("Must pass in directory, " + config.getDir() + " is a file");
+    
+    if (config.getDir().exists() && config.getDir().list().length != 0)
+      throw new IllegalArgumentException("Directory " + config.getDir() + " is not empty");
+    
+    this.config = config;
+    
+    libDir = new File(config.getDir(), "lib");
+    confDir = new File(config.getDir(), "conf");
+    accumuloDir = new File(config.getDir(), "accumulo");
+    zooKeeperDir = new File(config.getDir(), "zookeeper");
+    logDir = new File(config.getDir(), "logs");
+    walogDir = new File(config.getDir(), "walogs");
+    
+    confDir.mkdirs();
+    accumuloDir.mkdirs();
+    zooKeeperDir.mkdirs();
+    logDir.mkdirs();
+    walogDir.mkdirs();
+    libDir.mkdirs();
+    
+    zooKeeperPort = PortUtils.getRandomFreePort();
+    
+    File siteFile = new File(confDir, "accumulo-site.xml");
+    
+    FileWriter fileWriter = new FileWriter(siteFile);
+    fileWriter.append("<configuration>\n");
+    
+    HashMap<String,String> siteConfig = new HashMap<String,String>(config.getSiteConfig());
+    
+    appendProp(fileWriter, Property.INSTANCE_DFS_URI, "file:///", siteConfig);
+    appendProp(fileWriter, Property.INSTANCE_DFS_DIR, accumuloDir.getAbsolutePath(), siteConfig);
+    appendProp(fileWriter, Property.INSTANCE_ZK_HOST, "localhost:" + zooKeeperPort, siteConfig);
+    appendProp(fileWriter, Property.INSTANCE_SECRET, INSTANCE_SECRET, siteConfig);
+    appendProp(fileWriter, Property.MASTER_CLIENTPORT, "" + PortUtils.getRandomFreePort(), siteConfig);
+    appendProp(fileWriter, Property.TSERV_CLIENTPORT, "" + PortUtils.getRandomFreePort(), siteConfig);
+    appendProp(fileWriter, Property.TSERV_PORTSEARCH, "true", siteConfig);
+    appendProp(fileWriter, Property.LOGGER_DIR, walogDir.getAbsolutePath(), siteConfig);
+    appendProp(fileWriter, Property.TSERV_DATACACHE_SIZE, "10M", siteConfig);
+    appendProp(fileWriter, Property.TSERV_INDEXCACHE_SIZE, "10M", siteConfig);
+    appendProp(fileWriter, Property.TSERV_MAXMEM, "50M", siteConfig);
+    appendProp(fileWriter, Property.TSERV_WALOG_MAX_SIZE, "100M", siteConfig);
+    appendProp(fileWriter, Property.TSERV_NATIVEMAP_ENABLED, "false", siteConfig);
+    appendProp(fileWriter, Property.TRACE_PORT, "" + PortUtils.getRandomFreePort(), siteConfig);
+    appendProp(fileWriter, Property.LOGGER_SORT_BUFFER_SIZE, "50M", siteConfig);
+    appendProp(fileWriter, Property.LOGGER_PORTSEARCH, "true", siteConfig);
+    
+    // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5
+    appendProp(fileWriter, Property.TSERV_MAJC_DELAY, "3", siteConfig);
+    String cp = System.getenv("ACCUMULO_HOME") + "/lib/.*.jar," + "$ZOOKEEPER_HOME/zookeeper[^.].*.jar," + "$HADOOP_HOME/[^.].*.jar,"
+        + "$HADOOP_HOME/lib/[^.].*.jar," + "$HADOOP_PREFIX/share/hadoop/common/.*.jar," + "$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar,"
+        + "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar," + "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar";
+    appendProp(fileWriter, Property.GENERAL_CLASSPATHS, cp, siteConfig);
+    appendProp(fileWriter, Property.GENERAL_DYNAMIC_CLASSPATHS, libDir.getAbsolutePath(), siteConfig);
+    
+    for (Entry<String,String> entry : siteConfig.entrySet())
+      fileWriter.append("<property><name>" + entry.getKey() + "</name><value>" + entry.getValue() + "</value></property>\n");
+    fileWriter.append("</configuration>\n");
+    fileWriter.close();
+    
+    zooCfgFile = new File(confDir, "zoo.cfg");
+    fileWriter = new FileWriter(zooCfgFile);
+    
+    // zookeeper uses Properties to read its config, so use that to write in order to properly escape things like Windows paths
+    Properties zooCfg = new Properties();
+    zooCfg.setProperty("tickTime", "1000");
+    zooCfg.setProperty("initLimit", "10");
+    zooCfg.setProperty("syncLimit", "5");
+    zooCfg.setProperty("clientPort", zooKeeperPort + "");
+    zooCfg.setProperty("maxClientCnxns", "100");
+    zooCfg.setProperty("dataDir", zooKeeperDir.getAbsolutePath());
+    zooCfg.store(fileWriter, null);
+    
+    fileWriter.close();
+  }
+  
+  /**
+   * Starts Accumulo and Zookeeper processes. Can only be called once.
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws IllegalStateException
+   *           if already started
+   */
+  public void start() throws IOException, InterruptedException {
+    if (zooKeeperProcess != null)
+      throw new IllegalStateException("Already started");
+    
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        try {
+          MiniAccumuloCluster.this.stop();
+        } catch (IOException e) {
+          e.printStackTrace();
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+    });
+    
+    zooKeeperProcess = exec(Main.class, ZooKeeperServerMain.class.getName(), zooCfgFile.getAbsolutePath());
+    
+    // sleep a little bit to let zookeeper come up before calling init, seems to work better
+    UtilWaitThread.sleep(250);
+    
+    // TODO initialization could probably be done in process
+    Process initProcess = exec(Initialize.class);
+    initProcess.getOutputStream().write((INSTANCE_NAME + "\n").getBytes());
+    initProcess.getOutputStream().write((config.getRootPassword() + "\n").getBytes());
+    initProcess.getOutputStream().write((config.getRootPassword() + "\n").getBytes());
+    initProcess.getOutputStream().flush();
+    int ret = initProcess.waitFor();
+    if (ret != 0) {
+      throw new RuntimeException("Initialize process returned " + ret);
+    }
+    
+    tabletServerProcesses = new Process[config.getNumTservers()];
+    for (int i = 0; i < config.getNumTservers(); i++) {
+      tabletServerProcesses[i] = exec(TabletServer.class);
+    }
+    
+    loggerProcess = exec(LogService.class);
+    masterProcess = exec(Master.class);
+  }
+  
+  /**
+   * @return Accumulo instance name
+   */
+  
+  public String getInstanceName() {
+    return INSTANCE_NAME;
+  }
+  
+  /**
+   * @return zookeeper connection string
+   */
+  
+  public String getZooKeepers() {
+    return "localhost:" + zooKeeperPort;
+  }
+  
+  /**
+   * Stops Accumulo and Zookeeper processes. If stop is not called, there is a shutdown hook that is setup to kill the processes. Howerver its probably best to
+   * call stop in a finally block as soon as possible.
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  
+  public void stop() throws IOException, InterruptedException {
+    if (zooKeeperProcess != null)
+      zooKeeperProcess.destroy();
+    if (loggerProcess != null)
+      loggerProcess.destroy();
+    if (masterProcess != null)
+      masterProcess.destroy();
+    if (tabletServerProcesses != null) {
+      for (Process tserver : tabletServerProcesses) {
+        tserver.destroy();
+      }
+    }
+    
+    for (LogWriter lw : logWriters)
+      lw.flush();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e46360cf/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java b/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
new file mode 100644
index 0000000..a82f277
--- /dev/null
+++ b/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.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.server.mini;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Holds configuration for {@link MiniAccumuloCluster}. Required configurations must be passed to constructor and all other configurations are optional.
+ * 
+ * @since 1.5.0, 1.4.4
+ */
+
+public class MiniAccumuloConfig {
+  
+  private File dir = null;
+  private String rootPassword = null;
+  private Map<String,String> siteConfig = Collections.emptyMap();
+  private int numTservers = 2;
+  
+  /**
+   * @param dir
+   *          An empty or nonexistant temp directoy that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava,
+   *          and Junit provide methods for creating temporary directories.
+   * @param rootPassword
+   *          The initial password for the Accumulo root user
+   */
+  
+  public MiniAccumuloConfig(File dir, String rootPassword) {
+    this.dir = dir;
+    this.rootPassword = rootPassword;
+  }
+  
+  public File getDir() {
+    return dir;
+  }
+  
+  public String getRootPassword() {
+    return rootPassword;
+  }
+  
+  public int getNumTservers() {
+    return numTservers;
+  }
+  
+  /**
+   * Calling this method is optional. If not set, it defaults to two.
+   * 
+   * @param numTservers
+   *          the number of tablet servers that mini accumulo cluster should start
+   */
+  
+  public MiniAccumuloConfig setNumTservers(int numTservers) {
+    if (numTservers < 1)
+      throw new IllegalArgumentException("Must have at least one tablet server");
+    this.numTservers = numTservers;
+    return this;
+  }
+  
+  public Map<String,String> getSiteConfig() {
+    return siteConfig;
+  }
+  
+  /**
+   * Calling this method is optional. If not set, it defautls to an empty map.
+   * 
+   * @param siteConfig
+   *          key/values that you normally put in accumulo-site.xml can be put here
+   */
+  public MiniAccumuloConfig setSiteConfig(Map<String,String> siteConfig) {
+    this.siteConfig = siteConfig;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e46360cf/src/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
----------------------------------------------------------------------
diff --git a/src/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java b/src/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
new file mode 100644
index 0000000..083d4b8
--- /dev/null
+++ b/src/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.server.mini;
+
+import java.util.Collections;
+import java.util.Map.Entry;
+import java.util.UUID;
+
+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.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+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.iterators.LongCombiner;
+import org.apache.accumulo.core.iterators.user.SummingCombiner;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class MiniAccumuloClusterTest {
+  
+  private static final Logger logger = Logger.getLogger(MiniAccumuloClusterTest.class);
+  
+  private static TemporaryFolder tmpDir = new TemporaryFolder();
+  private static MiniAccumuloCluster accumulo;
+  
+  @BeforeClass
+  public static void setupMiniCluster() throws Exception {
+    
+    tmpDir.create();
+    logger.info("MiniCluster started @ " + tmpDir.getRoot());
+    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
+    
+    accumulo = new MiniAccumuloCluster(tmpDir.getRoot(), "superSecret");
+    accumulo.start();
+  }
+  
+  @Test(timeout = 30000)
+  public void test() throws Exception {
+    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", "superSecret".getBytes());
+    
+    conn.tableOperations().create("table1");
+    
+    conn.securityOperations().createUser("user1", "pass1".getBytes(), new Authorizations("A", "B"));
+    conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.READ);
+    
+    IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
+    SummingCombiner.setEncodingType(is, LongCombiner.Type.STRING);
+    SummingCombiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("META", "COUNT")));
+    
+    conn.tableOperations().attachIterator("table1", is);
+    
+    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("user1", "pass1".getBytes());
+    
+    BatchWriter bw = uconn.createBatchWriter("table1", 10000, 1000000, 2);
+    
+    UUID uuid = UUID.randomUUID();
+    
+    Mutation m = new Mutation(uuid.toString());
+    m.put("META", "SIZE", new ColumnVisibility("A|B"), "8");
+    m.put("META", "CRC", new ColumnVisibility("A|B"), "456");
+    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
+    m.put("DATA", "IMG", new ColumnVisibility("A&B"), "ABCDEFGH");
+    
+    bw.addMutation(m);
+    bw.flush();
+    
+    m = new Mutation(uuid.toString());
+    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
+    m.put("META", "CRC", new ColumnVisibility("A|B"), "123");
+    bw.addMutation(m);
+    
+    bw.close();
+    
+    int count = 0;
+    Scanner scanner = uconn.createScanner("table1", new Authorizations("A"));
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnQualifierData().toString().equals("COUNT")) {
+        Assert.assertEquals("2", entry.getValue().toString());
+      } else if (entry.getKey().getColumnQualifierData().toString().equals("SIZE")) {
+        Assert.assertEquals("8", entry.getValue().toString());
+      } else if (entry.getKey().getColumnQualifierData().toString().equals("CRC")) {
+        Assert.assertEquals("123", entry.getValue().toString());
+      } else {
+        Assert.assertTrue(false);
+      }
+      count++;
+    }
+    
+    Assert.assertEquals(3, count);
+    
+    count = 0;
+    scanner = uconn.createScanner("table1", new Authorizations("A", "B"));
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnQualifierData().toString().equals("IMG")) {
+        Assert.assertEquals("ABCDEFGH", entry.getValue().toString());
+      }
+      count++;
+    }
+    
+    Assert.assertEquals(4, count);
+    
+    conn.tableOperations().delete("table1");
+  }
+  
+  @Test(timeout = 20000)
+  public void testMultipleTabletServersRunning() throws AccumuloException, AccumuloSecurityException {
+    
+    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", "superSecret".getBytes());
+    
+    while (conn.instanceOperations().getTabletServers().size() != 2) {
+      UtilWaitThread.sleep(500);
+    }
+  }
+  
+  @AfterClass
+  public static void tearDownMiniCluster() throws Exception {
+    accumulo.stop();
+    tmpDir.delete();
+  }
+}