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

[12/50] [abbrv] git commit: ACCUMULO-1438 Adding Mini Module

ACCUMULO-1438 Adding Mini Module


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


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

Branch: refs/heads/master
Commit: 9060cb74555af860d9dee40d52b77b28e4291e10
Parents: 4f1ffc8
Author: Corey J. Nolet <cj...@apache.org>
Authored: Tue May 21 01:28:39 2013 +0000
Committer: Corey J. Nolet <cj...@apache.org>
Committed: Tue May 21 01:28:39 2013 +0000

----------------------------------------------------------------------
 src/mini/pom.xml                                |   1 +
 .../accumulo/mini/MiniAccumuloCluster.java      | 364 +++++++++++++++++++
 .../accumulo/mini/MiniAccumuloConfig.java       |  89 +++++
 .../accumulo/mini/MiniAccumuloClusterTest.java  |   1 +
 src/mini/src/test/resources/log4j.properties    |   1 +
 5 files changed, 456 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9060cb74/src/mini/pom.xml
----------------------------------------------------------------------
diff --git a/src/mini/pom.xml b/src/mini/pom.xml
new file mode 100644
index 0000000..a6177fa
--- /dev/null
+++ b/src/mini/pom.xml
@@ -0,0 +1 @@
+<?xml version="1.0" encoding="UTF-8"?>
<!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
  this work for additional information regarding copyright ownership.
  The ASF licenses this file to You under the Apache License, Version 2.0
  (the "License"); you may not use this file except in compliance with
  the License.  You may obtain a copy of the License at

      http://www.apache.org/licenses/LICENSE-2.0

  Unless required by applicable law or agreed to in writing, software
  distributed under the License is distributed on an "AS IS" BASIS,
  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  See the License for the specific language governing permissions and
  limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 ht
 tp://maven.apache.org/maven-v4_0_0.xsd">

    <parent>
        <groupId>org.apache.accumulo</groupId>
        <artifactId>accumulo</artifactId>
        <version>1.4.4-SNAPSHOT</version>
        <relativePath>../../</relativePath>
    </parent>

    <modelVersion>4.0.0</modelVersion>
    <artifactId>accumulo-mini</artifactId>
    <name>accumulo-mini</name>

    <build>
        <plugins>
            <plugin>
                <groupId>org.apache.maven.plugins</groupId>
                <artifactId>maven-jar-plugin</artifactId>
                <configuration>
                    <archive>
                        <manifestSections>
                            <manifestSection>
                                <name>accumulo/mini/</name>
                                <manifestEntries>
                                    <Sealed>true</Sealed>
                                </manifestEntries>
                            </manifestSection>
                        </manifestSections>
        
             </archive>
                </configuration>
            </plugin>
        </plugins>
    </build>


    <dependencies>
        <dependency>
            <groupId>org.apache.hadoop</groupId>
            <artifactId>hadoop-core</artifactId>
        </dependency>
        <dependency>
            <groupId>org.apache.hadoop</groupId>
            <artifactId>zookeeper</artifactId>
        </dependency>
        <dependency>
            <groupId>org.apache.accumulo</groupId>
            <artifactId>accumulo-core</artifactId>
        </dependency>
        <dependency>
            <groupId>org.apache.accumulo</groupId>
            <artifactId>accumulo-start</artifactId>
        </dependency>
        <dependency>
            <groupId>org.apache.accumulo</groupId>
            <artifactId>accumulo-server</artifactId>
        </dependency>
        <dependency>
            <groupId>log4j</groupId>
            <artifactId>log4j</artifactId>
        </dependency>
        <dependency>
    
         <groupId>commons-configuration</groupId>
            <artifactId>commons-configuration</artifactId>
        </dependency>
        <dependency>
            <groupId>commons-io</groupId>
            <artifactId>commons-io</artifactId>
        </dependency>
        <dependency>
            <groupId>commons-lang</groupId>
            <artifactId>commons-lang</artifactId>
        </dependency>
    </dependencies>

