You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/06/26 18:05:18 UTC

svn commit: r1496983 - in /accumulo/trunk: minicluster/src/main/java/org/apache/accumulo/minicluster/ test/ test/src/main/java/org/apache/accumulo/test/ test/src/main/java/org/apache/accumulo/test/functional/ test/src/test/java/org/apache/accumulo/test...

Author: ecn
Date: Wed Jun 26 16:05:18 2013
New Revision: 1496983

URL: http://svn.apache.org/r1496983
Log:
ACCUMULO-1537 start converting python functional tests to java integration tests

Added:
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java   (contents, props changed)
      - copied, changed from r1496470, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java   (contents, props changed)
      - copied, changed from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java   (with props)
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java   (with props)
Removed:
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsTest.java
    accumulo/trunk/test/system/auto/simple/deleterows.py
    accumulo/trunk/test/system/auto/simple/readwrite.py
    accumulo/trunk/test/system/auto/simple/rename.py
Modified:
    accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
    accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
    accumulo/trunk/test/pom.xml
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java

Modified: accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java (original)
+++ accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java Wed Jun 26 16:05:18 2013
@@ -122,7 +122,7 @@ public class MiniAccumuloCluster {
 
   private MiniAccumuloConfig config;
 
-  private Process exec(Class<? extends Object> clazz, String... args) throws IOException {
+  public Process exec(Class<? extends Object> clazz, String... args) throws IOException {
     return exec(clazz, Collections.singletonList("-Xmx" + config.getDefaultMemory()), args);
   }
 

Modified: accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java (original)
+++ accumulo/trunk/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java Wed Jun 26 16:05:18 2013
@@ -17,10 +17,12 @@
 package org.apache.accumulo.minicluster;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.server.util.PortUtils;
 
 /**
@@ -87,9 +89,18 @@ public class MiniAccumuloConfig {
       logDir = new File(dir, "logs");
       walogDir = new File(dir, "walogs");
       
-      String classpath = System.getenv("ACCUMULO_HOME") + "/lib/.*.jar," + "$ZOOKEEPER_HOME/zookeeper[^.].*.jar," + "$HADOOP_HOME/[^.].*.jar,"
-          + "$HADOOP_HOME/lib/[^.].*.jar," + "$HADOOP_PREFIX/share/hadoop/common/.*.jar," + "$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar,"
-          + "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar," + "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar";
+      String[] paths = {
+          "$ACCUMULO_HOME/lib/.*.jar",
+          "$ZOOKEEPER_HOME/zookeeper[^.].*.jar",
+          "$HADOOP_PREFIX/[^.].*.jar",
+          "$HADOOP_PREFIX/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 classpath = StringUtil.join(Arrays.asList(paths), ",");
       
       mergeProp(Property.INSTANCE_DFS_URI.getKey(), "file:///");
       mergeProp(Property.INSTANCE_DFS_DIR.getKey(), accumuloDir.getAbsolutePath());
@@ -109,6 +120,7 @@ public class MiniAccumuloConfig {
       mergePropWithRandomPort(Property.MASTER_CLIENTPORT.getKey());
       mergePropWithRandomPort(Property.TRACE_PORT.getKey());
       mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
+      mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
       
       // zookeeper port should be set explicitly in this class, not just on the site config
       if (zooKeeperPort == null)

Modified: accumulo/trunk/test/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/pom.xml?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/pom.xml (original)
+++ accumulo/trunk/test/pom.xml Wed Jun 26 16:05:18 2013
@@ -111,6 +111,11 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+    	<groupId>org.mortbay.jetty</groupId>
+    	<artifactId>jetty</artifactId>
+    	<scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <pluginManagement>

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java Wed Jun 26 16:05:18 2013
@@ -16,16 +16,20 @@
  */
 package org.apache.accumulo.test;
 
+import java.io.IOException;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TabletServerBatchWriter;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
@@ -59,58 +63,57 @@ import com.beust.jcommander.Parameter;
 public class TestIngest {
   public static final Authorizations AUTHS = new Authorizations("L1", "L2", "G1", "GROUP2");
   
-  static class Opts extends ClientOnDefaultTable {
+  public static class Opts extends ClientOnDefaultTable {
     
     @Parameter(names="--createTable")
-    boolean createTable = false;
+    public boolean createTable = false;
     
     @Parameter(names="--splits", description="the number of splits to use when creating the table")
-    int numsplits = 1;
+    public int numsplits = 1;
     
     @Parameter(names="--start", description="the starting row number")
-    int startRow = 0;
+    public int startRow = 0;
     
     @Parameter(names="--rows", description="the number of rows to ingest")
-    int rows = 100000;
+    public int rows = 100000;
     
     @Parameter(names="--cols", description="the number of columns to ingest per row")
-    int cols = 1;
+    public int cols = 1;
     
     @Parameter(names="--random", description="insert random rows and use the given number to seed the psuedo-random number generator")
-    Integer random = null;
+    public Integer random = null;
     
     @Parameter(names="--size", description="the size of the value to ingest")
-    int dataSize = 1000;
+    public int dataSize = 1000;
     
     @Parameter(names="--delete", description="delete values instead of inserting them")
-    boolean delete = false;
+    public boolean delete = false;
     
     @Parameter(names={"-ts", "--timestamp"}, description="timestamp to use for all values")
-    long timestamp = -1;
+    public long timestamp = -1;
     
     @Parameter(names="--rfile", description="generate data into a file that can be imported")
-    String outputFile = null;
+    public String outputFile = null;
     
     @Parameter(names="--stride", description="the difference between successive row ids")
-    int stride;
+    public int stride;
 
     @Parameter(names={"-cf","--columnFamily"}, description="place columns in this column family")
-    String columnFamily = "colf";
+    public String columnFamily = "colf";
 
     @Parameter(names={"-cv","--columnVisibility"}, description="place columns in this column family", converter=VisibilityConverter.class)
-    ColumnVisibility columnVisibility = new ColumnVisibility();
+    public ColumnVisibility columnVisibility = new ColumnVisibility();
 
-    Opts() { super("test_ingest"); }
+    public Opts() { super("test_ingest"); }
   }
   
   @SuppressWarnings("unused")
   private static final Logger log = Logger.getLogger(TestIngest.class);
   
-  public static void createTable(Opts args) throws Exception {
+  public static void createTable(Connector conn, Opts args) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     if (args.createTable) {
       TreeSet<Text> splits = getSplitPoints(args.startRow, args.startRow + args.rows, args.numsplits);
       
-      Connector conn = args.getConnector();
       if (!conn.tableOperations().exists(args.getTableName()))
         conn.tableOperations().create(args.getTableName());
       try {
@@ -136,15 +139,15 @@ public class TestIngest {
     return splits;
   }
   
-  public static byte[][] generateValues(Opts ingestArgs) {
+  public static byte[][] generateValues(int dataSize) {
     
     byte[][] bytevals = new byte[10][];
     
     byte[] letters = {'1', '2', '3', '4', '5', '6', '7', '8', '9', '0'};
     
     for (int i = 0; i < 10; i++) {
-      bytevals[i] = new byte[ingestArgs.dataSize];
-      for (int j = 0; j < ingestArgs.dataSize; j++)
+      bytevals[i] = new byte[dataSize];
+      for (int j = 0; j < dataSize; j++)
         bytevals[i][j] = letters[i];
     }
     return bytevals;
@@ -179,7 +182,6 @@ public class TestIngest {
     opts.parseArgs(TestIngest.class.getName(), args, bwOpts);
     opts.getInstance().setConfiguration(ServerConfiguration.getSiteConfiguration());
 
-    createTable(opts);
     
     Instance instance = opts.getInstance();
     
@@ -193,143 +195,149 @@ public class TestIngest {
       
       // test batch update
       
-      long stopTime;
-      
-      byte[][] bytevals = generateValues(opts);
-      
-      byte randomValue[] = new byte[opts.dataSize];
-      Random random = new Random();
-      
-      long bytesWritten = 0;
-      
-      BatchWriter bw = null;
-      FileSKVWriter writer = null;
-      
-      if (opts.outputFile != null) {
-        Configuration conf = CachedConfiguration.getInstance();
-        FileSystem fs = FileSystem.get(conf);
-        writer = FileOperations.getInstance().openWriter(opts.outputFile + "." + RFile.EXTENSION, fs, conf,
-            AccumuloConfiguration.getDefaultConfiguration());
-        writer.startDefaultLocalityGroup();
+      ingest(opts.getConnector(), opts, bwOpts);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    } finally {
+      Trace.off();
+    }
+  }
+
+  public static void ingest(Connector connector, Opts opts, BatchWriterOpts bwOpts) throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException,
+      MutationsRejectedException, TableExistsException {
+    long stopTime;
+    
+    byte[][] bytevals = generateValues(opts.dataSize);
+    
+    byte randomValue[] = new byte[opts.dataSize];
+    Random random = new Random();
+    
+    long bytesWritten = 0;
+
+    createTable(connector, opts);
+    
+    BatchWriter bw = null;
+    FileSKVWriter writer = null;
+    
+    if (opts.outputFile != null) {
+      Configuration conf = CachedConfiguration.getInstance();
+      FileSystem fs = FileSystem.get(conf);
+      writer = FileOperations.getInstance().openWriter(opts.outputFile + "." + RFile.EXTENSION, fs, conf,
+          AccumuloConfiguration.getDefaultConfiguration());
+      writer.startDefaultLocalityGroup();
+    } else {
+      bw = connector.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
+      connector.securityOperations().changeUserAuthorizations(opts.principal, AUTHS);
+    }
+    Text labBA = new Text(opts.columnVisibility.getExpression());
+    
+    long startTime = System.currentTimeMillis();
+    for (int i = 0; i < opts.rows; i++) {
+      int rowid;
+      if (opts.stride > 0) {
+        rowid = ((i % opts.stride) * (opts.rows / opts.stride)) + (i / opts.stride);
       } else {
-        Connector connector = opts.getConnector();
-        bw = connector.createBatchWriter(opts.getTableName(), bwOpts.getBatchWriterConfig());
-        connector.securityOperations().changeUserAuthorizations(opts.principal, AUTHS);
+        rowid = i;
       }
-      Text labBA = new Text(opts.columnVisibility.getExpression());
       
-      long startTime = System.currentTimeMillis();
-      for (int i = 0; i < opts.rows; i++) {
-        int rowid;
-        if (opts.stride > 0) {
-          rowid = ((i % opts.stride) * (opts.rows / opts.stride)) + (i / opts.stride);
-        } else {
-          rowid = i;
-        }
+      Text row = generateRow(rowid, opts.startRow);
+      Mutation m = new Mutation(row);
+      for (int j = 0; j < opts.cols; j++) {
+        Text colf = new Text(opts.columnFamily);
+        Text colq = new Text(FastFormat.toZeroPaddedString(j, 7, 10, COL_PREFIX));
         
-        Text row = generateRow(rowid, opts.startRow);
-        Mutation m = new Mutation(row);
-        for (int j = 0; j < opts.cols; j++) {
-          Text colf = new Text(opts.columnFamily);
-          Text colq = new Text(FastFormat.toZeroPaddedString(j, 7, 10, COL_PREFIX));
+        if (writer != null) {
+          Key key = new Key(row, colf, colq, labBA);
+          if (opts.timestamp >= 0) {
+            key.setTimestamp(opts.timestamp);
+          } else {
+            key.setTimestamp(startTime);
+          }
           
-          if (writer != null) {
-            Key key = new Key(row, colf, colq, labBA);
-            if (opts.timestamp >= 0) {
-              key.setTimestamp(opts.timestamp);
-            } else {
-              key.setTimestamp(startTime);
-            }
-            
-            if (opts.delete) {
-              key.setDeleted(true);
+          if (opts.delete) {
+            key.setDeleted(true);
+          } else {
+            key.setDeleted(false);
+          }
+          
+          bytesWritten += key.getSize();
+          
+          if (opts.delete) {
+            writer.append(key, new Value(new byte[0]));
+          } else {
+            byte value[];
+            if (opts.random != null) {
+              value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j);
             } else {
-              key.setDeleted(false);
+              value = bytevals[j % bytevals.length];
             }
             
-            bytesWritten += key.getSize();
-            
-            if (opts.delete) {
-              writer.append(key, new Value(new byte[0]));
+            Value v = new Value(value);
+            writer.append(key, v);
+            bytesWritten += v.getSize();
+          }
+          
+        } else {
+          Key key = new Key(row, colf, colq, labBA);
+          bytesWritten += key.getSize();
+          
+          if (opts.delete) {
+            if (opts.timestamp >= 0)
+              m.putDelete(colf, colq, opts.columnVisibility, opts.timestamp);
+            else
+              m.putDelete(colf, colq, opts.columnVisibility);
+          } else {
+            byte value[];
+            if (opts.random != null) {
+              value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j);
             } else {
-              byte value[];
-              if (opts.random != null) {
-                value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j);
-              } else {
-                value = bytevals[j % bytevals.length];
-              }
-              
-              Value v = new Value(value);
-              writer.append(key, v);
-              bytesWritten += v.getSize();
+              value = bytevals[j % bytevals.length];
             }
+            bytesWritten += value.length;
             
-          } else {
-            Key key = new Key(row, colf, colq, labBA);
-            bytesWritten += key.getSize();
-            
-            if (opts.delete) {
-              if (opts.timestamp >= 0)
-                m.putDelete(colf, colq, opts.columnVisibility, opts.timestamp);
-              else
-                m.putDelete(colf, colq, opts.columnVisibility);
+            if (opts.timestamp >= 0) {
+              m.put(colf, colq, opts.columnVisibility, opts.timestamp, new Value(value, true));
             } else {
-              byte value[];
-              if (opts.random != null) {
-                value = genRandomValue(random, randomValue, opts.random.intValue(), rowid + opts.startRow, j);
-              } else {
-                value = bytevals[j % bytevals.length];
-              }
-              bytesWritten += value.length;
+              m.put(colf, colq, opts.columnVisibility, new Value(value, true));
               
-              if (opts.timestamp >= 0) {
-                m.put(colf, colq, opts.columnVisibility, opts.timestamp, new Value(value, true));
-              } else {
-                m.put(colf, colq, opts.columnVisibility, new Value(value, true));
-                
-              }
             }
           }
-          
         }
-        if (bw != null)
-          bw.addMutation(m);
         
       }
+      if (bw != null)
+        bw.addMutation(m);
       
-      if (writer != null) {
-        writer.close();
-      } else if (bw != null) {
-        try {
-          bw.close();
-        } catch (MutationsRejectedException e) {
-          if (e.getAuthorizationFailuresMap().size() > 0) {
-            for (Entry<KeyExtent,Set<SecurityErrorCode>> entry : e.getAuthorizationFailuresMap().entrySet()) {
-              System.err.println("ERROR : Not authorized to write to : " + entry.getKey() + " due to " + entry.getValue());
-            }
+    }
+    
+    if (writer != null) {
+      writer.close();
+    } else if (bw != null) {
+      try {
+        bw.close();
+      } catch (MutationsRejectedException e) {
+        if (e.getAuthorizationFailuresMap().size() > 0) {
+          for (Entry<KeyExtent,Set<SecurityErrorCode>> entry : e.getAuthorizationFailuresMap().entrySet()) {
+            System.err.println("ERROR : Not authorized to write to : " + entry.getKey() + " due to " + entry.getValue());
           }
-          
-          if (e.getConstraintViolationSummaries().size() > 0) {
-            for (ConstraintViolationSummary cvs : e.getConstraintViolationSummaries()) {
-              System.err.println("ERROR : Constraint violates : " + cvs);
-            }
+        }
+        
+        if (e.getConstraintViolationSummaries().size() > 0) {
+          for (ConstraintViolationSummary cvs : e.getConstraintViolationSummaries()) {
+            System.err.println("ERROR : Constraint violates : " + cvs);
           }
-          
-          throw e;
         }
+        
+        throw e;
       }
-      
-      stopTime = System.currentTimeMillis();
-      
-      int totalValues = opts.rows * opts.cols;
-      double elapsed = (stopTime - startTime) / 1000.0;
-      
-      System.out.printf("%,12d records written | %,8d records/sec | %,12d bytes written | %,8d bytes/sec | %6.3f secs   %n", totalValues,
-          (int) (totalValues / elapsed), bytesWritten, (int) (bytesWritten / elapsed), elapsed);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    } finally {
-      Trace.off();
     }
+    
+    stopTime = System.currentTimeMillis();
+    
+    int totalValues = opts.rows * opts.cols;
+    double elapsed = (stopTime - startTime) / 1000.0;
+    
+    System.out.printf("%,12d records written | %,8d records/sec | %,12d bytes written | %,8d bytes/sec | %6.3f secs   %n", totalValues,
+        (int) (totalValues / elapsed), bytesWritten, (int) (bytesWritten / elapsed), elapsed);
   }
 }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java Wed Jun 26 16:05:18 2013
@@ -17,6 +17,7 @@
 package org.apache.accumulo.test;
 
 import java.util.ArrayList;
+import java.util.List;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
@@ -37,8 +38,6 @@ import com.beust.jcommander.Parameter;
 
 public class TestMultiTableIngest {
   
-  private static ArrayList<String> tableNames = new ArrayList<String>();
-  
   static class Opts extends ClientOpts {
     @Parameter(names="--readonly", description="read only")
     boolean readonly = false;
@@ -48,7 +47,7 @@ public class TestMultiTableIngest {
     int count = 10000;
   }
   
-  private static void readBack(Opts opts, ScannerOpts scanOpts, Connector conn) throws Exception {
+  private static void readBack(Opts opts, ScannerOpts scanOpts, Connector conn, List<String> tableNames) throws Exception {
     int i = 0;
     for (String table : tableNames) {
       Scanner scanner = conn.createScanner(table, opts.auths);
@@ -65,6 +64,8 @@ public class TestMultiTableIngest {
   }
   
   public static void main(String[] args) throws Exception {
+    ArrayList<String> tableNames = new ArrayList<String>();
+    
     Opts opts = new Opts();
     ScannerOpts scanOpts = new ScannerOpts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
@@ -106,7 +107,7 @@ public class TestMultiTableIngest {
       }
     }
     try {
-      readBack(opts, scanOpts, connector);
+      readBack(opts, scanOpts, connector, tableNames);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

Modified: accumulo/trunk/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java (original)
+++ accumulo/trunk/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java Wed Jun 26 16:05:18 2013
@@ -22,9 +22,12 @@ import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -52,10 +55,10 @@ public class VerifyIngest {
   
   public static class Opts extends TestIngest.Opts {
     @Parameter(names = "-useGet", description = "fetches values one at a time, instead of scanning")
-    boolean useGet = false;
+    public boolean useGet = false;
   }
   
-  public static void main(String[] args) {
+  public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     ScannerOpts scanOpts = new ScannerOpts();
     opts.parseArgs(VerifyIngest.class.getName(), args, scanOpts);
@@ -68,66 +71,138 @@ public class VerifyIngest {
         Trace.currentTrace().data("cmdLine", Arrays.asList(args).toString());
       }
       
-      Connector connector = opts.getConnector();
+      verifyIngest(opts.getConnector(), opts, scanOpts);
       
-      byte[][] bytevals = TestIngest.generateValues(opts);
-      
-      Authorizations labelAuths = new Authorizations("L1", "L2", "G1", "GROUP2");
-      connector.securityOperations().changeUserAuthorizations(opts.principal, labelAuths);
-      
-      int expectedRow = opts.startRow;
-      int expectedCol = 0;
-      int recsRead = 0;
-      
-      long bytesRead = 0;
-      long t1 = System.currentTimeMillis();
-      
-      byte randomValue[] = new byte[opts.dataSize];
-      Random random = new Random();
-      
-      Key endKey = new Key(new Text("row_" + String.format("%010d", opts.rows + opts.startRow)));
-      
-      int errors = 0;
-      
-      while (expectedRow < (opts.rows + opts.startRow)) {
+    } finally {
+      Trace.off();
+    }
+  }
+
+  public static void verifyIngest(Connector connector, Opts opts, ScannerOpts scanOpts) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    byte[][] bytevals = TestIngest.generateValues(opts.dataSize);
+    
+    Authorizations labelAuths = new Authorizations("L1", "L2", "G1", "GROUP2");
+    connector.securityOperations().changeUserAuthorizations(opts.principal, labelAuths);
+    
+    int expectedRow = opts.startRow;
+    int expectedCol = 0;
+    int recsRead = 0;
+    
+    long bytesRead = 0;
+    long t1 = System.currentTimeMillis();
+    
+    byte randomValue[] = new byte[opts.dataSize];
+    Random random = new Random();
+    
+    Key endKey = new Key(new Text("row_" + String.format("%010d", opts.rows + opts.startRow)));
+    
+    int errors = 0;
+    
+    while (expectedRow < (opts.rows + opts.startRow)) {
+      
+      if (opts.useGet) {
+        Text rowKey = new Text("row_" + String.format("%010d", expectedRow + opts.startRow));
+        Text colf = new Text(opts.columnFamily);
+        Text colq = new Text("col_" + String.format("%07d", expectedCol));
+        
+        Scanner scanner = connector.createScanner("test_ingest", labelAuths);
+        scanner.setBatchSize(1);
+        Key startKey = new Key(rowKey, colf, colq);
+        Range range = new Range(startKey, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL));
+        scanner.setRange(range);
+        
+        byte[] val = null; // t.get(rowKey, column);
+        
+        Iterator<Entry<Key,Value>> iter = scanner.iterator();
+        
+        if (iter.hasNext()) {
+          val = iter.next().getValue().get();
+        }
         
-        if (opts.useGet) {
-          Text rowKey = new Text("row_" + String.format("%010d", expectedRow + opts.startRow));
-          Text colf = new Text(opts.columnFamily);
-          Text colq = new Text("col_" + String.format("%07d", expectedCol));
+        byte ev[];
+        if (opts.random != null) {
+          ev = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, expectedCol);
+        } else {
+          ev = bytevals[expectedCol % bytevals.length];
+        }
+        
+        if (val == null) {
+          log.error("Did not find " + rowKey + " " + colf + " " + colq);
+          errors++;
+        } else {
+          recsRead++;
+          bytesRead += val.length;
+          Value value = new Value(val);
+          if (value.compareTo(ev) != 0) {
+            log.error("unexpected value  (" + rowKey + " " + colf + " " + colq + " : saw " + value + " expected " + new Value(ev));
+            errors++;
+          }
+        }
+        
+        expectedCol++;
+        if (expectedCol >= opts.cols) {
+          expectedCol = 0;
+          expectedRow++;
+        }
+        
+      } else {
+        
+        Key startKey = new Key(new Text("row_" + String.format("%010d", expectedRow)));
+        
+        Scanner scanner = connector.createScanner("test_ingest", labelAuths);
+        scanner.setBatchSize(scanOpts.scanBatchSize);
+        scanner.setRange(new Range(startKey, endKey));
+        for (int j = 0; j < opts.cols; j++) {
+          scanner.fetchColumn(new Text(opts.columnFamily), new Text("col_" + String.format("%07d", j)));
+        }
+        
+        int recsReadBefore = recsRead;
+        
+        for (Entry<Key,Value> entry : scanner) {
           
-          Scanner scanner = connector.createScanner("test_ingest", labelAuths);
-          scanner.setBatchSize(1);
-          Key startKey = new Key(rowKey, colf, colq);
-          Range range = new Range(startKey, startKey.followingKey(PartialKey.ROW_COLFAM_COLQUAL));
-          scanner.setRange(range);
+          recsRead++;
           
-          byte[] val = null; // t.get(rowKey, column);
+          bytesRead += entry.getKey().getLength();
+          bytesRead += entry.getValue().getSize();
           
-          Iterator<Entry<Key,Value>> iter = scanner.iterator();
+          int rowNum = getRow(entry.getKey());
+          int colNum = getCol(entry.getKey());
           
-          if (iter.hasNext()) {
-            val = iter.next().getValue().get();
+          if (rowNum != expectedRow) {
+            log.error("rowNum != expectedRow   " + rowNum + " != " + expectedRow);
+            errors++;
+            expectedRow = rowNum;
           }
           
-          byte ev[];
+          if (colNum != expectedCol) {
+            log.error("colNum != expectedCol  " + colNum + " != " + expectedCol + "  rowNum : " + rowNum);
+            errors++;
+          }
+          
+          if (expectedRow >= (opts.rows + opts.startRow)) {
+            log.error("expectedRow (" + expectedRow + ") >= (ingestArgs.rows + ingestArgs.startRow)  (" + (opts.rows + opts.startRow)
+                + "), get batch returned data passed end key");
+            errors++;
+            break;
+          }
+          
+          byte value[];
           if (opts.random != null) {
-            ev = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, expectedCol);
+            value = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, colNum);
           } else {
-            ev = bytevals[expectedCol % bytevals.length];
+            value = bytevals[colNum % bytevals.length];
           }
           
-          if (val == null) {
-            log.error("Did not find " + rowKey + " " + colf + " " + colq);
+          if (entry.getValue().compareTo(value) != 0) {
+            log.error("unexpected value, rowNum : " + rowNum + " colNum : " + colNum);
+            log.error(" saw = " + new String(entry.getValue().get()) + " expected = " + new String(value));
+            errors++;
+          }
+          
+          if (opts.timestamp >= 0 && entry.getKey().getTimestamp() != opts.timestamp) {
+            log.error("unexpected timestamp " + entry.getKey().getTimestamp() + ", rowNum : " + rowNum + " colNum : " + colNum);
             errors++;
-          } else {
-            recsRead++;
-            bytesRead += val.length;
-            Value value = new Value(val);
-            if (value.compareTo(ev) != 0) {
-              log.error("unexpected value  (" + rowKey + " " + colf + " " + colq + " : saw " + value + " expected " + new Value(ev));
-              errors++;
-            }
           }
           
           expectedCol++;
@@ -136,100 +211,27 @@ public class VerifyIngest {
             expectedRow++;
           }
           
-        } else {
-          
-          Key startKey = new Key(new Text("row_" + String.format("%010d", expectedRow)));
-          
-          Scanner scanner = connector.createScanner("test_ingest", labelAuths);
-          scanner.setBatchSize(scanOpts.scanBatchSize);
-          scanner.setRange(new Range(startKey, endKey));
-          for (int j = 0; j < opts.cols; j++) {
-            scanner.fetchColumn(new Text(opts.columnFamily), new Text("col_" + String.format("%07d", j)));
-          }
-          
-          int recsReadBefore = recsRead;
-          
-          for (Entry<Key,Value> entry : scanner) {
-            
-            recsRead++;
-            
-            bytesRead += entry.getKey().getLength();
-            bytesRead += entry.getValue().getSize();
-            
-            int rowNum = getRow(entry.getKey());
-            int colNum = getCol(entry.getKey());
-            
-            if (rowNum != expectedRow) {
-              log.error("rowNum != expectedRow   " + rowNum + " != " + expectedRow);
-              errors++;
-              expectedRow = rowNum;
-            }
-            
-            if (colNum != expectedCol) {
-              log.error("colNum != expectedCol  " + colNum + " != " + expectedCol + "  rowNum : " + rowNum);
-              errors++;
-            }
-            
-            if (expectedRow >= (opts.rows + opts.startRow)) {
-              log.error("expectedRow (" + expectedRow + ") >= (ingestArgs.rows + ingestArgs.startRow)  (" + (opts.rows + opts.startRow)
-                  + "), get batch returned data passed end key");
-              errors++;
-              break;
-            }
-            
-            byte value[];
-            if (opts.random != null) {
-              value = TestIngest.genRandomValue(random, randomValue, opts.random.intValue(), expectedRow, colNum);
-            } else {
-              value = bytevals[colNum % bytevals.length];
-            }
-            
-            if (entry.getValue().compareTo(value) != 0) {
-              log.error("unexpected value, rowNum : " + rowNum + " colNum : " + colNum);
-              log.error(" saw = " + new String(entry.getValue().get()) + " expected = " + new String(value));
-              errors++;
-            }
-            
-            if (opts.timestamp >= 0 && entry.getKey().getTimestamp() != opts.timestamp) {
-              log.error("unexpected timestamp " + entry.getKey().getTimestamp() + ", rowNum : " + rowNum + " colNum : " + colNum);
-              errors++;
-            }
-            
-            expectedCol++;
-            if (expectedCol >= opts.cols) {
-              expectedCol = 0;
-              expectedRow++;
-            }
-            
-          }
-          
-          if (recsRead == recsReadBefore) {
-            log.warn("Scan returned nothing, breaking...");
-            break;
-          }
-          
         }
+        
+        if (recsRead == recsReadBefore) {
+          log.warn("Scan returned nothing, breaking...");
+          break;
+        }
+        
       }
-      
-      long t2 = System.currentTimeMillis();
-      
-      if (errors > 0) {
-        log.error("saw " + errors + " errors ");
-        System.exit(1);
-      }
-      
-      if (expectedRow != (opts.rows + opts.startRow)) {
-        log.error("Did not read expected number of rows. Saw " + (expectedRow - opts.startRow) + " expected " + opts.rows);
-        System.exit(1);
-      } else {
-        System.out.printf("%,12d records read | %,8d records/sec | %,12d bytes read | %,8d bytes/sec | %6.3f secs   %n", recsRead,
-            (int) ((recsRead) / ((t2 - t1) / 1000.0)), bytesRead, (int) (bytesRead / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0);
-      }
-      
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    } finally {
-      Trace.off();
+    }
+    
+    long t2 = System.currentTimeMillis();
+    
+    if (errors > 0) {
+      throw new AccumuloException("saw " + errors + " errors ");
+    }
+    
+    if (expectedRow != (opts.rows + opts.startRow)) {
+      throw new AccumuloException("Did not read expected number of rows. Saw " + (expectedRow - opts.startRow) + " expected " + opts.rows);
+    } else {
+      System.out.printf("%,12d records read | %,8d records/sec | %,12d bytes read | %,8d bytes/sec | %6.3f secs   %n", recsRead,
+          (int) ((recsRead) / ((t2 - t1) / 1000.0)), bytesRead, (int) (bytesRead / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0);
     }
   }
   

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java Wed Jun 26 16:05:18 2013
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.client.C
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -69,13 +68,13 @@ public class AuditMessageTest {
   private static MiniAccumuloCluster accumulo;
   private static File logDir;
   private static List<MiniAccumuloCluster.LogWriter> logWriters;
-  private final String AUDIT_USER_1 = "AuditUser1";
-  private final String AUDIT_USER_2 = "AuditUser2";
-  private final PasswordToken PASSWORD_TOKEN = new PasswordToken("password");
-  private final String OLD_TEST_TABLE_NAME = "apples";
-  private final String NEW_TEST_TABLE_NAME = "oranges";
-  private final String THIRD_TEST_TABLE_NAME = "pears";
-  private final Authorizations auths = new Authorizations("private", "public");
+  private static final String AUDIT_USER_1 = "AuditUser1";
+  private static final String AUDIT_USER_2 = "AuditUser2";
+  private static final String PASSWORD = "password";
+  private static final String OLD_TEST_TABLE_NAME = "apples";
+  private static final String NEW_TEST_TABLE_NAME = "oranges";
+  private static final String THIRD_TEST_TABLE_NAME = "pears";
+  private static final Authorizations auths = new Authorizations("private", "public");
   private static TemporaryFolder folder = new TemporaryFolder();
   
   // Must be static to survive Junit re-initialising the class every time.
@@ -160,7 +159,7 @@ public class AuditMessageTest {
   
   @Before
   public void setup() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, IOException {
-    conn = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector("root", new PasswordToken("superSecret"));
+    conn = accumulo.getConnector("root", "superSecret");
     
     // I don't want to recreate the instance for every test since it will take ages.
     // If we run every test as non-root users, I can drop these users every test which should effectively
@@ -187,13 +186,13 @@ public class AuditMessageTest {
   public void testTableOperationsAudits() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
     
-    conn.securityOperations().createLocalUser(AUDIT_USER_1, PASSWORD_TOKEN);
+    conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.SYSTEM);
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.CREATE_TABLE);
     
     // Connect as Audit User and do a bunch of stuff.
     // Testing activity begins here
-    auditConnector = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector(AUDIT_USER_1, PASSWORD_TOKEN);
+    auditConnector = accumulo.getConnector(AUDIT_USER_1, PASSWORD);
     auditConnector.tableOperations().create(OLD_TEST_TABLE_NAME);
     auditConnector.tableOperations().rename(OLD_TEST_TABLE_NAME, NEW_TEST_TABLE_NAME);
     auditConnector.tableOperations().clone(NEW_TEST_TABLE_NAME, OLD_TEST_TABLE_NAME, true, Collections.EMPTY_MAP, Collections.EMPTY_SET);
@@ -216,15 +215,15 @@ public class AuditMessageTest {
   @Test
   public void testUserOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, InterruptedException, IOException {
     
-    conn.securityOperations().createLocalUser(AUDIT_USER_1, PASSWORD_TOKEN);
+    conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.SYSTEM);
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.CREATE_USER);
     grantEverySystemPriv(conn, AUDIT_USER_1);
     
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditConnector = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector(AUDIT_USER_1, PASSWORD_TOKEN);
-    auditConnector.securityOperations().createLocalUser(AUDIT_USER_2, PASSWORD_TOKEN);
+    auditConnector = accumulo.getConnector(AUDIT_USER_1, PASSWORD);
+    auditConnector.securityOperations().createLocalUser(AUDIT_USER_2, new PasswordToken(PASSWORD));
     
     // It seems only root can grant stuff.
     conn.securityOperations().grantSystemPermission(AUDIT_USER_2, SystemPermission.ALTER_TABLE);
@@ -266,14 +265,14 @@ public class AuditMessageTest {
   public void testImportExportOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException,
       IOException, InterruptedException {
     
-    conn.securityOperations().createLocalUser(AUDIT_USER_1, PASSWORD_TOKEN);
+    conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.SYSTEM);
     conn.securityOperations().changeUserAuthorizations(AUDIT_USER_1, auths);
     grantEverySystemPriv(conn, AUDIT_USER_1);
     
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditConnector = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector(AUDIT_USER_1, PASSWORD_TOKEN);
+    auditConnector = accumulo.getConnector(AUDIT_USER_1, PASSWORD);
     auditConnector.tableOperations().create(OLD_TEST_TABLE_NAME);
     
     // Insert some play data
@@ -346,14 +345,14 @@ public class AuditMessageTest {
   public void testDataOperationsAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException,
       InterruptedException {
     
-    conn.securityOperations().createLocalUser(AUDIT_USER_1, PASSWORD_TOKEN);
+    conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     conn.securityOperations().grantSystemPermission(AUDIT_USER_1, SystemPermission.SYSTEM);
     conn.securityOperations().changeUserAuthorizations(AUDIT_USER_1, auths);
     grantEverySystemPriv(conn, AUDIT_USER_1);
     
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditConnector = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector(AUDIT_USER_1, PASSWORD_TOKEN);
+    auditConnector = accumulo.getConnector(AUDIT_USER_1, PASSWORD);
     auditConnector.tableOperations().create(OLD_TEST_TABLE_NAME);
     
     // Insert some play data
@@ -400,9 +399,9 @@ public class AuditMessageTest {
       InterruptedException {
     
     // Create our user with no privs
-    conn.securityOperations().createLocalUser(AUDIT_USER_1, PASSWORD_TOKEN);
+    conn.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     conn.tableOperations().create(OLD_TEST_TABLE_NAME);
-    auditConnector = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()).getConnector(AUDIT_USER_1, PASSWORD_TOKEN);
+    auditConnector = accumulo.getConnector(AUDIT_USER_1, PASSWORD);
     
     // Start testing activities
     // We should get denied or / failed audit messages here.

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Wed Jun 26 16:05:18 2013
@@ -22,10 +22,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-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.core.util.MetadataTable;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.minicluster.MiniAccumuloConfig;
@@ -66,8 +63,7 @@ public class MetaSplitTest {
   
   @Test(timeout = 60000)
   public void testMetaSplit() throws Exception {
-    Instance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
-    Connector connector = instance.getConnector("root", new PasswordToken(secret));
+    Connector connector = cluster.getConnector("root", secret);
     TableOperations opts = connector.tableOperations();
     for (int i = 1; i <= 10; i++) {
       opts.create("" + i);

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Wed Jun 26 16:05:18 2013
@@ -34,8 +34,6 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -705,8 +703,7 @@ public class ShellServerTest {
       @Override
       public void run() {
         try {
-          ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
-          Connector connector = instance.getConnector("root", new PasswordToken(secret));
+          Connector connector = cluster.getConnector("root", secret);
           Scanner s = connector.createScanner("t", Authorizations.EMPTY);
           for (@SuppressWarnings("unused")
           Entry<Key,Value> kv : s)

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TableOperationsIT.java Wed Jun 26 16:05:18 2013
@@ -35,9 +35,7 @@ import org.apache.accumulo.core.client.B
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.constraints.DefaultKeySizeConstraint;
 import org.apache.accumulo.core.data.Mutation;
@@ -71,8 +69,7 @@ public class TableOperationsIT {
     
     accumuloCluster.start();
     
-    ZooKeeperInstance instance = new ZooKeeperInstance(accumuloCluster.getConfig().getInstanceName(), accumuloCluster.getConfig().getZooKeepers());
-    connector = instance.getConnector(ROOT, new PasswordToken(ROOT_PASS.getBytes()));
+    connector = accumuloCluster.getConnector(ROOT, ROOT_PASS);
   }
   
   @Test

Modified: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java?rev=1496983&r1=1496982&r2=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java Wed Jun 26 16:05:18 2013
@@ -25,8 +25,6 @@ import org.apache.accumulo.core.client.B
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
@@ -86,8 +84,7 @@ public class TestAccumuloSplitRecovery {
     
     for (int tn = 0; tn < 2; tn++) {
       
-      ZooKeeperInstance instance = new ZooKeeperInstance(accumulo.getConfig().getInstanceName(), accumulo.getConfig().getZooKeepers());
-      Connector connector = instance.getConnector("root", new PasswordToken(secret));
+      Connector connector = accumulo.getConnector("root", secret);
       // create a table and put some data in it
       connector.tableOperations().create(TABLE);
       BatchWriter bw = connector.createBatchWriter(TABLE, new BatchWriterConfig());

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java (from r1496470, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsTest.java&r1=1496470&r2=1496983&rev=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java Wed Jun 26 16:05:18 2013
@@ -16,12 +16,13 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -35,9 +36,10 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
+import org.junit.Test;
 
-public class DeleteRowsTest extends FunctionalTest {
-  private static final Logger log = Logger.getLogger(DeleteRowsTest.class);
+public class DeleteRowsIT extends MacTest {
+  private static final Logger log = Logger.getLogger(DeleteRowsIT.class);
   
   private static final int ROWS_PER_TABLET = 10;
   private static final String[] LETTERS = new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", "t",
@@ -55,21 +57,8 @@ public class DeleteRowsTest extends Func
     ROWS.add("{");
   }
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.emptyList();
-  }
-  
-  @Override
-  public void run() throws Exception {
+  @Test
+  public void test() throws Exception {
     // Delete ranges of rows, and verify the tablets are removed.
     int i = 0;
     // Eliminate whole tablets
@@ -138,20 +127,4 @@ public class DeleteRowsTest extends Func
     assertEquals(entries, count);
   }
   
-  private void assertEquals(int expected, int value) {
-    if (expected != value)
-      throw new RuntimeException("Test failed, expected " + expected + " != " + value);
-    
-  }
-  
-  private void assertTrue(boolean b) {
-    if (!b)
-      throw new RuntimeException("test failed, false value");
-  }
-  
-  private void assertEquals(String expected, String value) {
-    if (!expected.equals(value))
-      throw new RuntimeException("expected " + expected + " != " + value);
-  }
-  
 }

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java (from r1496936, accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitTest.java)
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java?p2=accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java&p1=accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitTest.java&r1=1496936&r2=1496983&rev=1496983&view=diff
==============================================================================
--- accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitTest.java (original)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java Wed Jun 26 16:05:18 2013
@@ -16,10 +16,11 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.junit.Assert.assertTrue;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.SortedSet;
@@ -35,11 +36,12 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
+import org.junit.Test;
 
 // attempt to reproduce ACCUMULO-315
-public class DeleteRowsSplitTest extends FunctionalTest {
+public class DeleteRowsSplitIT extends MacTest {
   
-  private static final Logger log = Logger.getLogger(DeleteRowsSplitTest.class);
+  private static final Logger log = Logger.getLogger(DeleteRowsSplitIT.class);
   
   private static final String LETTERS = "abcdefghijklmnopqrstuvwxyz";
   static final SortedSet<Text> SPLITS = new TreeSet<Text>();
@@ -51,26 +53,13 @@ public class DeleteRowsSplitTest extends
     }
   }
   
-  @Override
-  public void cleanup() throws Exception {}
-  
-  @Override
-  public Map<String,String> getInitialConfig() {
-    return Collections.emptyMap();
-  }
-  
-  @Override
-  public List<TableSetup> getTablesToCreate() {
-    return Collections.emptyList();
-  }
-  
   static final String TABLE;
   static {
     Random random = new Random();
     TABLE = "table" + Long.toHexString(random.nextLong());
   }
   
-  @Override
+  @Test
   public void run() throws Exception {
     // Delete ranges of rows, and verify the are removed
     // Do this while adding many splits
@@ -151,10 +140,4 @@ public class DeleteRowsSplitTest extends
     }
     bw.close();
   }
-  
-  private void assertTrue(boolean b) {
-    if (!b)
-      throw new RuntimeException("test failed, false value");
-  }
-  
 }

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java?rev=1496983&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java Wed Jun 26 16:05:18 2013
@@ -0,0 +1,53 @@
+/*
+ * 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.functional;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.rules.TemporaryFolder;
+
+public class MacTest {
+  public static final Logger log = Logger.getLogger(MacTest.class);
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  public static final String PASSWORD = "secret";
+  
+  public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
+    return cluster.getConnector("root", PASSWORD);
+  }
+  
+  @Before
+  public void setUp() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), PASSWORD);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    cluster.stop();
+//    folder.delete();
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/MacTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java?rev=1496983&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java Wed Jun 26 16:05:18 2013
@@ -0,0 +1,296 @@
+/*
+ * 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.functional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.BatchScanner;
+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.client.admin.TableOperations;
+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.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.MetadataTable;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster.LogWriter;
+import org.apache.accumulo.server.monitor.Monitor;
+import org.apache.accumulo.server.util.Admin;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.TestMultiTableIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class ReadWriteIT extends MacTest {
+  
+  static final int ROWS = 20000;
+  static final int COLS = 1;
+  static final String COLF = "colf";
+  
+  @Test(timeout=60*1000)
+  public void sunnyDay() throws Exception {
+    // Start accumulo, create a table, insert some data, verify we can read it out.
+    // Shutdown cleanly.
+    log.debug("Starting Monitor");
+    Process monitor = cluster.exec(Monitor.class);
+    Connector connector = getConnector();
+    ingest(connector, ROWS, COLS, 50, 0);
+    verify(connector, ROWS, COLS, 50, 0);
+    URL url = new URL("http://" + InetAddress.getLocalHost().getHostName() + ":" + cluster.getConfig().getSiteConfig().get(Property.MONITOR_PORT.getKey()));
+    log.debug("Fetching web page " + url);
+    String result = readAll(url.openStream());
+    assertTrue(result.length() > 100);
+    log.debug("Stopping mini accumulo cluster");
+    Process shutdown = cluster.exec(Admin.class, "stopAll");
+    shutdown.waitFor();
+    assertTrue(shutdown.exitValue() == 0);
+    log.debug("success!");
+    monitor.destroy();
+  }
+  
+  private String readAll(InputStream is) throws IOException {
+    byte[] buffer = new byte[4096];
+    StringBuffer result = new StringBuffer();
+    while (true) {
+      int n = is.read(buffer);
+      if (n <= 0)
+        break;
+      result.append(new String(buffer, 0, n));
+    }
+    return result.toString();
+  }
+  
+  public void ingest(Connector connector, int rows, int cols, int width, int offset) throws Exception {
+    ingest(connector, rows, cols, width, offset, COLF);
+  }
+  
+  public void ingest(Connector connector, int rows, int cols, int width, int offset, String colf) throws Exception {
+    TestIngest.Opts opts = new TestIngest.Opts();
+    opts.rows = rows;
+    opts.cols = cols;
+    opts.dataSize = width;
+    opts.startRow = offset;
+    opts.columnFamily = colf;
+    opts.createTable = true;
+    TestIngest.ingest(connector, opts, new BatchWriterOpts());
+  }
+  
+  private void verify(Connector connector, int rows, int cols, int width, int offset) throws Exception {
+    verify(connector, rows, cols, width, offset, COLF);
+  }
+  private void verify(Connector connector, int rows, int cols, int width, int offset, String colf) throws Exception {
+    ScannerOpts scannerOpts = new ScannerOpts();
+    VerifyIngest.Opts opts = new VerifyIngest.Opts();
+    opts.rows = rows;
+    opts.cols = cols;
+    opts.dataSize = width;
+    opts.startRow = offset;
+    opts.columnFamily = colf;
+    VerifyIngest.verifyIngest(connector, opts, scannerOpts);
+  }
+  
+  public static String[] args(String ...args) {
+    return args;
+  }
+  
+  @Test(timeout=60*1000)
+  public void multiTableTest() throws Exception {
+    // Write to multiple tables
+    String instance = cluster.getInstanceName();
+    String keepers = cluster.getZooKeepers();
+    TestMultiTableIngest.main(args("--count", "" + ROWS, "-u", "root", "-i", instance, "-z", keepers, "-p", PASSWORD));
+    TestMultiTableIngest.main(args("--count", "" + ROWS, "--readonly", "-u", "root", "-i", instance, "-z", keepers, "-p", PASSWORD));
+  }
+  
+  @Test(timeout=60*1000)
+  public void largeTest() throws Exception {
+    // write a few large values
+    Connector connector = getConnector();
+    ingest(connector, 2, 1, 500000, 0);
+    verify(connector, 2, 1, 500000, 0);
+  }
+  
+  @Test(timeout=60*1000)
+  public void interleaved() throws Exception {
+    // read and write concurrently
+    final Connector connector = getConnector();
+    final AtomicBoolean fail = new AtomicBoolean(false);
+    final int CHUNKSIZE = ROWS / 10;
+    ingest(connector, CHUNKSIZE, 1, 50, 0);
+    int i;
+    for (i = 0; i < ROWS; i += CHUNKSIZE) {
+      final int start = i;
+      Thread verify = new Thread() {
+        public void run() {
+          try {
+            verify(connector, CHUNKSIZE, 1, 50, start);
+          } catch (Exception ex) {
+            fail.set(true);
+          }
+        }
+      };
+      ingest(connector, CHUNKSIZE, 1, 50, i + CHUNKSIZE);
+      verify.join();
+      assertFalse(fail.get());
+    }
+    verify(connector, CHUNKSIZE, 1, 50, i);
+  }
+  
+  public static Text t(String s) { return new Text(s); }
+  
+  public static Mutation m(String row, String cf, String cq, String value) {
+    Mutation m = new Mutation(t(row));
+    m.put(t(cf), t(cq), new Value(value.getBytes()));
+    return m;
+  }
+  
+  
+  @Test(timeout=60*1000)
+  public void localityGroupPerf() throws Exception {
+    // verify that locality groups can make look-ups faster 
+    final Connector connector = getConnector();
+    connector.tableOperations().create("test_ingest");
+    connector.tableOperations().setProperty("test_ingest", "table.group.g1", "colf");
+    connector.tableOperations().setProperty("test_ingest", "table.groups.enabled", "g1");
+    ingest(connector, 2000, 1, 50, 0);
+    connector.tableOperations().compact("test_ingest", null, null, true, true);
+    BatchWriter bw = connector.createBatchWriter("test_ingest", new BatchWriterConfig());
+    bw.addMutation(m("zzzzzzzzzzz", "colf2", "cq", "value"));
+    bw.close();
+    long now = System.currentTimeMillis();
+    Scanner scanner = connector.createScanner("test_ingest", Authorizations.EMPTY);
+    scanner.fetchColumnFamily(new Text("colf"));
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry : scanner)
+      ;
+    long diff = System.currentTimeMillis() - now;
+    now = System.currentTimeMillis();
+    scanner = connector.createScanner("test_ingest", Authorizations.EMPTY);
+    scanner.fetchColumnFamily(new Text("colf2"));
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry : scanner)
+      ;
+    bw.close();
+    long diff2 = System.currentTimeMillis() - now;
+    assertTrue(diff2 < diff);
+  }
+  
+  @Test(timeout=60*1000)
+  public void sunnyLG() throws Exception {
+    // create a locality group, write to it and ensure it exists in the RFiles that result
+    final Connector connector = getConnector();
+    connector.tableOperations().create("test_ingest");
+    Map<String, Set<Text>> groups = new TreeMap<String, Set<Text>>();
+    groups.put("g1", Collections.singleton(t("colf")));
+    connector.tableOperations().setLocalityGroups("test_ingest", groups);
+    ingest(connector, 2000, 1, 50, 0);
+    verify(connector, 2000, 1, 50, 0);
+    connector.tableOperations().flush("test_ingest", null, null, true);
+    BatchScanner bscanner = connector.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 1);
+    String tableId = connector.tableOperations().tableIdMap().get("test_ingest");
+    bscanner.setRanges(Collections.singletonList(new Range(new Text(tableId + ";"), new Text(tableId + "<"))));
+    bscanner.fetchColumnFamily(MetadataTable.DATAFILE_COLUMN_FAMILY);
+    boolean foundFile = false;
+    for (Entry<Key,Value> entry: bscanner) {
+      foundFile = true;
+      Process info = cluster.exec(org.apache.accumulo.core.file.rfile.PrintInfo.class, entry.getKey().getColumnQualifier().toString());
+      assertEquals(0, info.waitFor());
+      for (LogWriter writer : cluster.getLogWriters())
+        writer.flush();
+      String out = readAll(new FileInputStream(cluster.getConfig().getLogDir() + "/PrintInfo_" + info.hashCode() + ".out"));
+      assertTrue(out.contains("Locality group         : g1"));
+      assertTrue(out.contains("families      : [colf]"));
+    }
+    assertTrue(foundFile);
+  }
+  
+  @Test(timeout=60*1000)
+  public void localityGroupChange() throws Exception {
+    // Make changes to locality groups and ensure nothing is lostssh
+    final Connector connector = getConnector();
+    TableOperations to = connector.tableOperations();
+    to.create("test_ingest");
+    String[] config = new String[] {
+      "lg1:colf",
+      null,
+      "lg1:colf,xyz",
+      "lg1:colf,xyz;lg2:c1,c2"
+    };
+    int i = 0;
+    for (String cfg : config) {
+      to.setLocalityGroups("test_ingest", getGroups(cfg));
+      ingest(connector, ROWS * (i+1), 1, 50, ROWS * i);
+      to.flush("test_ingest", null, null, true);
+      verify(connector, 0, 1, 50, ROWS * (i+1));
+      i++;
+    }
+    to.delete("test_ingest");
+    to.create("test_ingest");
+    config = new String[] {
+        "lg1:colf",
+        null,
+        "lg1:colf,xyz",
+        "lg1:colf;lg2:colf",
+    };
+    i = 1;
+    for (String cfg : config) {
+      ingest(connector, ROWS * i, 1, 50, 0);
+      ingest(connector, ROWS * i, 1, 50, 0, "xyz");
+      to.setLocalityGroups("test_ingest", getGroups(cfg));
+      to.flush("test_ingest", null, null, true);
+      verify(connector, ROWS * i, 1, 50, 0);
+      verify(connector, ROWS * i, 1, 50, 0, "xyz");
+      i++;
+    }
+  }
+
+  private Map<String,Set<Text>> getGroups(String cfg) {
+    Map<String, Set<Text>> groups = new TreeMap<String, Set<Text>>();
+    if (cfg != null) {
+      for (String group: cfg.split(";")) {
+        String[] parts = group.split(":");
+        Set<Text> cols = new HashSet<Text>();
+        for (String col : parts[1].split(",")) {
+          cols.add(t(col));
+        }
+        groups.put(parts[1], cols);
+      }
+    }
+    return groups;
+  }
+
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java?rev=1496983&view=auto
==============================================================================
--- accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java (added)
+++ accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java Wed Jun 26 16:05:18 2013
@@ -0,0 +1,45 @@
+/*
+ * 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.functional;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.junit.Test;
+
+public class RenameIT extends MacTest {
+  
+  @Test(timeout=60*1000)
+  public void renameTest() throws Exception {
+    BatchWriterOpts bwOpts = new BatchWriterOpts();
+    ScannerOpts scanOpts = new ScannerOpts();
+    TestIngest.Opts opts = new TestIngest.Opts();
+    opts.createTable = true;
+    Connector c = getConnector();
+    TestIngest.ingest(c, opts, bwOpts);
+    c.tableOperations().rename("test_ingest", "renamed");
+    TestIngest.ingest(c, opts, bwOpts);
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    VerifyIngest.verifyIngest(c, vopts, scanOpts);
+    c.tableOperations().delete("test_ingest");
+    c.tableOperations().rename("renamed", "test_ingest");
+    VerifyIngest.verifyIngest(c, vopts, scanOpts);
+  }
+  
+}

Propchange: accumulo/trunk/test/src/test/java/org/apache/accumulo/test/functional/RenameIT.java
------------------------------------------------------------------------------
    svn:eol-style = native