You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2014/10/21 22:41:43 UTC

git commit: ACCUMULO-2984 adding ability to run MAC against a permanent accumulo instance

Repository: accumulo
Updated Branches:
  refs/heads/1.6 758a364bd -> e3a743cb4


ACCUMULO-2984 adding ability to run MAC against a permanent accumulo instance

Addressing NPEs with existingInstance

ACCUMULO-2984 add test for existing mac using existing instance

Fixing my NPE fix and integrating Keith's test

Adding ability to clean out singleton stuff

Now no longer doing weird things with statics

Warning cleanup

Removing added features no longer needed

Cleaning up minor ticket items

Fixing another Configuration plumbing issue and adding some info statements


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

Branch: refs/heads/1.6
Commit: e3a743cb445723a3d5664a4bf1ebf37833152aae
Parents: 758a364
Author: John Vines <vi...@apache.org>
Authored: Wed Oct 15 18:37:29 2014 -0400
Committer: John Vines <vi...@apache.org>
Committed: Tue Oct 21 16:06:25 2014 -0400

----------------------------------------------------------------------
 .../core/volume/VolumeConfiguration.java        |  12 +-
 .../apache/accumulo/core/zookeeper/ZooUtil.java |  17 +-
 .../minicluster/MiniAccumuloConfig.java         |  19 +-
 .../minicluster/MiniAccumuloRunner.java         |   2 +-
 .../impl/MiniAccumuloClusterImpl.java           | 208 ++++++++++++-------
 .../impl/MiniAccumuloConfigImpl.java            | 145 +++++++++++--
 .../server/conf/ServerConfiguration.java        |   1 -
 .../accumulo/server/fs/VolumeManagerImpl.java   |   7 +-
 .../accumulo/server/util/AccumuloStatus.java    |  74 +++++++
 .../org/apache/accumulo/test/ExistingMacIT.java | 152 ++++++++++++++
 10 files changed, 537 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
index c901768..8d56d9e 100644
--- a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
@@ -60,16 +60,15 @@ public class VolumeConfiguration {
   }
 
   /**
-   * @see org.apache.accumulo.core.volume.VolumeConfiguration#getVolumeUris(AccumuloConfiguration)
+   * @see org.apache.accumulo.core.volume.VolumeConfiguration#getVolumeUris(AccumuloConfiguration,Configuration)
    */
   @Deprecated