</project>
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9060cb74/src/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/src/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java b/src/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java
new file mode 100644
index 0000000..7472b13
--- /dev/null
+++ b/src/mini/src/main/java/org/apache/accumulo/mini/MiniAccumuloConfig.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.mini;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Holds configuration for {@link MiniAccumuloCluster}. Required configurations must be passed to constructor and all other configurations are optional.
+ * 
+ * @since 1.5.0, 1.4.4
+ */
+
+public class MiniAccumuloConfig {
+  
+  private File dir = null;
+  private String rootPassword = null;
+  private Map<String,String> siteConfig = Collections.emptyMap();
+  private int numTservers = 2;
+  
+  /**
+   * @param dir
+   *          An empty or nonexistant temp directoy that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava,
+   *          and Junit provide methods for creating temporary directories.
+   * @param rootPassword
+   *          The initial password for the Accumulo root user
+   */
+  
+  public MiniAccumuloConfig(File dir, String rootPassword) {
+    this.dir = dir;
+    this.rootPassword = rootPassword;
+  }
+  
+  public File getDir() {
+    return dir;
+  }
+  
+  public String getRootPassword() {
+    return rootPassword;
+  }
+  
+  public int getNumTservers() {
+    return numTservers;
+  }
+  
+  /**
+   * Calling this method is optional. If not set, it defaults to two.
+   * 
+   * @param numTservers
+   *          the number of tablet servers that mini accumulo cluster should start
+   */
+  
+  public MiniAccumuloConfig setNumTservers(int numTservers) {
+    if (numTservers < 1)
+      throw new IllegalArgumentException("Must have at least one tablet server");
+    this.numTservers = numTservers;
+    return this;
+  }
+  
+  public Map<String,String> getSiteConfig() {
+    return siteConfig;
+  }
+  
+  /**
+   * Calling this method is optional. If not set, it defautls to an empty map.
+   * 
+   * @param siteConfig
+   *          key/values that you normally put in accumulo-site.xml can be put here
+   */
+  public MiniAccumuloConfig setSiteConfig(Map<String,String> siteConfig) {
+    this.siteConfig = siteConfig;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9060cb74/src/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java
----------------------------------------------------------------------
diff --git a/src/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java b/src/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java
new file mode 100644
index 0000000..041eb2a
--- /dev/null
+++ b/src/mini/src/test/java/org/apache/accumulo/mini/MiniAccumuloClusterTest.java
@@ -0,0 +1 @@
+/*
 * 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.mini;

import java.util.Collections;
import java.util.Map.Entry;
import java.util.UUID;

import org.apache.accumulo.core.client.AccumuloException;
import org.apache.ac
 cumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.ZooKeeperInstance;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.LongCombiner;
import org.apache.accumulo.core.iterators.user.SummingCombiner;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.security.TablePermission;
import org.apache.accumulo.core.util.UtilWaitThread;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolde
 r;

public class MiniAccumuloClusterTest {

  private static final Logger logger = Logger.getLogger(MiniAccumuloClusterTest.class);

  private static TemporaryFolder tmpDir = new TemporaryFolder();
  private static MiniAccumuloCluster accumulo;

  @BeforeClass
  public static void setupMiniCluster() throws Exception {

    tmpDir.create();
    logger.info("MiniCluster started @ " + tmpDir.getRoot());
    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);

    accumulo = new MiniAccumuloCluster(tmpDir.getRoot(), "superSecret");
    accumulo.start();
  }

  @Test(timeout = 30000)
  public void test() throws Exception {
    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", "superSecret".getBytes());

    conn.tableOperations().create("table1");

    conn.securityOperations().createUser("user1", "pass1".getBytes(), new Authorizations("A", "B"));
    conn.securityOperations().grantTablePermission("user1", "table
 1", TablePermission.WRITE);
    conn.securityOperations().grantTablePermission("user1", "table1", TablePermission.READ);

    IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
    SummingCombiner.setEncodingType(is, LongCombiner.Type.STRING);
    SummingCombiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("META", "COUNT")));

    conn.tableOperations().attachIterator("table1", is);

    Connector uconn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("user1", "pass1".getBytes());

    BatchWriter bw = uconn.createBatchWriter("table1", 10000, 1000000, 2);

    UUID uuid = UUID.randomUUID();

    Mutation m = new Mutation(uuid.toString());
    m.put("META", "SIZE", new ColumnVisibility("A|B"), "8");
    m.put("META", "CRC", new ColumnVisibility("A|B"), "456");
    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
    m.put("DATA", "IMG", new ColumnVisibility("A&B"), "ABCDEFGH");

    bw.addMut
 ation(m);
    bw.flush();

    m = new Mutation(uuid.toString());
    m.put("META", "COUNT", new ColumnVisibility("A|B"), "1");
    m.put("META", "CRC", new ColumnVisibility("A|B"), "123");
    bw.addMutation(m);

    bw.close();

    int count = 0;
    Scanner scanner = uconn.createScanner("table1", new Authorizations("A"));
    for (Entry<Key,Value> entry : scanner) {
      if (entry.getKey().getColumnQualifierData().toString().equals("COUNT")) {
        Assert.assertEquals("2", entry.getValue().toString());
      } else if (entry.getKey().getColumnQualifierData().toString().equals("SIZE")) {
        Assert.assertEquals("8", entry.getValue().toString());
      } else if (entry.getKey().getColumnQualifierData().toString().equals("CRC")) {
        Assert.assertEquals("123", entry.getValue().toString());
      } else {
        Assert.assertTrue(false);
      }
      count++;
    }

    Assert.assertEquals(3, count);

    count = 0;
    scanner = uconn.createScanner("table1", new Auth
 orizations("A", "B"));
    for (Entry<Key,Value> entry : scanner) {
      if (entry.getKey().getColumnQualifierData().toString().equals("IMG")) {
        Assert.assertEquals("ABCDEFGH", entry.getValue().toString());
      }
      count++;
    }

    Assert.assertEquals(4, count);

    conn.tableOperations().delete("table1");
  }

  @Test(timeout = 20000)
  public void testMultipleTabletServersRunning() throws AccumuloException, AccumuloSecurityException {

    Connector conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", "superSecret".getBytes());

    while (conn.instanceOperations().getTabletServers().size() != 2) {
      UtilWaitThread.sleep(500);
    }
  }

  @AfterClass
  public static void tearDownMiniCluster() throws Exception {
    accumulo.stop();
    tmpDir.delete();
  }
}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9060cb74/src/mini/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/src/mini/src/test/resources/log4j.properties b/src/mini/src/test/resources/log4j.properties
new file mode 100644
index 0000000..c833d57
--- /dev/null
+++ b/src/mini/src/test/resources/log4j.properties
@@ -0,0 +1 @@
+log4j.rootLogger=INFO, CA
log4j.appender.CA=org.apache.log4j.ConsoleAppender
log4j.appender.CA.layout=org.apache.log4j.PatternLayout
log4j.appender.CA.layout.ConversionPattern=[%t] %-5p %c %x - %m%n

log4j.logger.org.apache.zookeeper=ERROR,CA
log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
\ No newline at end of file