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/05/05 03:55:53 UTC

svn commit: r1479227 - in /accumulo/trunk: proxy/src/main/java/org/apache/accumulo/proxy/ proxy/src/test/java/org/apache/accumulo/proxy/ server/src/main/java/org/apache/accumulo/server/mini/ server/src/test/java/org/apache/accumulo/server/mini/ test/sr...

Author: elserj
Date: Sun May  5 01:55:52 2013
New Revision: 1479227

URL: http://svn.apache.org/r1479227
Log:
ACCUMULO-1367 Applying patch from Corey Nolet to make MAC a little more configurable.

Added:
    accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java   (with props)
Modified:
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
    accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
    accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java

Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java (original)
+++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java Sun May  5 01:55:52 2013
@@ -41,7 +41,7 @@ import com.google.common.io.Files;
 
 public class Proxy {
   
-  private static final Logger log = Logger.getLogger(Proxy.class); 
+  private static final Logger log = Logger.getLogger(Proxy.class);
   
   public static class PropertiesConverter implements IStringConverter<Properties> {
     @Override
@@ -96,8 +96,8 @@ public class Proxy {
       final File folder = Files.createTempDir();
       final MiniAccumuloCluster accumulo = new MiniAccumuloCluster(folder, "secret");
       accumulo.start();
-      opts.prop.setProperty("instance", accumulo.getInstanceName());
-      opts.prop.setProperty("zookeepers", accumulo.getZooKeepers());
+      opts.prop.setProperty("instance", accumulo.getConfig().getInstanceName());
+      opts.prop.setProperty("zookeepers", accumulo.getConfig().getZooKeepers());
       Runtime.getRuntime().addShutdownHook(new Thread() {
         public void start() {
           try {
@@ -110,7 +110,7 @@ public class Proxy {
         }
       });
     }
-
+    
     Class<? extends TProtocolFactory> protoFactoryClass = Class.forName(opts.prop.getProperty("protocolFactory", TCompactProtocol.Factory.class.getName()))
         .asSubclass(TProtocolFactory.class);
     int port = Integer.parseInt(opts.prop.getProperty("port"));
@@ -124,7 +124,7 @@ public class Proxy {
     
     // create the implementor
     Object impl = implementor.getConstructor(Properties.class).newInstance(properties);
-
+    
     Class<?> proxyProcClass = Class.forName(api.getName() + "$Processor");
     Class<?> proxyIfaceClass = Class.forName(api.getName() + "$Iface");
     @SuppressWarnings("unchecked")

Modified: accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java (original)
+++ accumulo/trunk/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java Sun May  5 01:55:52 2013
@@ -141,8 +141,8 @@ public class SimpleTest {
     accumulo.start();
     
     Properties props = new Properties();
-    props.put("instance", accumulo.getInstanceName());
-    props.put("zookeepers", accumulo.getZooKeepers());
+    props.put("instance", accumulo.getConfig().getInstanceName());
+    props.put("zookeepers", accumulo.getConfig().getZooKeepers());
     props.put("tokenClass", PasswordToken.class.getName());
     
     protocolClass = getRandomProtocol();
@@ -873,5 +873,4 @@ public class SimpleTest {
     accumulo.stop();
     folder.delete();
   }
-  
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java Sun May  5 01:55:52 2013
@@ -25,18 +25,14 @@ 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 org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.UtilWaitThread;
 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;
@@ -49,9 +45,6 @@ import org.apache.zookeeper.server.ZooKe
  */
 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;
@@ -97,48 +90,40 @@ public class MiniAccumuloCluster {
           in.close();
         }
         
-      } catch (IOException e) {
-      }
+      } 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[] tabletServerProcesses;
   
-  private int zooKeeperPort;
+  private File zooCfgFile;
   
   private List<LogWriter> logWriters = new ArrayList<MiniAccumuloCluster.LogWriter>();
   
   private MiniAccumuloConfig config;
-  private Process[] tabletServerProcesses;
   
   private 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;
+    classpath = config.getConfDir().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(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());
+    builder.environment().put("ACCUMULO_LOG_DIR", config.getLogDir().getAbsolutePath());
     
     // if we're running under accumulo.start, we forward these env vars
     String env = System.getenv("HADOOP_PREFIX");
@@ -151,25 +136,16 @@ public class MiniAccumuloCluster {
     Process process = builder.start();
     
     LogWriter lw;
-    lw = new LogWriter(process.getErrorStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".err"));
+    lw = new LogWriter(process.getErrorStream(), new File(config.getLogDir(), clazz.getSimpleName() + "_" + process.hashCode() + ".err"));
     logWriters.add(lw);
     lw.start();
-    lw = new LogWriter(process.getInputStream(), new File(logDir, clazz.getSimpleName() + "_" + process.hashCode() + ".out"));
+    lw = new LogWriter(process.getInputStream(), new File(config.getLogDir(), 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
@@ -188,7 +164,6 @@ public class MiniAccumuloCluster {
    *          initial configuration
    * @throws IOException
    */
-  
   public MiniAccumuloCluster(MiniAccumuloConfig config) throws IOException {
     
     if (config.getDir().exists() && !config.getDir().isDirectory())
@@ -197,66 +172,26 @@ public class MiniAccumuloCluster {
     if (config.getDir().exists() && config.getDir().list().length != 0)
       throw new IllegalArgumentException("Directory " + config.getDir() + " is not empty");
     
-    this.config = config;
+    this.config = config.initialize();
     
-    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();
+    config.getConfDir().mkdirs();
+    config.getAccumuloDir().mkdirs();
+    config.getZooKeeperDir().mkdirs();
+    config.getLogDir().mkdirs();
+    config.getWalogDir().mkdirs();
+    config.getLibDir().mkdirs();
     
-    zooKeeperPort = PortUtils.getRandomFreePort();
-    
-    File siteFile = new File(confDir, "accumulo-site.xml");
+    File siteFile = new File(config.getConfDir(), "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_TOKEN_PROPERTY_PREFIX + ".password", config.getRootPassword(), siteConfig);
-    appendProp(fileWriter, Property.TRACE_PORT, "" + PortUtils.getRandomFreePort(), 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())
+    for (Entry<String,String> entry : config.getSiteConfig().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");
+    zooCfgFile = new File(config.getConfDir(), "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
@@ -264,13 +199,12 @@ public class MiniAccumuloCluster {
     zooCfg.setProperty("tickTime", "1000");
     zooCfg.setProperty("initLimit", "10");
     zooCfg.setProperty("syncLimit", "5");
-    zooCfg.setProperty("clientPort", zooKeeperPort + "");
+    zooCfg.setProperty("clientPort", config.getZooKeeperPort() + "");
     zooCfg.setProperty("maxClientCnxns", "100");
-    zooCfg.setProperty("dataDir", zooKeeperDir.getAbsolutePath());
+    zooCfg.setProperty("dataDir", config.getZooKeeperDir().getAbsolutePath());
     zooCfg.store(fileWriter, null);
     
     fileWriter.close();
-    
   }
   
   /**
@@ -281,7 +215,6 @@ public class MiniAccumuloCluster {
    * @throws IllegalStateException
    *           if already started
    */
-  
   public void start() throws IOException, InterruptedException {
     if (zooKeeperProcess != null)
       throw new IllegalStateException("Already started");
@@ -304,7 +237,7 @@ public class MiniAccumuloCluster {
     // sleep a little bit to let zookeeper come up before calling init, seems to work better
     UtilWaitThread.sleep(250);
     
-    Process initProcess = exec(Initialize.class, "--instance-name", INSTANCE_NAME, "--password", config.getRootPassword(), "--username", "root");
+    Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword(), "--username", "root");
     int ret = initProcess.waitFor();
     if (ret != 0) {
       throw new RuntimeException("Initialize process returned " + ret);
@@ -319,29 +252,12 @@ public class MiniAccumuloCluster {
   }
   
   /**
-   * @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
+   * Stops Accumulo and Zookeeper processes. If stop is not called, there is a shutdown hook that is setup to kill the processes. However 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();
@@ -356,4 +272,8 @@ public class MiniAccumuloCluster {
     for (LogWriter lw : logWriters)
       lw.flush();
   }
+  
+  public MiniAccumuloConfig getConfig() {
+    return config;
+  }
 }

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java Sun May  5 01:55:52 2013
@@ -17,45 +17,116 @@
 package org.apache.accumulo.server.mini;
 
 import java.io.File;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.util.PortUtils;
+
 /**
- * Holds configuration for {@link MiniAccumuloCluster}. Required configurations must be passed to constructor and all other configurations are optional.
+ * Holds configuration for {@link MiniAccumuloCluster}. Required configurations must be passed to constructor(s) and all other configurations are optional.
  * 
  * @since 1.5.0
  */
-
 public class MiniAccumuloConfig {
   
+  private static final String DEFAULT_INSTANCE_SECRET = "DONTTELL";
+  
   private File dir = null;
   private String rootPassword = null;
-  private Map<String,String> siteConfig = Collections.emptyMap();
+  private Map<String,String> siteConfig = new HashMap<String,String>();
   private int numTservers = 2;
   
+  private String instanceName = "miniInstance";
+  
+  private File libDir;
+  private File confDir;
+  private File zooKeeperDir;
+  private File accumuloDir;
+  private File logDir;
+  private File walogDir;
+  
+  private Integer zooKeeperPort;
+  
+  private boolean initialized = false;
+  
   /**
    * @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.
+   *          An empty or nonexistant 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;
+
+  /**
+   * Set directories and fully populate site config
+   */
+  public MiniAccumuloConfig initialize() {
+    if (!initialized) {
+      libDir = new File(dir, "lib");
+      confDir = new File(dir, "conf");
+      accumuloDir = new File(dir, "accumulo");
+      zooKeeperDir = new File(dir, "zookeeper");
+      logDir = new File(dir, "logs");
+      walogDir = new File(dir, "walogs");
+      
+      String classpath = 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";
+      
+      mergeProp(Property.INSTANCE_DFS_URI.getKey(), "file:///");
+      mergeProp(Property.INSTANCE_DFS_DIR.getKey(), accumuloDir.getAbsolutePath());
+      mergeProp(Property.INSTANCE_SECRET.getKey(), DEFAULT_INSTANCE_SECRET);
+      mergeProp(Property.TSERV_PORTSEARCH.getKey(), "true");
+      mergeProp(Property.LOGGER_DIR.getKey(), walogDir.getAbsolutePath());
+      mergeProp(Property.TSERV_DATACACHE_SIZE.getKey(), "10M");
+      mergeProp(Property.TSERV_INDEXCACHE_SIZE.getKey(), "10M");
+      mergeProp(Property.TSERV_MAXMEM.getKey(), "50M");
+      mergeProp(Property.TSERV_WALOG_MAX_SIZE.getKey(), "100M");
+      mergeProp(Property.TSERV_NATIVEMAP_ENABLED.getKey(), "false");
+      mergeProp(Property.TRACE_TOKEN_PROPERTY_PREFIX + ".password", getRootPassword());
+      // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5
+      mergeProp(Property.TSERV_MAJC_DELAY.getKey(), "3");
+      mergeProp(Property.GENERAL_CLASSPATHS.getKey(), classpath);
+      mergeProp(Property.GENERAL_DYNAMIC_CLASSPATHS.getKey(), libDir.getAbsolutePath());
+      mergePropWithRandomPort(Property.MASTER_CLIENTPORT.getKey());
+      mergePropWithRandomPort(Property.TRACE_PORT.getKey());
+      mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
+      
+      // zookeeper port should be set explicitly in this class, not just on the site config
+      if (zooKeeperPort == null)
+        zooKeeperPort = PortUtils.getRandomFreePort();
+      siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), "localhost:" + zooKeeperPort);
+      initialized = true;
+    }
+    return this;
   }
   
-  public String getRootPassword() {
-    return rootPassword;
+  /**
+   * Set a given key/value on the site config if it doesn't already exist
+   * 
+   * @param key
+   * @param value
+   */
+  private void mergeProp(String key, String value) {
+    if (!siteConfig.containsKey(key)) {
+      siteConfig.put(key, value);
+    }
   }
   
-  public int getNumTservers() {
-    return numTservers;
+  /**
+   * Sets a given key with a random port for the value on the site config if it doesn't already exist.
+   * 
+   * @param key
+   */
+  private void mergePropWithRandomPort(String key) {
+    if (!siteConfig.containsKey(key)) {
+      siteConfig.put(key, PortUtils.getRandomFreePort() + "");
+    }
   }
   
   /**
@@ -64,27 +135,137 @@ public class MiniAccumuloConfig {
    * @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.
+   * Calling this method is optional. If not set, defaults to 'miniInstance'
    * 
+   * @param instanceName
+   * @return
+   */
+  public MiniAccumuloConfig setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+    return this;
+  }
+
+  /**
+   * Calling this method is optional. If not set, it defaults to an empty map.
+   *
    * @param siteConfig
-   *          key/values that you normally put in accumulo-site.xml can be put here
+   *          key/values that you normally put in accumulo-site.xml can be put here.
    */
-  
   public MiniAccumuloConfig setSiteConfig(Map<String,String> siteConfig) {
-    this.siteConfig = siteConfig;
+    this.siteConfig = new HashMap<String,String>(siteConfig);
     return this;
   }
+
+  /**
+   * Calling this method is optional. A random port is generated by default
+   * 
+   * @param zooKeeperPort
+   *          A valid (and unused) port to use for the zookeeper
+   * @return
+   */
+  public MiniAccumuloConfig setZooKeeperPort(int zooKeeperPort) {
+    this.zooKeeperPort = zooKeeperPort;
+    return this;
+  }
+
+  /**
+   * @return a copy of the site config
+   */
+  public Map<String,String> getSiteConfig() {
+    return new HashMap<String,String>(siteConfig);
+  }
+
+  /**
+   * @return name of configured instance
+   */
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  /**
+   * @return The configured zookeeper port
+   */
+  public int getZooKeeperPort() {
+    return zooKeeperPort;
+  }
+
+  public File getLibDir() {
+    return libDir;
+  }
+  
+  public File getConfDir() {
+    return confDir;
+  }
+  
+  public File getZooKeeperDir() {
+    return zooKeeperDir;
+  }
+  
+  public File getAccumuloDir() {
+    return accumuloDir;
+  }
+  
+  public File getLogDir() {
+    return logDir;
+  }
+  
+  public File getWalogDir() {
+    return walogDir;
+  }
+  
+  /**
+   * @return zookeeper connection string
+   */
+  public String getZooKeepers() {
+    return siteConfig.get(Property.INSTANCE_ZK_HOST.getKey());
+  }
+
+  /**
+   * @return master client port
+   */
+  public String getMasterClientPort() {
+    return siteConfig.get(Property.MASTER_CLIENTPORT.getKey());
+  }
+
+  /**
+   * @return trace port
+   */
+  public String getTracePort() {
+    return siteConfig.get(Property.TRACE_PORT.getKey());
+  }
+
+  /**
+   * @return tablet server client port
+   */
+  public String getTabletServerClientPort() {
+    return siteConfig.get(Property.TSERV_CLIENTPORT.getKey());
+  }
+
+  /**
+   * @return Has the current instance been initialized?
+   */
+  public boolean isInitialized() {
+    return initialized;
+  }
+
+  public File getDir() {
+    return dir;
+  }
+
+  public String getRootPassword() {
+    return rootPassword;
+  }
+
+  public int getNumTservers() {
+    return numTservers;
+  }
+
 }

Modified: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java (original)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java Sun May  5 01:55:52 2013
@@ -68,7 +68,8 @@ public class MiniAccumuloClusterTest {
   
   @Test(timeout = 30000)
   public void test() throws Exception {
-    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", new PasswordToken("superSecret"));
+    Connector conn = new ZooKeeperInstance(accumulo.getConfig().getInstanceName(), accumulo.getConfig().getZooKeepers()).getConnector("root",
+        new PasswordToken("superSecret"));
     
     conn.tableOperations().create("table1");
     
@@ -83,7 +84,8 @@ public class MiniAccumuloClusterTest {
     
     conn.tableOperations().attachIterator("table1", is);
     
-    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("user1", new PasswordToken("pass1"));
+    Connector uconn = new ZooKeeperInstance(accumulo.getConfig().getInstanceName(), accumulo.getConfig().getZooKeepers()).getConnector("user1",
+        new PasswordToken("pass1"));
     
     BatchWriter bw = uconn.createBatchWriter("table1", new BatchWriterConfig());
     
@@ -139,7 +141,8 @@ public class MiniAccumuloClusterTest {
   @Test(timeout = 60000)
   public void testPerTableClasspath() throws Exception {
     
-    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", new PasswordToken("superSecret"));
+    Connector conn = new ZooKeeperInstance(accumulo.getConfig().getInstanceName(), accumulo.getConfig().getZooKeepers()).getConnector("root",
+        new PasswordToken("superSecret"));
     
     conn.tableOperations().create("table2");
     
@@ -179,7 +182,6 @@ public class MiniAccumuloClusterTest {
     
     conn.instanceOperations().removeProperty(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
     conn.tableOperations().delete("table2");
-    
   }
   
   @AfterClass

Added: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java?rev=1479227&view=auto
==============================================================================
--- accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java (added)
+++ accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java Sun May  5 01:55:52 2013
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.mini;
+
+import org.apache.accumulo.core.conf.Property;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class MiniAccumuloConfigTest {
+
+  static TemporaryFolder tempFolder = new TemporaryFolder();
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    tempFolder.create();
+  }
+
+  @Test
+  public void testZookeeperPort(){
+
+    // set specific zookeeper port
+    MiniAccumuloConfig config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").setZooKeeperPort(5000).initialize();
+    assertEquals(5000, config.getZooKeeperPort());
+
+    // generate zookeeper port
+    config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").initialize();
+    assertTrue(config.getZooKeeperPort() > 0);
+  }
+
+  @Test
+  public void testSiteConfig() {
+
+    // constructor site config overrides default props
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), "hdfs://");
+    MiniAccumuloConfig config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").setSiteConfig(siteConfig).initialize();
+    assertEquals("hdfs://", config.getSiteConfig().get(Property.INSTANCE_DFS_URI.getKey()));
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    tempFolder.delete();
+  }
+}

Propchange: accumulo/trunk/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloConfigTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java Sun May  5 01:55:52 2013
@@ -101,16 +101,16 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testDeleteParent() throws Exception {
-    accumulo.getZooKeepers();
+    accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     
     // intentionally created parent after lock
     zk.mkdirs(parent);
@@ -135,11 +135,11 @@ public class ZooLockTest {
   
   @Test(timeout = 10000)
   public void testNoParent() throws Exception {
-    accumulo.getZooKeepers();
+    accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
     
@@ -157,14 +157,14 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testDeleteLock() throws Exception {
-    accumulo.getZooKeepers();
+    accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
     
@@ -190,14 +190,14 @@ public class ZooLockTest {
   
   @Test(timeout = 10000)
   public void testDeleteWaiting() throws Exception {
-    accumulo.getZooKeepers();
+    accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
     
@@ -213,7 +213,7 @@ public class ZooLockTest {
     Assert.assertNull(lw.reason);
     
     
-    ZooLock zl2 = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl2 = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
     
     TestALW lw2 = new TestALW();
     
@@ -222,7 +222,7 @@ public class ZooLockTest {
     Assert.assertFalse(lw2.locked);
     Assert.assertFalse(zl2.isLocked());
     
-    ZooLock zl3 = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl3 = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
     
     TestALW lw3 = new TestALW();
     
@@ -259,16 +259,16 @@ public class ZooLockTest {
   
   @Test(timeout = 10000)
   public void testUnexpectedEvent() throws Exception {
-    accumulo.getZooKeepers();
+    accumulo.getConfig().getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 30000, null);
+    ZooKeeper zk = new ZooKeeper(accumulo.getConfig().getZooKeepers(), 30000, null);
     zk.addAuthInfo("digest", "secret".getBytes());
     
     zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
     
     Assert.assertFalse(zl.isLocked());
     
@@ -302,9 +302,9 @@ public class ZooLockTest {
   public void testTryLock() throws Exception {
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
-    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
 
-    ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, null);
+    ZooKeeper zk = new ZooKeeper(accumulo.getConfig().getZooKeepers(), 1000, null);
     zk.addAuthInfo("digest", "secret".getBytes());
     
     for (int i = 0; i < 10; i++) {

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Sun May  5 01:55:52 2013
@@ -67,7 +67,7 @@ public class MetaSplitTest {
   
   @Test(timeout = 60000)
   public void testMetaSplit() throws Exception {
-    Instance instance = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Instance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
     Connector connector = instance.getConnector("root", new PasswordToken(secret));
     TableOperations opts = connector.tableOperations();
     for (int i = 1; i <= 10; i++) {

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Sun May  5 01:55:52 2013
@@ -143,7 +143,7 @@ public class ShellServerTest {
     output = new TestOutputStream();
     shell = new Shell(new ConsoleReader(new FileInputStream(FileDescriptor.in), new OutputStreamWriter(output)));
     shell.setLogErrorsToConsole();
-    shell.config("-u", "root", "-p", secret, "-z", cluster.getInstanceName(), cluster.getZooKeepers());
+    shell.config("-u", "root", "-p", secret, "-z", cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
     exec("quit", true);
     shell.start();
     shell.setExit(false);
@@ -690,7 +690,7 @@ public class ShellServerTest {
     Thread thread = new Thread() {
       public void run() {
         try {
-          ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+          ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
           Connector connector = instance.getConnector("root", new PasswordToken(secret));
           Scanner s = connector.createScanner("t", Constants.NO_AUTHS);
           for (@SuppressWarnings("unused") Entry<Key,Value> kv : s)

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java Sun May  5 01:55:52 2013
@@ -71,7 +71,7 @@ public class TableOperationsIT {
     
     accumuloCluster.start();
     
-    ZooKeeperInstance instance = new ZooKeeperInstance(accumuloCluster.getInstanceName(), accumuloCluster.getZooKeepers());
+    ZooKeeperInstance instance = new ZooKeeperInstance(accumuloCluster.getConfig().getInstanceName(), accumuloCluster.getConfig().getZooKeepers());
     connector = instance.getConnector(ROOT, new PasswordToken(ROOT_PASS.getBytes()));
   }
   
@@ -159,7 +159,7 @@ public class TableOperationsIT {
     assertEquals(DefaultKeySizeConstraint.class.getName(), props.get(Property.TABLE_CONSTRAINT_PREFIX.toString() + "1"));
     connector.tableOperations().delete("table1");
   }
-
+  
   private Map<String,String> propsToMap(Iterable<Map.Entry<String,String>> props) {
     Map<String,String> map = new HashMap<String,String>();
     for (Map.Entry<String,String> prop : props) {

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java?rev=1479227&r1=1479226&r2=1479227&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java Sun May  5 01:55:52 2013
@@ -84,7 +84,7 @@ public class TestAccumuloSplitRecovery {
     
     for (int tn = 0; tn < 2; tn++) {
     
-      ZooKeeperInstance instance = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers());
+      ZooKeeperInstance instance = new ZooKeeperInstance(accumulo.getConfig().getInstanceName(), accumulo.getConfig().getZooKeepers());
       Connector connector = instance.getConnector("root", new PasswordToken(secret));
       // create a table and put some data in it
       connector.tableOperations().create(TABLE);