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 2014/04/01 22:32:28 UTC

[2/3] git commit: ACCUMULO-2592 Create AccumuloCluster and AccumuloConfig interfaces to allow for proper non-minicluster implementations.

ACCUMULO-2592 Create AccumuloCluster and AccumuloConfig interfaces to allow for proper non-minicluster implementations.

Includes package-level javadocs in lieu of some specific annotation specifying api "experimental" status.


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

Branch: refs/heads/master
Commit: 3a1b38719c69d34abb27e181fce3bfdb7758bf92
Parents: 7ac4b54
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 26 15:04:05 2014 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Apr 1 15:45:15 2014 -0400

----------------------------------------------------------------------
 minicluster/pom.xml                             |   5 +
 .../accumulo/cluster/AccumuloCluster.java       |  73 ++++++++++
 .../accumulo/cluster/AccumuloClusters.java      |  43 ++++++
 .../apache/accumulo/cluster/AccumuloConfig.java | 140 +++++++++++++++++++
 .../apache/accumulo/cluster/package-info.java   |  27 ++++
 .../impl/MiniAccumuloClusterImpl.java           |  10 +-
 .../impl/MiniAccumuloConfigImpl.java            |  25 +++-
 .../accumulo/cluster/AccumuloClustersTest.java  |  63 +++++++++
 8 files changed, 383 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index 21890e3..0a3967d 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -92,6 +92,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