-  public static String getConfiguredBaseDir(AccumuloConfiguration conf) {
+  public static String getConfiguredBaseDir(AccumuloConfiguration conf, Configuration hadoopConfig) {
     String singleNamespace = conf.get(Property.INSTANCE_DFS_DIR);
     String dfsUri = conf.get(Property.INSTANCE_DFS_URI);
     String baseDir;
 
     if (dfsUri == null || dfsUri.isEmpty()) {
-      Configuration hadoopConfig = CachedConfiguration.getInstance();
       try {
         baseDir = FileSystem.get(hadoopConfig).getUri().toString() + singleNamespace;
       } catch (IOException e) {
@@ -85,15 +84,20 @@ public class VolumeConfiguration {
 
   /**
    * Compute the URIs to be used by Accumulo
+   * 
    */
   public static String[] getVolumeUris(AccumuloConfiguration conf) {
+    return getVolumeUris(conf, CachedConfiguration.getInstance());
+  }
+  
+  public static String[] getVolumeUris(AccumuloConfiguration conf, Configuration hadoopConfig) {
     String ns = conf.get(Property.INSTANCE_VOLUMES);
 
     String configuredBaseDirs[];
 
     if (ns == null || ns.isEmpty()) {
       // Fall back to using the old config values
-      configuredBaseDirs = new String[] {getConfiguredBaseDir(conf)};
+      configuredBaseDirs = new String[] {getConfiguredBaseDir(conf, hadoopConfig)};
     } else {
       String namespaces[] = ns.split(",");
       configuredBaseDirs = new String[namespaces.length];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
index d536f42..9ee7c99 100644
--- a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
@@ -25,31 +25,34 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
-  
+
   private static final Logger log = Logger.getLogger(ZooUtil.class);
-  
+
   public static String getRoot(final Instance instance) {
     return getRoot(instance.getInstanceID());
   }
-  
+
   public static String getRoot(final String instanceId) {
     return Constants.ZROOT + "/" + instanceId;
   }
-  
+
   /**
    * Utility to support certain client side utilities to minimize command-line options.
    */
-
   public static String getInstanceIDFromHdfs(Path instanceDirectory, AccumuloConfiguration conf) {
-    try {
+    return getInstanceIDFromHdfs(instanceDirectory, conf, CachedConfiguration.getInstance());
+  }
 
-      FileSystem fs = VolumeConfiguration.getVolume(instanceDirectory.toString(), CachedConfiguration.getInstance(), conf).getFileSystem();
+  public static String getInstanceIDFromHdfs(Path instanceDirectory, AccumuloConfiguration conf, Configuration hadoopConf) {
+    try {
+      FileSystem fs = VolumeConfiguration.getVolume(instanceDirectory.toString(), hadoopConf, conf).getFileSystem();
       FileStatus[] files = null;
       try {
         files = fs.listStatus(instanceDirectory);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index be80f85..68e30fa 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.minicluster;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.Map;
 
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -95,8 +96,7 @@ public class MiniAccumuloConfig {
   }
 
   /**
-   * Configure the time to wait for ZooKeeper to startup.
-   * Calling this method is optional. The default is 20000 milliseconds
+   * Configure the time to wait for ZooKeeper to startup. Calling this method is optional. The default is 20000 milliseconds
    * 
    * @param zooKeeperStartupTime
    *          Time to wait for ZooKeeper to startup, in milliseconds
@@ -252,4 +252,19 @@ public class MiniAccumuloConfig {
     impl.setNativeLibPaths(nativePathItems);
     return this;
   }
+
+  /**
+   * Informs MAC that it's running against an existing accumulo instance. It is assumed that it's already initialized and hdfs/zookeeper are already running.
+   *
+   * @param accumuloSite
+   *          a File representation of the accumulo-site.xml file for the instance being run
+   * @param hadoopConfDir
+   *          a File representation of the hadoop configuration directory containing core-site.xml and hdfs-site.xml
+   *
+   * @since 1.6.2
+   */
+  public MiniAccumuloConfig useExistingInstance(File accumuloSite, File hadoopConfDir) throws IOException {
+    impl.useExistingInstance(accumuloSite, hadoopConfDir);
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index e261faa..22eca84 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -21,7 +21,6 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.ServerSocket;
-import java.net.Socket;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
@@ -31,6 +30,7 @@ import java.util.regex.Pattern;
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.commons.io.FileUtils;
+
 import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
 import com.google.common.io.Files;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 5d8501e..d5dc1e9 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -50,6 +50,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
@@ -59,22 +60,31 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.state.SetGoalState;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.init.Initialize;
 import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.util.AccumuloStatus;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.server.util.time.SimpleTimer;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriterFactory;
 import org.apache.accumulo.start.Main;
 import org.apache.accumulo.start.classloader.vfs.MiniDFSUtil;
 import org.apache.accumulo.trace.instrument.Tracer;
@@ -84,12 +94,17 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.vfs2.FileObject;
 import org.apache.commons.vfs2.impl.VFSClassLoader;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Maps;
@@ -101,7 +116,7 @@ import com.google.common.collect.Maps;
  * @since 1.6.0
  */
 public class MiniAccumuloClusterImpl implements AccumuloCluster {
-  private static final Logger log = Logger.getLogger(MiniAccumuloClusterImpl.class);
+  private static final Logger log = LoggerFactory.getLogger(MiniAccumuloClusterImpl.class);
 
   public static class LogWriter extends Daemon {
     private BufferedReader in;
@@ -219,6 +234,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       StringBuilder classpathBuilder = new StringBuilder();
       classpathBuilder.append(config.getConfDir().getAbsolutePath());
 
+      if (config.getHadoopConfDir() != null)
+        classpathBuilder.append(File.pathSeparator).append(config.getHadoopConfDir().getAbsolutePath());
+
       if (config.getClasspathItems() == null) {
 
         // assume 0 is the system classloader and skip it
@@ -268,7 +286,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     for (Entry<String,String> sysProp : config.getSystemProperties().entrySet()) {
       argList.add(String.format("-D%s=%s", sysProp.getKey(), sysProp.getValue()));
     }
-    argList.addAll(Arrays.asList("-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75", "-Dapple.awt.UIElement=true", Main.class.getName(), className));
+    argList.addAll(Arrays.asList("-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75", "-Dapple.awt.UIElement=true", Main.class.getName(),
+        className));
     argList.addAll(Arrays.asList(args));
 
     ProcessBuilder builder = new ProcessBuilder(argList);
@@ -290,6 +309,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     builder.environment().put("ACCUMULO_CONF_DIR", config.getConfDir().getAbsolutePath());
     // hadoop-2.2 puts error messages in the logs if this is not set
     builder.environment().put("HADOOP_HOME", config.getDir().getAbsolutePath());
+    if (config.getHadoopConfDir() != null)
+      builder.environment().put("HADOOP_CONF_DIR", config.getHadoopConfDir().getAbsolutePath());
 
     Process process = builder.start();
 
@@ -339,13 +360,16 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     this.config = config.initialize();
 
     config.getConfDir().mkdirs();
-    config.getAccumuloDir().mkdirs();
-    config.getZooKeeperDir().mkdirs();
     config.getLogDir().mkdirs();
-    config.getWalogDir().mkdirs();
     config.getLibDir().mkdirs();
     config.getLibExtDir().mkdirs();
 
+    if (!config.useExistingInstance()) {
+      config.getZooKeeperDir().mkdirs();
+      config.getWalogDir().mkdirs();
+      config.getAccumuloDir().mkdirs();
+    }
+
     if (config.useMiniDFS()) {
       File nn = new File(config.getAccumuloDir(), "nn");
       nn.mkdirs();
@@ -378,6 +402,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), dfsUri);
       siteConfig.put(Property.INSTANCE_DFS_DIR.getKey(), "/accumulo");
       config.setSiteConfig(siteConfig);
+    } else if (config.useExistingInstance()) {
+      dfsUri = CachedConfiguration.getInstance().get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY);
     } else {
       dfsUri = "file://";
     }
@@ -394,20 +420,22 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     File siteFile = new File(config.getConfDir(), "accumulo-site.xml");
     writeConfig(siteFile, config.getSiteConfig().entrySet());
 
-    zooCfgFile = new File(config.getConfDir(), "zoo.cfg");
-    FileWriter 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", "2000");
-    zooCfg.setProperty("initLimit", "10");
-    zooCfg.setProperty("syncLimit", "5");
-    zooCfg.setProperty("clientPort", config.getZooKeeperPort() + "");
-    zooCfg.setProperty("maxClientCnxns", "1000");
-    zooCfg.setProperty("dataDir", config.getZooKeeperDir().getAbsolutePath());
-    zooCfg.store(fileWriter, null);
-
-    fileWriter.close();
+    if (!config.useExistingInstance()) {
+      zooCfgFile = new File(config.getConfDir(), "zoo.cfg");
+      FileWriter 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", "2000");
+      zooCfg.setProperty("initLimit", "10");
+      zooCfg.setProperty("syncLimit", "5");
+      zooCfg.setProperty("clientPort", config.getZooKeeperPort() + "");
+      zooCfg.setProperty("maxClientCnxns", "1000");
+      zooCfg.setProperty("dataDir", config.getZooKeeperDir().getAbsolutePath());
+      zooCfg.store(fileWriter, null);
+
+      fileWriter.close();
+    }
 
     // disable audit logging for mini....
     InputStream auditStream = this.getClass().getResourceAsStream("/auditLog.xml");
@@ -445,58 +473,98 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
    */
   @Override
   public synchronized void start() throws IOException, InterruptedException {
+    if (config.useExistingInstance()) {
+      Configuration acuConf = config.getAccumuloConfiguration();
+      Configuration hadoopConf = config.getHadoopConfiguration();
+
+      ConfigurationCopy cc = new ConfigurationCopy(acuConf);
+      VolumeManager fs;
+      try {
+        fs = VolumeManagerImpl.get(cc, hadoopConf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(fs);
 
-    if (!initialized) {
+      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, cc, hadoopConf);
+      IZooReaderWriter zrw = new ZooReaderWriterFactory().getZooReaderWriter(cc.get(Property.INSTANCE_ZK_HOST),
+          (int) cc.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), cc.get(Property.INSTANCE_SECRET));
 
-      Runtime.getRuntime().addShutdownHook(new Thread() {
-        @Override
-        public void run() {
-          try {
-            MiniAccumuloClusterImpl.this.stop();
-          } catch (IOException e) {
-            e.printStackTrace();
-          } catch (InterruptedException e) {
-            e.printStackTrace();
+      String rootPath = ZooUtil.getRoot(instanceIdFromFile);
+
+      String instanceName = null;
+      try {
+        for (String name : zrw.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
+          String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
+          byte[] bytes = zrw.getData(instanceNamePath, new Stat());
+          String iid = new String(bytes, Constants.UTF8);
+          if (iid.equals(instanceIdFromFile)) {
+            instanceName = name;
           }
         }
-      });
-    }
+      } catch (KeeperException e) {
+        throw new RuntimeException("Unable to read instance name from zookeeper.", e);
+      }
+      if (instanceName == null)
+        throw new RuntimeException("Unable to read instance name from zookeeper.");
 
-    if (zooKeeperProcess == null) {
-      zooKeeperProcess = _exec(ZooKeeperServerMain.class, ServerType.ZOOKEEPER, zooCfgFile.getAbsolutePath());
-    }
+      config.setInstanceName(instanceName);
+      if (!AccumuloStatus.isAccumuloOffline(zrw, rootPath))
+        throw new RuntimeException("The Accumulo instance being used is already running. Aborting.");
+    } else {
+      if (!initialized) {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+          @Override
+          public void run() {
+            try {
+              MiniAccumuloClusterImpl.this.stop();
+            } catch (IOException e) {
+              e.printStackTrace();
+            } catch (InterruptedException e) {
+              e.printStackTrace();
+            }
+          }
+        });
+      }
+
+      if (zooKeeperProcess == null) {
+        zooKeeperProcess = _exec(ZooKeeperServerMain.class, ServerType.ZOOKEEPER, zooCfgFile.getAbsolutePath());
+      }
 
-    if (!initialized) {
-      // sleep a little bit to let zookeeper come up before calling init, seems to work better
-      long startTime = System.currentTimeMillis();
-      while (true) {
-        Socket s = null;
-        try {
-          s = new Socket("localhost", config.getZooKeeperPort());
-          s.getOutputStream().write("ruok\n".getBytes());
-          s.getOutputStream().flush();
-          byte buffer[] = new byte[100];
-          int n = s.getInputStream().read(buffer);
-          if (n >= 4 && new String(buffer, 0, 4).equals("imok"))
-            break;
-        } catch (Exception e) {
-          if (System.currentTimeMillis() - startTime >= config.getZooKeeperStartupTime()) {
-            throw new ZooKeeperBindException("Zookeeper did not start within " + (config.getZooKeeperStartupTime() / 1000) + " seconds. Check the logs in "
-                + config.getLogDir() + " for errors.  Last exception: " + e);
+      if (!initialized) {
+        // sleep a little bit to let zookeeper come up before calling init, seems to work better
+        long startTime = System.currentTimeMillis();
+        while (true) {
+          Socket s = null;
+          try {
+            s = new Socket("localhost", config.getZooKeeperPort());
+            s.getOutputStream().write("ruok\n".getBytes());
+            s.getOutputStream().flush();
+            byte buffer[] = new byte[100];
+            int n = s.getInputStream().read(buffer);
+            if (n >= 4 && new String(buffer, 0, 4).equals("imok"))
+              break;
+          } catch (Exception e) {
+            if (System.currentTimeMillis() - startTime >= config.getZooKeeperStartupTime()) {
+              throw new ZooKeeperBindException("Zookeeper did not start within " + (config.getZooKeeperStartupTime() / 1000) + " seconds. Check the logs in "
+                  + config.getLogDir() + " for errors.  Last exception: " + e);
+            }
+          } finally {
+            if (s != null)
+              s.close();
           }
-          UtilWaitThread.sleep(250);
-        } finally {
-          if (s != null)
-            s.close();
         }
+        Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword());
+        int ret = initProcess.waitFor();
+        if (ret != 0) {
+          throw new RuntimeException("Initialize process returned " + ret + ". Check the logs in " + config.getLogDir() + " for errors.");
+        }
+        initialized = true;
       }
-      Process initProcess = exec(Initialize.class, "--instance-name", config.getInstanceName(), "--password", config.getRootPassword());
-      int ret = initProcess.waitFor();
-      if (ret != 0) {
-        throw new RuntimeException("Initialize process returned " + ret + ". Check the logs in " + config.getLogDir() + " for errors.");
-      }
-      initialized = true;
     }
+    
+    log.info("Starting MAC against instance {} and zookeeper(s) {}.", config.getInstanceName(), config.getZooKeepers());
+    
     synchronized (tabletServerProcesses) {
       for (int i = tabletServerProcesses.size(); i < config.getNumTservers(); i++) {
         tabletServerProcesses.add(_exec(TabletServer.class, ServerType.TABLET_SERVER));
@@ -744,11 +812,11 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
   private int stopProcessWithTimeout(final Process proc, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
     FutureTask<Integer> future = new FutureTask<Integer>(new Callable<Integer>() {
-        @Override
-        public Integer call() throws InterruptedException {
-          proc.destroy();
-          return proc.waitFor();
-        }
+      @Override
+      public Integer call() throws InterruptedException {
+        proc.destroy();
+        return proc.waitFor();
+      }
     });
 
     executor.execute(future);
@@ -757,9 +825,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   }
 
   /**
-   * Get programmatic interface to information available in a normal monitor.
-   * XXX the returned structure won't contain information about the metadata table until there is data in it.
-   * e.g. if you want to see the metadata table you should create a table.
+   * Get programmatic interface to information available in a normal monitor. XXX the returned structure won't contain information about the metadata table
+   * until there is data in it. e.g. if you want to see the metadata table you should create a table.
+   * 
    * @since 1.6.1
    */
   public MasterMonitorInfo getMasterMonitorInfo() throws AccumuloException, AccumuloSecurityException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index e9ad045..2d7103e 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.minicluster.impl;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.MalformedURLException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -25,6 +26,7 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.AccumuloConfig;
 import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
@@ -55,6 +57,7 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
   private File libDir;
   private File libExtDir;
   private File confDir;
+  private File hadoopConfDir = null;
   private File zooKeeperDir;
   private File accumuloDir;
   private File logDir;
@@ -62,11 +65,12 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
 
   private int zooKeeperPort = 0;
   private int configuredZooKeeperPort = 0;
-  private long zooKeeperStartupTime = 20*1000;
+  private long zooKeeperStartupTime = 20 * 1000;
 
   private long defaultMemorySize = 128 * 1024 * 1024;
 
   private boolean initialized = false;
+  private Boolean existingInstance = null;
 
   private boolean useMiniDFS = false;
 
@@ -76,6 +80,10 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
 
   private String[] nativePathItems = null;
 
+  // These are only used on top of existing instances
+  private Configuration hadoopConf;
+  private Configuration accumuloConf;
+
   /**
    * @param dir
    *          An empty or nonexistant directory that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava, and
@@ -109,18 +117,22 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
       logDir = new File(dir, "logs");
       walogDir = new File(dir, "walogs");
 
-      // TODO ACCUMULO-XXXX replace usage of instance.dfs.{dir,uri} with instance.volumes
-      setInstanceLocation();
+      // Never want to override these if an existing instance, which may be using the defaults
+      if (existingInstance == null || !existingInstance) {
+        existingInstance = false;
+        // TODO ACCUMULO-XXXX replace usage of instance.dfs.{dir,uri} with instance.volumes
+        setInstanceLocation();
+        mergeProp(Property.INSTANCE_SECRET.getKey(), DEFAULT_INSTANCE_SECRET);
+        mergeProp(Property.LOGGER_DIR.getKey(), walogDir.getAbsolutePath());
+        mergeProp(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey() + "password", getRootPassword());
+      }
 
-      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.getKey() + "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(), libDir.getAbsolutePath() + "/[^.].*[.]jar");
@@ -138,10 +150,13 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
         updateConfigForCredentialProvider();
       }
 
-      // zookeeper port should be set explicitly in this class, not just on the site config
-      if (zooKeeperPort == 0)
-        zooKeeperPort = PortUtils.getRandomFreePort();
-      siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), "localhost:" + zooKeeperPort);
+      if (existingInstance == null || !existingInstance) {
+        existingInstance = false;
+        // zookeeper port should be set explicitly in this class, not just on the site config
+        if (zooKeeperPort == 0)
+          zooKeeperPort = PortUtils.getRandomFreePort();
+        siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), "localhost:" + zooKeeperPort);
+      }
       initialized = true;
     }
     return this;
@@ -244,6 +259,15 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
    */
   @Override
   public MiniAccumuloConfigImpl setSiteConfig(Map<String,String> siteConfig) {
+    if (existingInstance != null && existingInstance.booleanValue())
+      throw new UnsupportedOperationException("Cannot set set config info when using an existing instance.");
+
+    this.existingInstance = Boolean.FALSE;
+
+    return _setSiteConfig(siteConfig);
+  }
+
+  private MiniAccumuloConfigImpl _setSiteConfig(Map<String,String> siteConfig) {
     this.siteConfig = new HashMap<String,String>(siteConfig);
     this.configuredSiteConig = new HashMap<String,String>(siteConfig);
     return this;
@@ -259,14 +283,19 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
    */
   @Override
   public MiniAccumuloConfigImpl setZooKeeperPort(int zooKeeperPort) {
+    if (existingInstance != null && existingInstance.booleanValue())
+      throw new UnsupportedOperationException("Cannot set zookeeper info when using an existing instance.");
+
+    this.existingInstance = Boolean.FALSE;
+
     this.configuredZooKeeperPort = zooKeeperPort;
     this.zooKeeperPort = zooKeeperPort;
     return this;
   }
 
   /**
-   * Configure the time to wait for ZooKeeper to startup.
-   * Calling this method is optional. The default is 20000 milliseconds
+   * <<<<<<< HEAD Configure the time to wait for ZooKeeper to startup. Calling this method is optional. The default is 20000 milliseconds ======= Configure the
+   * time to wait for ZooKeeper to startup. Calling this method is optional. The default is 20000 milliseconds >>>>>>> ACCUMULO-2984
    *
    * @param zooKeeperStartupTime
    *          Time to wait for ZooKeeper to startup, in milliseconds
@@ -275,6 +304,11 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
    */
   @Override
   public MiniAccumuloConfigImpl setZooKeeperStartupTime(long zooKeeperStartupTime) {
+    if (existingInstance != null && existingInstance.booleanValue())
+      throw new UnsupportedOperationException("Cannot set zookeeper info when using an existing instance.");
+
+    this.existingInstance = Boolean.FALSE;
+
     this.zooKeeperStartupTime = zooKeeperStartupTime;
     return this;
   }
@@ -557,7 +591,8 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
   }
 
   /**
-   * @param useCredentialProvider the useCredentialProvider to set
+   * @param useCredentialProvider
+   *          the useCredentialProvider to set
    */
   public void setUseCredentialProvider(boolean useCredentialProvider) {
     this.useCredentialProvider = useCredentialProvider;
@@ -567,4 +602,88 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
   public MiniAccumuloClusterImpl build() throws IOException {
     return new MiniAccumuloClusterImpl(this);
   }
+
+  /**
+   * Informs MAC that it's running against an existing accumulo instance. It is assumed that it's already initialized and hdfs/zookeeper are already running.
+   *
+   * @param accumuloSite
+   *          a File representation of the accumulo-site.xml file for the instance being run
+   * @param hadoopConfDir
+   *          a File representation of the hadoop configuration directory containing core-site.xml and hdfs-site.xml
+   * 
+   * @return MiniAccumuloConfigImpl which uses an existing accumulo configuration
+   *
+   * @since 1.6.2
+   *
+   * @throws IOException
+   *           when there are issues converting the provided Files to URLs
+   */
+  public MiniAccumuloConfigImpl useExistingInstance(File accumuloSite, File hadoopConfDir) throws IOException {
+    if (existingInstance != null && !existingInstance.booleanValue())
+      throw new UnsupportedOperationException("Cannot set to useExistingInstance after specifying config/zookeeper");
+
+    this.existingInstance = Boolean.TRUE;
+
+    System.setProperty("org.apache.accumulo.config.file", "accumulo-site.xml");
+    this.hadoopConfDir = hadoopConfDir;
+    hadoopConf = new Configuration(false);
+    accumuloConf = new Configuration(false);
+    File coreSite = new File(hadoopConfDir, "core-site.xml");
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+
+    try {
+      accumuloConf.addResource(accumuloSite.toURI().toURL());
+      hadoopConf.addResource(coreSite.toURI().toURL());
+      hadoopConf.addResource(hdfsSite.toURI().toURL());
+    } catch (MalformedURLException e1) {
+      throw e1;
+    }
+
+    Map<String,String> siteConfigMap = new HashMap<String,String>();
+    for (Entry<String,String> e : accumuloConf) {
+      siteConfigMap.put(e.getKey(), e.getValue());
+    }
+    _setSiteConfig(siteConfigMap);
+    
+    for (Entry<String,String> entry : DefaultConfiguration.getDefaultConfiguration())
+      accumuloConf.setIfUnset(entry.getKey(), entry.getValue());
+
+    return this;
+  }
+
+  /**
+   * @return MAC should run assuming it's configured for an initialized accumulo instance
+   *
+   * @since 1.6.2
+   */
+  public boolean useExistingInstance() {
+    return existingInstance != null && existingInstance;
+  }
+
+  /**
+   * @return hadoop configuration directory being used
+   *
+   * @since 1.6.2
+   */
+  public File getHadoopConfDir() {
+    return this.hadoopConfDir;
+  }
+
+  /**
+   * @return accumulo Configuration being used
+   * 
+   * @since 1.6.2
+   */
+  public Configuration getAccumuloConfiguration() {
+    return accumuloConf;
+  }
+
+  /**
+   * @return hadoop Configuration being used
+   * 
+   * @since 1.6.2
+   */
+  public Configuration getHadoopConfiguration() {
+    return hadoopConf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
index 50dec57..904f483 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
@@ -98,5 +98,4 @@ public class ServerConfiguration {
   public Instance getInstance() {
     return scf.getInstance();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 8ddeb4f..54d7e2a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -391,11 +391,14 @@ public class VolumeManagerImpl implements VolumeManager {
   static private final String DEFAULT = "";
 
   public static VolumeManager get(AccumuloConfiguration conf) throws IOException {
+    return get(conf, CachedConfiguration.getInstance());
+  }
+
+  public static VolumeManager get(AccumuloConfiguration conf, final Configuration hadoopConf) throws IOException {
     final Map<String,Volume> volumes = new HashMap<String,Volume>();
-    final Configuration hadoopConf = CachedConfiguration.getInstance();
 
     // The "default" Volume for Accumulo (in case no volumes are specified)
-    for (String volumeUriOrDir : VolumeConfiguration.getVolumeUris(conf)) {
+    for (String volumeUriOrDir : VolumeConfiguration.getVolumeUris(conf, hadoopConf)) {
       if (volumeUriOrDir.equals(DEFAULT))
         throw new IllegalArgumentException("Cannot re-define the default volume");
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
new file mode 100644
index 0000000..7e1cc97
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.IZooReader;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.zookeeper.KeeperException;
+
+public class AccumuloStatus {
+  /**
+   * Determines if there could be an accumulo instance running via zookeeper lock checking
+   *
+   * @param reader
+   *
+   * @return true iff all servers show no indication of being registered in zookeeper, otherwise false
+   * @throws IOException
+   *           if there are issues connecting to ZooKeeper to determine service status
+   */
+  public static boolean isAccumuloOffline(IZooReader reader) throws IOException {
+    String rootPath = ZooUtil.getRoot(HdfsZooInstance.getInstance());
+    return isAccumuloOffline(reader, rootPath);
+  }
+
+  /**
+   * Determines if there could be an accumulo instance running via zookeeper lock checking
+   *
+   * @param reader
+   * @param rootPath
+   *
+   * @return true iff all servers show no indication of being registered in zookeeper, otherwise false
+   * @throws IOException
+   *           if there are issues connecting to ZooKeeper to determine service status
+   */
+  public static boolean isAccumuloOffline(IZooReader reader, String rootPath) throws IOException {
+    try {
+      for (String child : reader.getChildren(rootPath + Constants.ZTSERVERS)) {
+        if (!reader.getChildren(rootPath + Constants.ZTSERVERS + "/" + child).isEmpty())
+          return false;
+      }
+      if (!reader.getChildren(rootPath + Constants.ZTRACERS).isEmpty())
+        return false;
+      if (!reader.getChildren(rootPath + Constants.ZMASTER_LOCK).isEmpty())
+        return false;
+      if (!reader.getChildren(rootPath + Constants.ZMONITOR_LOCK).isEmpty())
+        return false;
+      if (!reader.getChildren(rootPath + Constants.ZGC_LOCK).isEmpty())
+        return false;
+    } catch (KeeperException e) {
+      throw new IOException("Issues contacting ZooKeeper to get Accumulo status.", e);
+    } catch (InterruptedException e) {
+      throw new IOException("Issues contacting ZooKeeper to get Accumulo status.", e);
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3a743cb/test/src/test/java/org/apache/accumulo/test/ExistingMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/test/java/org/apache/accumulo/test/ExistingMacIT.java
new file mode 100644
index 0000000..5d1978e
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ExistingMacIT extends ConfigurableMacIT {
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+
+    // use raw local file system so walogs sync and flush will work
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  private void createEmptyConfig(File confFile) throws IOException {
+    Configuration conf = new Configuration(false);
+    OutputStream hcOut = new FileOutputStream(confFile);
+    conf.writeXml(hcOut);
+    hcOut.close();
+  }
+
+  @Test
+  public void testExistingInstance() throws Exception {
+
+    Connector conn = getCluster().getConnector("root", ROOT_PASSWORD);
+
+    conn.tableOperations().create("table1");
+
+    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+
+    Mutation m1 = new Mutation("00081");
+    m1.put("math", "sqroot", "9");
+    m1.put("math", "sq", "6560");
+
+    bw.addMutation(m1);
+    bw.close();
+
+    conn.tableOperations().flush("table1", null, null, true);
+    // TOOD use constants
+    conn.tableOperations().flush(MetadataTable.NAME, null, null, true);
+    conn.tableOperations().flush(RootTable.NAME, null, null, true);
+
+    Set<Entry<ServerType,Collection<ProcessReference>>> procs = getCluster().getProcesses().entrySet();
+    for (Entry<ServerType,Collection<ProcessReference>> entry : procs) {
+      if (entry.getKey() == ServerType.ZOOKEEPER)
+        continue;
+      for (ProcessReference pr : entry.getValue())
+        getCluster().killProcess(entry.getKey(), pr);
+    }
+
+    // TODO clean out zookeeper? following sleep waits for ephemeral nodes to go away
+    UtilWaitThread.sleep(10000);
+
+    File hadoopConfDir = createTestDir(ExistingMacIT.class.getSimpleName() + "_hadoop_conf");
+    FileUtils.deleteQuietly(hadoopConfDir);
+    hadoopConfDir.mkdirs();
+    createEmptyConfig(new File(hadoopConfDir, "core-site.xml"));
+    createEmptyConfig(new File(hadoopConfDir, "hdfs-site.xml"));
+
+    File testDir2 = createTestDir(ExistingMacIT.class.getSimpleName() + "_2");
+    FileUtils.deleteQuietly(testDir2);
+
+    MiniAccumuloConfig macConfig2 = new MiniAccumuloConfig(testDir2, "notused");
+    macConfig2.useExistingInstance(new File(getCluster().getConfig().getConfDir(), "accumulo-site.xml"), hadoopConfDir);
+
+    MiniAccumuloCluster accumulo2 = new MiniAccumuloCluster(macConfig2);
+    accumulo2.start();
+
+    conn = accumulo2.getConnector("root", ROOT_PASSWORD);
+
+    Scanner scanner = conn.createScanner("table1", Authorizations.EMPTY);
+
+    int sum = 0;
+    for (Entry<Key,Value> entry : scanner) {
+      sum += Integer.parseInt(entry.getValue().toString());
+    }
+
+    Assert.assertEquals(6569, sum);
+
+    accumulo2.stop();
+  }
+
+  @Test
+  public void testExistingRunningInstance() throws Exception {
+    File hadoopConfDir = createTestDir(ExistingMacIT.class.getSimpleName() + "_hadoop_conf_2");
+    FileUtils.deleteQuietly(hadoopConfDir);
+    hadoopConfDir.mkdirs();
+    createEmptyConfig(new File(hadoopConfDir, "core-site.xml"));
+    createEmptyConfig(new File(hadoopConfDir, "hdfs-site.xml"));
+
+    File testDir2 = createTestDir(ExistingMacIT.class.getSimpleName() + "_3");
+    FileUtils.deleteQuietly(testDir2);
+
+    MiniAccumuloConfig macConfig2 = new MiniAccumuloConfig(testDir2, "notused");
+    macConfig2.useExistingInstance(new File(getCluster().getConfig().getConfDir(), "accumulo-site.xml"), hadoopConfDir);
+
+    System.out.println("conf " + new File(getCluster().getConfig().getConfDir(), "accumulo-site.xml"));
+
+    MiniAccumuloCluster accumulo2 = new MiniAccumuloCluster(macConfig2);
+    try {
+      accumulo2.start();
+      Assert.fail();
+    } catch (RuntimeException e) {
+      // TODO check message or throw more explicit exception
+    }
+  }
+}