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

[21/50] [abbrv] git commit: ACCUMULO-1438, ACCUMULO-1414 Move minicluster into its own package and add proxy to user manual

ACCUMULO-1438, ACCUMULO-1414 Move minicluster into its own package and add proxy to user manual


git-svn-id: https://svn.apache.org/repos/asf/accumulo/branches/1.4@1484935 13f79535-47bb-0310-9956-ffa450edef68


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

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 49201bd2aecbc7fda0adb6a2e5c3f29b62366c44
Parents: 7d63bdc
Author: Corey J. Nolet <cj...@apache.org>
Authored: Tue May 21 20:17:35 2013 +0000
Committer: Corey J. Nolet <cj...@apache.org>
Committed: Tue May 21 20:17:35 2013 +0000

----------------------------------------------------------------------
 docs/src/user_manual/chapters/clients.tex       | 130 +++++++
 pom.xml                                         |   6 +
 src/minicluster/pom.xml                         | 109 ++++++
 .../minicluster/MiniAccumuloCluster.java        | 364 +++++++++++++++++++
 .../minicluster/MiniAccumuloConfig.java         |  89 +++++
 .../minicluster/MiniAccumuloClusterTest.java    | 149 ++++++++
 src/proxy/pom.xml                               |   5 +
 .../java/org/apache/accumulo/proxy/Proxy.java   |   2 +-
 .../org/apache/accumulo/proxy/SimpleTest.java   |   8 +-
 .../server/mini/MiniAccumuloCluster.java        | 364 -------------------
 .../server/mini/MiniAccumuloConfig.java         |  89 -----
 .../server/mini/MiniAccumuloClusterTest.java    | 150 --------
 12 files changed, 857 insertions(+), 608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/docs/src/user_manual/chapters/clients.tex
----------------------------------------------------------------------
diff --git a/docs/src/user_manual/chapters/clients.tex b/docs/src/user_manual/chapters/clients.tex
index 86336d6..12e8b69 100644
--- a/docs/src/user_manual/chapters/clients.tex
+++ b/docs/src/user_manual/chapters/clients.tex
@@ -202,3 +202,133 @@ for(Entry<Key,Value> entry : scan)
 An example of the BatchScanner can be found at\\
 accumulo/docs/examples/README.batch
 
+
+\section{Proxy}
+
+The proxy API allows the interaction with Accumulo with languages other than Java.
+A proxy server is provided in the codebase and a client can further be generated.
+
+\subsection{Prequisites}
+
+The proxy server can live on any node in which the basic client API would work. That
+means it must be able to communicate with the Master, ZooKeepers, NameNode, and the
+Data nodes. A proxy client only needs the ability to communicate with the proxy server.
+
+
+\subsection{Configuration}
+
+The configuration options for the proxy server live inside of a properties file. At
+the very least, you need to supply the following properties:
+
+\small
+\begin{verbatim}
+protocolFactory=org.apache.thrift.protocol.TCompactProtocol$Factory
+tokenClass=org.apache.accumulo.core.client.security.tokens.PasswordToken
+port=42424
+instance=test
+zookeepers=localhost:2181
+\end{verbatim}
+\normalsize
+
+You can find a sample configuration file in your distribution:
+
+\small
+\begin{verbatim}
+$ACCUMULO_HOME/proxy/proxy.properties.
+\end{verbatim}
+\normalsize
+
+This sample configuration file further demonstrates an abilty to back the proxy server
+by MockAccumulo or the MiniAccumuloCluster.
+
+\subsection{Running the Proxy Server}
+
+After the properties file holding the configuration is created, the proxy server
+can be started using the following command in the Accumulo distribution (assuming
+you your properties file is named config.properties):
+
+\small
+\begin{verbatim}
+$ACCUMULO_HOME/bin/accumulo proxy -p config.properties
+\end{verbatim}
+\normalsize
+
+\subsection{Creating a Proxy Client}
+
+Aside from installing the Thrift compiler, you will also need the language-specific library
+for Thrift installed to generate client code in that language. Typically, your operating
+system's package manager will be able to automatically install these for you in an expected
+location such as /usr/lib/python/site-packages/thrift.
+
+You can find the thrift file for generating the client:
+
+\small
+\begin{verbatim}
+$ACCUMULO_HOME/proxy/proxy.thrift.
+\end{verbatim}
+\normalsize
+
+After a client is generated, the port specified in the configuration properties above will be
+used to connect to the server.
+
+\subsection{Using a Proxy Client}
+
+The following examples have been written in Java and the method signatures may be
+slightly different depending on the language specified when generating client with
+the Thrift compiler. After initiating a connection to the Proxy (see Apache Thrift's
+documentation for examples of connecting to a Thrift service), the methods on the
+proxy client will be available. The first thing to do is log in:
+
+\small
+\begin{verbatim}
+Map password = new HashMap<String,String>();
+password.put("password", "secret");
+ByteBuffer token = client.login("root", password);
+\end{verbatim}
+\normalsize
+
+Once logged in, the token returned will be used for most subsequent calls to the client.
+Let's create a table, add some data, scan the table, and delete it.
+
+
+First, create a table.
+
+\small
+\begin{verbatim}
+client.createTable(token, "myTable", true, TimeType.MILLIS);
+\end{verbatim}
+\normalsize
+
+
+Next, add some data:
+
+\small
+\begin{verbatim}
+// first, create a writer on the server
+String writer = client.createWriter(token, "myTable", new WriterOptions());
+
+// build column updates
+Map<ByteBuffer, List<ColumnUpdate> cells> cellsToUpdate = //...
+
+// send updates to the server
+client.updateAndFlush(writer, "myTable", cellsToUpdate);
+
+client.closeWriter(writer);
+\end{verbatim}
+\normalsize
+
+
+Scan for the data and batch the return of the results on the server:
+
+\small
+\begin{verbatim}
+String scanner = client.createScanner(token, "myTable", new ScanOptions());
+ScanResult results = client.nextK(scanner, 100);
+
+for(KeyValue keyValue : results.getResultsIterator()) {
+  // do something with results
+}
+
+client.closeScanner(scanner);
+\end{verbatim}
+\normalsize
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5dcc0c2..d1877b1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -37,6 +37,7 @@
     <module>src/start</module>
     <module>src/examples</module>
     <module>src/proxy</module>
