You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2013/05/21 20:08:58 UTC

svn commit: r1484896 - in /accumulo/branches/1.5: ./ assemble/ minicluster/ minicluster/src/ minicluster/src/main/ minicluster/src/main/java/ minicluster/src/main/java/org/ minicluster/src/main/java/org/apache/ minicluster/src/main/java/org/apache/accu...

Author: cjnolet
Date: Tue May 21 18:08:57 2013
New Revision: 1484896

URL: http://svn.apache.org/r1484896
Log:
ACCUMULO-1438 Move MAC to it's own module and keep history

Added:
    accumulo/branches/1.5/minicluster/   (with props)
    accumulo/branches/1.5/minicluster/pom.xml   (with props)
    accumulo/branches/1.5/minicluster/src/
    accumulo/branches/1.5/minicluster/src/main/
    accumulo/branches/1.5/minicluster/src/main/java/
    accumulo/branches/1.5/minicluster/src/main/java/org/
    accumulo/branches/1.5/minicluster/src/main/java/org/apache/
    accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/
    accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/
    accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
      - copied, changed from r1484866, accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java
    accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
      - copied, changed from r1484866, accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java
    accumulo/branches/1.5/minicluster/src/test/
    accumulo/branches/1.5/minicluster/src/test/java/
    accumulo/branches/1.5/minicluster/src/test/java/org/
    accumulo/branches/1.5/minicluster/src/test/java/org/apache/
    accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/
    accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/
    accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
      - copied, changed from r1484866, accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
    accumulo/branches/1.5/minicluster/src/test/resources/
    accumulo/branches/1.5/minicluster/src/test/resources/FooFilter.jar
      - copied unchanged from r1484866, accumulo/branches/1.5/server/src/test/resources/FooFilter.jar
    accumulo/branches/1.5/minicluster/src/test/resources/log4j.properties
Removed:
    accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/
    accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java
    accumulo/branches/1.5/server/src/test/resources/FooFilter.jar
Modified:
    accumulo/branches/1.5/assemble/pom.xml
    accumulo/branches/1.5/pom.xml
    accumulo/branches/1.5/proxy/pom.xml
    accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
    accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
    accumulo/branches/1.5/server/src/test/resources/log4j.properties
    accumulo/branches/1.5/test/pom.xml
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
    accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java

Modified: accumulo/branches/1.5/assemble/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/assemble/pom.xml?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/assemble/pom.xml (original)
+++ accumulo/branches/1.5/assemble/pom.xml Tue May 21 18:08:57 2013
@@ -51,6 +51,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-minicluster</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-proxy</artifactId>
     </dependency>
     <dependency>

Propchange: accumulo/branches/1.5/minicluster/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Tue May 21 18:08:57 2013
@@ -0,0 +1,5 @@
+target
+.project
+.settings
+.classpath
+accumulo-minicluster.iml

Added: accumulo/branches/1.5/minicluster/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/minicluster/pom.xml?rev=1484896&view=auto
==============================================================================
Binary file - no diff available.

Propchange: accumulo/branches/1.5/minicluster/pom.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Copied: accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java (from r1484866, accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java)
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java?p2=accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java&p1=accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java&r1=1484866&r2=1484896&rev=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloCluster.java (original)
+++ accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java Tue May 21 18:08:57 2013
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.mini;
+package org.apache.accumulo.minicluster;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -97,8 +97,7 @@ public class MiniAccumuloCluster {
           in.close();
         }
         
-      } catch (IOException e) {
-      }
+      } catch (IOException e) {}
     }
   }
   