new file mode 100644
index 0000000..c982de0
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -0,0 +1,73 @@
+/*
+ * 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.cluster;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.Connector;
+
+/**
+ * Defines a minimum required set of methods to run an Accumulo cluster.
+ *
+ * (Experimental, not guaranteed to stay backwards compatible)
+ *
+ * @since 1.6.0
+ */
+
+public interface AccumuloCluster {
+  /**
+   * Starts Accumulo and Zookeeper processes. Can only be called once.
+   *
+   * @throws IllegalStateException
+   *           if already started
+   */
+  public void start() throws IOException, InterruptedException;
+
+  /**
+   * @return Accumulo instance name
+   */
+  public String getInstanceName();
+
+  /**
+   * @return zookeeper connection string
+   */
+  public String getZooKeepers();
+
+  /**
+   * 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.
+   */
+  public void stop() throws IOException, InterruptedException;
+
+  /**
+   * Get the configuration for the cluster
+   */
+  public AccumuloConfig getConfig();
+
+  /**
+   * Utility method to get a connector to the cluster.
+   */
+  public Connector getConnector(String user, String passwd) throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Get the client configuration for the cluster
+   */
+  public ClientConfiguration getClientConfig();
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloClusters.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloClusters.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloClusters.java
new file mode 100644
index 0000000..50cb9db
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloClusters.java
@@ -0,0 +1,43 @@
+/*
+ * 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.cluster;
+
+import java.io.IOException;
+
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+
+/**
+ * Facilitates programmatic creation of an {@link AccumuloCluster}
+ *
+ * (Experimental, not guaranteed to stay backwards compatible)
+ *
+ * @since 1.6.0
+ */
+public class AccumuloClusters {
+
+  private AccumuloClusters() {}
+
+  public static AccumuloCluster create(AccumuloConfig cfg) throws IOException {
+    return cfg.build();
+  }
+
+  public static MiniAccumuloCluster createMiniCluster(MiniAccumuloConfig cfg) throws IOException {
+    return new MiniAccumuloCluster(cfg);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloConfig.java
new file mode 100644
index 0000000..471bd05
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloConfig.java
@@ -0,0 +1,140 @@
+/*
+ * 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.cluster;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+
+/**
+ * Configuration class for an {@link AccumuloCluster}
+ *
+ * (Experimental, not guaranteed to stay backwards compatible)
+ *
+ * @since 1.6.0
+ */
+public interface AccumuloConfig {
+
+  /**
+   * 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 AccumuloConfig setNumTservers(int numTservers);
+
+  /**
+   * Calling this method is optional. If not set, defaults to 'miniInstance'
+   */
+  public AccumuloConfig setInstanceName(String instanceName);
+
+  /**
+   * 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.
+   */
+  public AccumuloConfig setSiteConfig(Map<String,String> siteConfig);
+
+  /**
+   * Calling this method is optional. A random port is generated by default
+   *
+   * @param zooKeeperPort
+   *          A valid (and unused) port to use for the zookeeper
+   */
+  public AccumuloConfig setZooKeeperPort(int zooKeeperPort);
+
+  /**
+   * Sets the amount of memory to use in the master process. Calling this method is optional. Default memory is 128M
+   *
+   * @param serverType
+   *          the type of server to apply the memory settings
+   * @param memory
+   *          amount of memory to set
+   *
+   * @param memoryUnit
+   *          the units for which to apply with the memory size
+   */
+  public AccumuloConfig setMemory(ServerType serverType, long memory, MemoryUnit memoryUnit);
+
+  /**
+   * Sets the default memory size to use. This value is also used when a ServerType has not been configured explicitly. Calling this method is optional. Default
+   * memory is 128M
+   *
+   * @param memory
+   *          amount of memory to set
+   *
+   * @param memoryUnit
+   *          the units for which to apply with the memory size
+   */
+  public AccumuloConfig setDefaultMemory(long memory, MemoryUnit memoryUnit);
+
+  /**
+   * @return a copy of the site config
+   */
+  public Map<String,String> getSiteConfig();
+
+  /**
+   * @return name of configured instance
+   */
+  public String getInstanceName();
+
+  /**
+   * @param serverType
+   *          get configuration for this server type
+   *
+   * @return memory configured in bytes, returns default if this server type is not configured
+   */
+  public long getMemory(ServerType serverType);
+
+  /**
+   * @return memory configured in bytes
+   */
+  public long getDefaultMemory();
+
+  /**
+   * @return the root password of this cluster configuration
+   */
+  public String getRootPassword();
+
+  /**
+   * @return the number of tservers configured for this cluster
+   */
+  public int getNumTservers();
+
+  /**
+   * @return the paths to use for loading native libraries
+   */
+  public String[] getNativeLibPaths();
+
+  /**
+   * Sets the path for processes to use for loading native libraries
+   *
+   * @param nativePathItems
+   *          the nativePathItems to set
+   */
+  public AccumuloConfig setNativeLibPaths(String... nativePathItems);
+
+  /**
+   * Build the appropriate {@link NewMiniAccumuloCluster} from this configuration
+   *
+   * @return A {@link NewMiniAccumuloCluster}
+   */
+  public AccumuloCluster build() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/src/main/java/org/apache/accumulo/cluster/package-info.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/package-info.java b/minicluster/src/main/java/org/apache/accumulo/cluster/package-info.java
new file mode 100644
index 0000000..f1b649d
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains a set of "experimental" changes to the MiniAccumuloCluster class.
+ *
+ * The intent is to decouple the notion of the MiniCluster from the implementation which
+ * will allow for various Accumulo testing implementations that all adhere to the same interface.
+ *
+ * Presently, this interface can be used to create an MiniAccumuloCluster, but the API is not
+ * guaranteed to make non-breaking changes going forward.
+ */
+package org.apache.accumulo.cluster;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/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 009988e..fbd7ef3 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
@@ -42,6 +42,7 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
+import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
@@ -83,7 +84,7 @@ import com.google.common.collect.Maps;
  * 
  * @since 1.5.0
  */
-public class MiniAccumuloClusterImpl {
+public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
   public static class LogWriter extends Daemon {
     private BufferedReader in;
@@ -418,6 +419,7 @@ public class MiniAccumuloClusterImpl {
    * @throws IllegalStateException
    *           if already started
    */
+  @Override
   public void start() throws IOException, InterruptedException {
 
     if (!initialized) {
@@ -573,6 +575,7 @@ public class MiniAccumuloClusterImpl {
   /**
    * @return Accumulo instance name
    */
+  @Override
   public String getInstanceName() {
     return config.getInstanceName();
   }
@@ -580,6 +583,7 @@ public class MiniAccumuloClusterImpl {
   /**
    * @return zookeeper connection string
    */
+  @Override
   public String getZooKeepers() {
     return config.getZooKeepers();
   }
@@ -588,6 +592,7 @@ public class MiniAccumuloClusterImpl {
    * 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.
    */
+  @Override
   public void stop() throws IOException, InterruptedException {
     for (LogWriter lw : logWriters) {
       lw.flush();
@@ -630,6 +635,7 @@ public class MiniAccumuloClusterImpl {
   /**
    * @since 1.6.0
    */
+  @Override
   public MiniAccumuloConfigImpl getConfig() {
     return config;
   }
@@ -639,11 +645,13 @@ public class MiniAccumuloClusterImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public Connector getConnector(String user, String passwd) throws AccumuloException, AccumuloSecurityException {
     Instance instance = new ZooKeeperInstance(getClientConfig());
     return instance.getConnector(user, new PasswordToken(passwd));
   }
 
+  @Override
   public ClientConfiguration getClientConfig() {
     return new ClientConfiguration(Arrays.asList(new MapConfiguration(config.getSiteConfig()))).withInstance(this.getInstanceName()).withZkHosts(
         this.getZooKeepers());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/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 897b55b..8f65786 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
@@ -17,9 +17,11 @@
 package org.apache.accumulo.minicluster.impl;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.accumulo.cluster.AccumuloConfig;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
@@ -30,7 +32,7 @@ import org.apache.accumulo.server.util.PortUtils;
  * 
  * @since 1.5.0
  */
-public class MiniAccumuloConfigImpl {
+public class MiniAccumuloConfigImpl implements AccumuloConfig {
 
   private static final String DEFAULT_INSTANCE_SECRET = "DONTTELL";
 
@@ -162,6 +164,7 @@ public class MiniAccumuloConfigImpl {
    * @param numTservers
    *          the number of tablet servers that mini accumulo cluster should start
    */
+  @Override
   public MiniAccumuloConfigImpl setNumTservers(int numTservers) {
     if (numTservers < 1)
       throw new IllegalArgumentException("Must have at least one tablet server");
@@ -174,6 +177,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public MiniAccumuloConfigImpl setInstanceName(String instanceName) {
     this.instanceName = instanceName;
     return this;
@@ -185,6 +189,7 @@ public class MiniAccumuloConfigImpl {
    * @param siteConfig
    *          key/values that you normally put in accumulo-site.xml can be put here.
    */
+  @Override
   public MiniAccumuloConfigImpl setSiteConfig(Map<String,String> siteConfig) {
     this.siteConfig = new HashMap<String,String>(siteConfig);
     this.configuredSiteConig = new HashMap<String,String>(siteConfig);
@@ -218,6 +223,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public MiniAccumuloConfigImpl setMemory(ServerType serverType, long memory, MemoryUnit memoryUnit) {
     this.memoryConfig.put(serverType, memoryUnit.toBytes(memory));
     return this;
@@ -235,6 +241,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public MiniAccumuloConfigImpl setDefaultMemory(long memory, MemoryUnit memoryUnit) {
     this.defaultMemorySize = memoryUnit.toBytes(memory);
     return this;
@@ -243,6 +250,7 @@ public class MiniAccumuloConfigImpl {
   /**
    * @return a copy of the site config
    */
+  @Override
   public Map<String,String> getSiteConfig() {
     return new HashMap<String,String>(siteConfig);
   }
@@ -256,6 +264,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public String getInstanceName() {
     return instanceName;
   }
@@ -309,6 +318,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public long getMemory(ServerType serverType) {
     return memoryConfig.containsKey(serverType) ? memoryConfig.get(serverType) : defaultMemorySize;
   }
@@ -318,6 +328,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public long getDefaultMemory() {
     return defaultMemorySize;
   }
@@ -341,6 +352,7 @@ public class MiniAccumuloConfigImpl {
   /**
    * @return the root password of this cluster configuration
    */
+  @Override
   public String getRootPassword() {
     return rootPassword;
   }
@@ -348,6 +360,7 @@ public class MiniAccumuloConfigImpl {
   /**
    * @return the number of tservers configured for this cluster
    */
+  @Override
   public int getNumTservers() {
     return numTservers;
   }
@@ -435,6 +448,7 @@ public class MiniAccumuloConfigImpl {
    * 
    * @since 1.6.0
    */
+  @Override
   public String[] getNativeLibPaths() {
     return this.nativePathItems == null ? new String[0] : this.nativePathItems;
   }
@@ -446,8 +460,10 @@ public class MiniAccumuloConfigImpl {
    *          the nativePathItems to set
    * @since 1.6.0
    */
-  public void setNativeLibPaths(String... nativePathItems) {
+  @Override
+  public MiniAccumuloConfigImpl setNativeLibPaths(String... nativePathItems) {
     this.nativePathItems = nativePathItems;
+    return this;
   }
 
   /**
@@ -458,4 +474,9 @@ public class MiniAccumuloConfigImpl {
   public void setProperty(Property p, String value) {
     this.siteConfig.put(p.getKey(), value);
   }
+
+  @Override
+  public MiniAccumuloClusterImpl build() throws IOException {
+    return new MiniAccumuloClusterImpl(this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a1b3871/minicluster/src/test/java/org/apache/accumulo/cluster/AccumuloClustersTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/cluster/AccumuloClustersTest.java b/minicluster/src/test/java/org/apache/accumulo/cluster/AccumuloClustersTest.java
new file mode 100644
index 0000000..e368240
--- /dev/null
+++ b/minicluster/src/test/java/org/apache/accumulo/cluster/AccumuloClustersTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.cluster;
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.accumulo.cluster.AccumuloClusters;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.io.Files;
+
+public class AccumuloClustersTest {
+
+  @Test
+  public void testReturnType() throws IOException {
+    MiniAccumuloConfigImpl cfg = createMock(MiniAccumuloConfigImpl.class);
+    MiniAccumuloClusterImpl cluster = createMock(MiniAccumuloClusterImpl.class);
+
+    expect(cfg.build()).andReturn(cluster);
+    replay(cfg);
+    cfg.build();
+  }
+
+  @Test
+  public void testFactoryReturn() throws IOException {
+    File dir = Files.createTempDir();
+    try {
+      MiniAccumuloConfig cfg = new MiniAccumuloConfig(dir, "foo");
+      Assert.assertEquals(MiniAccumuloCluster.class, AccumuloClusters.createMiniCluster(cfg).getClass());
+      Assert.assertTrue(FileUtils.deleteQuietly(dir));
+      Assert.assertTrue(dir.mkdirs());
+      MiniAccumuloConfigImpl cfgImpl = new MiniAccumuloConfigImpl(dir, "foo");
+      Assert.assertEquals(MiniAccumuloClusterImpl.class, AccumuloClusters.create(cfgImpl).getClass());
+    } finally {
+      FileUtils.deleteQuietly(dir);
+    }
+  }
+}