+    <module>src/minicluster</module>
   </modules>
 
   <build>
@@ -698,6 +699,11 @@
         <artifactId>accumulo-server</artifactId>
         <version>1.4.4-SNAPSHOT</version>
       </dependency>
+    <dependency>
+        <groupId>org.apache.accumulo</groupId>
+        <artifactId>accumulo-minicluster</artifactId>
+        <version>1.4.4-SNAPSHOT</version>
+    </dependency>
 
       <!-- additional dependencies we need to provide -->
       <dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/src/minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/src/minicluster/pom.xml b/src/minicluster/pom.xml
new file mode 100644
index 0000000..198d1f7
--- /dev/null
+++ b/src/minicluster/pom.xml
@@ -0,0 +1,109 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <parent>
+    <groupId>org.apache.accumulo</groupId>
+    <artifactId>accumulo</artifactId>
+    <version>1.4.4-SNAPSHOT</version>
+    <relativePath>../../</relativePath>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>accumulo-minicluster</artifactId>
+  <name>accumulo-minicluster</name>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <configuration>
+          <archive>
+            <manifestSections>
+              <manifestSection>
+                <name>accumulo/minicluster/</name>
+                <manifestEntries>
+                  <Sealed>true</Sealed>
+                </manifestEntries>
+              </manifestSection>
+            </manifestSections>
+          </archive>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-start</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>cloudtrace</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>jline</groupId>
+      <artifactId>jline</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
new file mode 100644
index 0000000..2335560
--- /dev/null
+++ b/src/minicluster/src/main/java/org/apache/accumulo/minicluster/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.minicluster;
+
+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/49201bd2/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/src/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
new file mode 100644
index 0000000..e0a9e94
--- /dev/null
+++ b/src/minicluster/src/main/java/org/apache/accumulo/minicluster/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.minicluster;
+
+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/49201bd2/src/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
----------------------------------------------------------------------
diff --git a/src/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java b/src/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
new file mode 100644
index 0000000..7f1e441
--- /dev/null
+++ b/src/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.minicluster;
+
+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.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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/src/proxy/pom.xml
----------------------------------------------------------------------
diff --git a/src/proxy/pom.xml b/src/proxy/pom.xml
index 5cfde7b..cc54f82 100644
--- a/src/proxy/pom.xml
+++ b/src/proxy/pom.xml
@@ -77,6 +77,11 @@
         <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-minicluster</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-core</artifactId>
         <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/src/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
----------------------------------------------------------------------
diff --git a/src/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java b/src/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
index 7b106dd..a65e5b6 100644
--- a/src/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
+++ b/src/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
@@ -23,7 +23,7 @@ import java.lang.reflect.Constructor;
 import java.util.Properties;
 
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TCompactProtocol;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
----------------------------------------------------------------------
diff --git a/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java b/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
index 8a898f4..dd1bb19 100644
--- a/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
+++ b/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.proxy.thrift.AccumuloException;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
 import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
@@ -74,7 +75,6 @@ import org.apache.accumulo.proxy.thrift.TimeType;
 import org.apache.accumulo.proxy.thrift.UnknownScanner;
 import org.apache.accumulo.proxy.thrift.UnknownWriter;
 import org.apache.accumulo.proxy.thrift.WriterOptions;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
 import org.apache.accumulo.server.test.functional.SlowIterator;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -133,7 +133,7 @@ public class SimpleTest {
     Properties props = new Properties();
     props.put("instance", accumulo.getInstanceName());
     props.put("zookeepers", accumulo.getZooKeepers());
-
+    
     protocolClass = getRandomProtocol();
     System.out.println(protocolClass.getName());
     
@@ -325,7 +325,7 @@ public class SimpleTest {
       client.hasNext(scanner);
       fail("exception not thrown");
     } catch (UnknownScanner us) {}
-
+    
     try {
       client.closeScanner(scanner);
       fail("exception not thrown");
@@ -365,7 +365,7 @@ public class SimpleTest {
     
     // this is a oneway call, so it does not throw exceptions
     client.update(writer, mutation("row2", "cf", "cq", "value2"));
-
+    
     try {
       client.flush(writer);
       fail("exception not thrown");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/49201bd2/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
deleted file mode 100644
index 2cb3ae2..0000000
--- a/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * 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/49201bd2/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
deleted file mode 100644
index a82f277..0000000
--- a/src/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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/49201bd2/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
deleted file mode 100644
index 083d4b8..0000000
--- a/src/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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();
-  }
-}