@@ -131,7 +130,8 @@ public class MiniAccumuloCluster {
     
     ArrayList<String> argList = new ArrayList<String>();
     
-    argList.addAll(Arrays.asList(javaBin, "-cp", classpath, "-Xmx128m", "-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75", Main.class.getName(), className));
+    argList.addAll(Arrays.asList(javaBin, "-cp", classpath, "-Xmx128m", "-XX:+UseConcMarkSweepGC", "-XX:CMSInitiatingOccupancyFraction=75",
+        Main.class.getName(), className));
     
     argList.addAll(Arrays.asList(args));
     
@@ -239,15 +239,9 @@ public class MiniAccumuloCluster {
     appendProp(fileWriter, Property.TRACE_PORT, "" + PortUtils.getRandomFreePort(), siteConfig);
     // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5
     appendProp(fileWriter, Property.TSERV_MAJC_DELAY, "3", siteConfig);
-    String cp = System.getenv("ACCUMULO_HOME")+"/lib/.*.jar,"+
-        "$ZOOKEEPER_HOME/zookeeper[^.].*.jar,"+
-        "$HADOOP_HOME/[^.].*.jar,"+
-        "$HADOOP_HOME/lib/[^.].*.jar,"+
-        "$HADOOP_PREFIX/share/hadoop/common/.*.jar," +
-        "$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar," +
-        "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar," +
-        "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar"
-        ; 
+    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);
     

Copied: accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java (from r1484866, accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java)
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java?p2=accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java&p1=accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java&r1=1484866&r2=1484896&rev=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/mini/MiniAccumuloConfig.java (original)
+++ accumulo/branches/1.5/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java Tue May 21 18:08:57 2013
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.mini;
+package org.apache.accumulo.minicluster;
 
 import java.io.File;
 import java.util.Collections;

Copied: accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java (from r1484866, accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java)
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java?p2=accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java&p1=accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java&r1=1484866&r2=1484896&rev=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/mini/MiniAccumuloClusterTest.java (original)
+++ accumulo/branches/1.5/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java Tue May 21 18:08:57 2013
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.mini;
+package org.apache.accumulo.minicluster;
 
 import java.io.File;
 import java.util.Collections;
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.iterator
 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.server.mini.MiniAccumuloCluster;
 import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;

Added: accumulo/branches/1.5/minicluster/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/minicluster/src/test/resources/log4j.properties?rev=1484896&view=auto
==============================================================================
--- accumulo/branches/1.5/minicluster/src/test/resources/log4j.properties (added)
+++ accumulo/branches/1.5/minicluster/src/test/resources/log4j.properties Tue May 21 18:08:57 2013
@@ -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.

log4j.rootLogger=INFO, CA
log4j.appender.CA=org.apache.log4j.ConsoleAppender
log4j.appender.CA.layout=org.apache.log4j.PatternLayout
log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n

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

Modified: accumulo/branches/1.5/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/pom.xml?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/pom.xml (original)
+++ accumulo/branches/1.5/pom.xml Tue May 21 18:08:57 2013
@@ -81,6 +81,7 @@
     <module>assemble</module>
     <module>proxy</module>
     <module>test</module>
+    <module>minicluster</module>
   </modules>
   <scm>
     <connection>scm:svn:http://svn.apache.org/repos/asf/accumulo/branches/1.5</connection>
@@ -219,6 +220,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.accumulo</groupId>
+        <artifactId>accumulo-minicluster</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.accumulo</groupId>
         <artifactId>accumulo-proxy</artifactId>
         <version>${project.version}</version>
       </dependency>

Modified: accumulo/branches/1.5/proxy/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/pom.xml?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/pom.xml (original)
+++ accumulo/branches/1.5/proxy/pom.xml Tue May 21 18:08:57 2013
@@ -39,6 +39,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-minicluster</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-server</artifactId>
     </dependency>
     <dependency>

Modified: accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java (original)
+++ accumulo/branches/1.5/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java Tue May 21 18:08:57 2013
@@ -24,8 +24,8 @@ import java.lang.reflect.Constructor;
 import java.util.Properties;
 
 import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -41,7 +41,7 @@ import com.google.common.io.Files;
 
 public class Proxy {
   
-  private static final Logger log = Logger.getLogger(Proxy.class); 
+  private static final Logger log = Logger.getLogger(Proxy.class);
   
   public static class PropertiesConverter implements IStringConverter<Properties> {
     @Override
@@ -110,7 +110,7 @@ public class Proxy {
         }
       });
     }
-
+    
     Class<? extends TProtocolFactory> protoFactoryClass = Class.forName(opts.prop.getProperty("protocolFactory", TCompactProtocol.Factory.class.getName()))
         .asSubclass(TProtocolFactory.class);
     int port = Integer.parseInt(opts.prop.getProperty("port"));
@@ -124,7 +124,7 @@ public class Proxy {
     
     // create the implementor
     Object impl = implementor.getConstructor(Properties.class).newInstance(properties);
-
+    
     Class<?> proxyProcClass = Class.forName(api.getName() + "$Processor");
     Class<?> proxyIfaceClass = Class.forName(api.getName() + "$Iface");
     @SuppressWarnings("unchecked")

Modified: accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java (original)
+++ accumulo/branches/1.5/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java Tue May 21 18:08:57 2013
@@ -53,6 +53,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
 import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
 import org.apache.accumulo.proxy.thrift.ActiveCompaction;
@@ -80,7 +81,6 @@ import org.apache.accumulo.proxy.thrift.
 import org.apache.accumulo.proxy.thrift.UnknownScanner;
 import org.apache.accumulo.proxy.thrift.UnknownWriter;
 import org.apache.accumulo.proxy.thrift.WriterOptions;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.commons.io.FileUtils;
@@ -617,7 +617,7 @@ public class SimpleTest {
       client.hasNext(scanner);
       fail("exception not thrown");
     } catch (UnknownScanner us) {}
-
+    
     try {
       client.closeScanner(scanner);
       fail("exception not thrown");
@@ -657,7 +657,7 @@ public class SimpleTest {
     
     // this is a oneway call, so it does not throw exceptions
     client.update(writer, mutation("row2", "cf", "cq", "value2"));
-
+    
     try {
       client.flush(writer);
       fail("exception not thrown");

Modified: accumulo/branches/1.5/server/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/test/resources/log4j.properties?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/test/resources/log4j.properties (original)
+++ accumulo/branches/1.5/server/src/test/resources/log4j.properties Tue May 21 18:08:57 2013
@@ -18,5 +18,4 @@ log4j.appender.CA=org.apache.log4j.Conso
 log4j.appender.CA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 
-log4j.logger.org.apache.accumulo.server.util.TabletIterator=ERROR
-log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
+log4j.logger.org.apache.accumulo.server.util.TabletIterator=ERROR
\ No newline at end of file

Modified: accumulo/branches/1.5/test/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/pom.xml?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/test/pom.xml (original)
+++ accumulo/branches/1.5/test/pom.xml Tue May 21 18:08:57 2013
@@ -43,6 +43,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-minicluster</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-server</artifactId>
     </dependency>
     <dependency>

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java Tue May 21 18:08:57 2013
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.accumulo.fate.zookeeper.ZooLock.AsyncLockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
@@ -38,7 +38,7 @@ import org.junit.rules.TemporaryFolder;
  * 
  */
 public class ZooLockTest {
-
+  
   public static TemporaryFolder folder = new TemporaryFolder();
   
   private static MiniAccumuloCluster accumulo;
@@ -83,7 +83,7 @@ public class ZooLockTest {
       this.notifyAll();
     }
   }
-
+  
   @BeforeClass
   public static void setupMiniCluster() throws Exception {
     
@@ -98,18 +98,17 @@ public class ZooLockTest {
   }
   
   private static int pdCount = 0;
-
+  
   @Test(timeout = 10000)
   public void testDeleteParent() throws Exception {
     accumulo.getZooKeepers();
     
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
-
     
     ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
-
+    
     Assert.assertFalse(zl.isLocked());
-
+    
     ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
     
     // intentionally created parent after lock
@@ -118,13 +117,13 @@ public class ZooLockTest {
     zk.delete(parent, -1);
     
     zk.mkdirs(parent);
-
+    
     TestALW lw = new TestALW();
     
     zl.lockAsync(lw, "test1".getBytes());
     
     lw.waitForChanges(1);
-
+    
     Assert.assertTrue(lw.locked);
     Assert.assertTrue(zl.isLocked());
     Assert.assertNull(lw.exception);
@@ -140,7 +139,7 @@ public class ZooLockTest {
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
     ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
-
+    
     Assert.assertFalse(zl.isLocked());
     
     TestALW lw = new TestALW();
@@ -148,13 +147,13 @@ public class ZooLockTest {
     zl.lockAsync(lw, "test1".getBytes());
     
     lw.waitForChanges(1);
-
+    
     Assert.assertFalse(lw.locked);
     Assert.assertFalse(zl.isLocked());
     Assert.assertNotNull(lw.exception);
     Assert.assertNull(lw.reason);
   }
-
+  
   @Test(timeout = 10000)
   public void testDeleteLock() throws Exception {
     accumulo.getZooKeepers();
@@ -165,7 +164,7 @@ public class ZooLockTest {
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
-
+    
     Assert.assertFalse(zl.isLocked());
     
     TestALW lw = new TestALW();
@@ -173,7 +172,7 @@ public class ZooLockTest {
     zl.lockAsync(lw, "test1".getBytes());
     
     lw.waitForChanges(1);
-
+    
     Assert.assertTrue(lw.locked);
     Assert.assertTrue(zl.isLocked());
     Assert.assertNull(lw.exception);
@@ -182,10 +181,10 @@ public class ZooLockTest {
     zk.delete(zl.getLockPath(), -1);
     
     lw.waitForChanges(2);
-
+    
     Assert.assertEquals(LockLossReason.LOCK_DELETED, lw.reason);
     Assert.assertNull(lw.exception);
-
+    
   }
   
   @Test(timeout = 10000)
@@ -198,7 +197,7 @@ public class ZooLockTest {
     zk.mkdirs(parent);
     
     ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
-
+    
     Assert.assertFalse(zl.isLocked());
     
     TestALW lw = new TestALW();
@@ -206,13 +205,12 @@ public class ZooLockTest {
     zl.lockAsync(lw, "test1".getBytes());
     
     lw.waitForChanges(1);
-
+    
     Assert.assertTrue(lw.locked);
     Assert.assertTrue(zl.isLocked());
     Assert.assertNull(lw.exception);
     Assert.assertNull(lw.reason);
     
-    
     ZooLock zl2 = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
     
     TestALW lw2 = new TestALW();
@@ -227,7 +225,7 @@ public class ZooLockTest {
     TestALW lw3 = new TestALW();
     
     zl3.lockAsync(lw3, "test3".getBytes());
-
+    
     List<String> children = zk.getChildren(parent);
     Collections.sort(children);
     
@@ -254,7 +252,7 @@ public class ZooLockTest {
     Assert.assertNull(lw3.reason);
     
     zl3.unlock();
-
+    
   }
   
   @Test(timeout = 10000)
@@ -280,7 +278,7 @@ public class ZooLockTest {
     zl.lockAsync(lw, "test1".getBytes());
     
     lw.waitForChanges(1);
-
+    
     Assert.assertTrue(lw.locked);
     Assert.assertTrue(zl.isLocked());
     Assert.assertNull(lw.exception);
@@ -290,20 +288,20 @@ public class ZooLockTest {
     zk.setData(zl.getLockPath(), "bar".getBytes(), -1);
     
     zk.delete(zl.getLockPath(), -1);
-
+    
     lw.waitForChanges(2);
-
+    
     Assert.assertEquals(LockLossReason.LOCK_DELETED, lw.reason);
     Assert.assertNull(lw.exception);
-
+    
   }
-
+  
   @Test(timeout = 10000)
   public void testTryLock() throws Exception {
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
     
     ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
-
+    
     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, null);
     zk.addAuthInfo("digest", "secret".getBytes());
     
@@ -313,7 +311,7 @@ public class ZooLockTest {
     }
     
     zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
+    
     TestALW lw = new TestALW();
     
     boolean ret = zl.tryLock(lw, "test1".getBytes());
@@ -329,11 +327,11 @@ public class ZooLockTest {
     
     zl.unlock();
   }
-
+  
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();
     folder.delete();
   }
-
+  
 }

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Tue May 21 18:08:57 2013
@@ -27,8 +27,8 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
-import org.apache.accumulo.server.mini.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.hadoop.io.Text;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -40,7 +40,6 @@ public class MetaSplitTest {
   private static String secret = "superSecret";
   public static TemporaryFolder folder = new TemporaryFolder();
   public static MiniAccumuloCluster cluster;
-
   
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -48,7 +47,7 @@ public class MetaSplitTest {
     MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
     cluster = new MiniAccumuloCluster(cfg);
     cluster.start();
-
+    
   }
   
   @AfterClass
@@ -57,7 +56,7 @@ public class MetaSplitTest {
     folder.delete();
   }
   
-  private void addSplits(TableOperations opts, String ... points) throws Exception {
+  private void addSplits(TableOperations opts, String... points) throws Exception {
     SortedSet<Text> splits = new TreeSet<Text>();
     for (String point : points) {
       splits.add(new Text(point));

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Tue May 21 18:08:57 2013
@@ -47,8 +47,8 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.shell.Shell;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
-import org.apache.accumulo.server.mini.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.accumulo.server.trace.TraceServer;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -117,7 +117,7 @@ public class ShellServerTest {
   static void assertGoodExit(String s, boolean stringPresent) {
     Shell.log.debug(output.get());
     assertEquals(0, shell.getExitCode());
-
+    
     if (s.length() > 0)
       assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
   }
@@ -147,12 +147,12 @@ public class ShellServerTest {
     exec("quit", true);
     shell.start();
     shell.setExit(false);
-
+    
     // use reflection to call this method so it does not need to be made public
     Method method = cluster.getClass().getDeclaredMethod("exec", Class.class, String[].class);
     method.setAccessible(true);
     traceProcess = (Process) method.invoke(cluster, TraceServer.class, new String[0]);
-
+    
     // give the tracer some time to start
     UtilWaitThread.sleep(1000);
   }
@@ -185,6 +185,7 @@ public class ShellServerTest {
     exec("deletetable -f t", true);
     exec("deletetable -f t2", true);
   }
+  
   private DistCp newDistCp() {
     try {
       @SuppressWarnings("unchecked")
@@ -204,7 +205,7 @@ public class ShellServerTest {
     }
     throw new RuntimeException("Unexpected constructors for DistCp");
   }
-
+  
   @Test(timeout = 30000)
   public void setscaniterDeletescaniter() throws Exception {
     // setscaniter, deletescaniter
@@ -524,18 +525,11 @@ public class ShellServerTest {
     exec("help -np", true, "Help Commands", true);
     shell.getReader().setInput(new ByteArrayInputStream("\n\n".getBytes()));
     exec("?", true, "Help Commands", true);
-    for (String c : (
-        "bye exit quit " +
-            "about help info ? " + 
-            "deleteiter deletescaniter listiter setiter setscaniter " +
-            "grant revoke systempermissions tablepermissions userpermissions " +
-            "execfile history " +
-            "authenticate cls clear notable sleep table user whoami " +
-            "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables " +
-            "addsplits compact constraint flush getgropus getsplits merge setgroups " +
-            "addauths createuser deleteuser dropuser getauths passwd setauths users " +
-            "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan"
-        ).split(" ")) {
+    for (String c : ("bye exit quit " + "about help info ? " + "deleteiter deletescaniter listiter setiter setscaniter "
+        + "grant revoke systempermissions tablepermissions userpermissions " + "execfile history " + "authenticate cls clear notable sleep table user whoami "
+        + "clonetable config createtable deletetable droptable du exporttable importtable offline online renametable tables "
+        + "addsplits compact constraint flush getgropus getsplits merge setgroups " + "addauths createuser deleteuser dropuser getauths passwd setauths users "
+        + "delete deletemany deleterows egrep formatter interpreter grep importdirectory insert maxrow scan").split(" ")) {
       exec("help " + c, true);
     }
   }
@@ -693,7 +687,8 @@ public class ShellServerTest {
           ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
           Connector connector = instance.getConnector("root", new PasswordToken(secret));
           Scanner s = connector.createScanner("t", Constants.NO_AUTHS);
-          for (@SuppressWarnings("unused") Entry<Key,Value> kv : s)
+          for (@SuppressWarnings("unused")
+          Entry<Key,Value> kv : s)
             ;
         } catch (Exception ex) {
           throw new RuntimeException(ex);
@@ -736,22 +731,22 @@ public class ShellServerTest {
     exec("insert foo f q v", true);
     
     UtilWaitThread.sleep(100);
-
+    
     exec("scan -np", true, "foo", false);
-
+    
     exec("constraint -a FooConstraint", true);
     
     exec("offline ptc");
     UtilWaitThread.sleep(500);
     exec("online ptc");
-
+    
     exec("table ptc", true);
     exec("insert foo f q v", false);
     exec("insert ok foo q v", true);
     
     exec("deletetable ptc", true);
     exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
-
+    
   }
   
   @Test(timeout = 30000)

Modified: accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java?rev=1484896&r1=1484895&r2=1484896&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java (original)
+++ accumulo/branches/1.5/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java Tue May 21 18:08:57 2013
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.mini.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.hadoop.io.Text;
 import org.junit.After;
 import org.junit.Before;
@@ -48,7 +48,7 @@ public class TestAccumuloSplitRecovery {
   public static TemporaryFolder folder = new TemporaryFolder();
   private MiniAccumuloCluster accumulo;
   private String secret = "secret";
-
+  
   @Before
   public void setUp() throws Exception {
     folder.create();
@@ -73,8 +73,9 @@ public class TestAccumuloSplitRecovery {
     Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
     scanner.setRange(new Range(new Text(tableId + ";"), new Text(tableId + "<")));
     scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
-    for (@SuppressWarnings("unused") Entry<Key,Value> entry : scanner) {
-        return false;
+    for (@SuppressWarnings("unused")
+    Entry<Key,Value> entry : scanner) {
+      return false;
     }
     return true;
   }
@@ -83,7 +84,7 @@ public class TestAccumuloSplitRecovery {
   public void test() throws Exception {
     
     for (int tn = 0; tn < 2; tn++) {
-    
+      
       ZooKeeperInstance instance = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers());
       Connector connector = instance.getConnector("root", new PasswordToken(secret));
       // create a table and put some data in it
@@ -143,9 +144,9 @@ public class TestAccumuloSplitRecovery {
         i++;
       }
       assertEquals(3, i);
-
+      
       connector.tableOperations().delete(TABLE);
-    
+      
     }
   }