You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2013/06/24 23:34:25 UTC

svn commit: r1496226 [13/13] - in /accumulo/branches/ACCUMULO-CURATOR: ./ assemble/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf/examples/3GB/native-standalon...

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java Mon Jun 24 21:34:20 2013
@@ -63,14 +63,14 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.Stat;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.cli.ClientOnRequiredTable;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.util.MetadataTable;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -105,7 +105,7 @@ public class CollectTabletStats {
       columnsTmp = opts.columns.split(",");
     final String columns[] = columnsTmp;
     
-    final FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    final VolumeManager fs = VolumeManagerImpl.get();
 
     Instance instance = opts.getInstance();
     final ServerConfiguration sconf = new ServerConfiguration(instance);
@@ -126,10 +126,10 @@ public class CollectTabletStats {
     
     List<KeyExtent> tabletsToTest = selectRandomTablets(opts.numThreads, candidates);
     
-    Map<KeyExtent,List<String>> tabletFiles = new HashMap<KeyExtent,List<String>>();
+    Map<KeyExtent,List<FileRef>> tabletFiles = new HashMap<KeyExtent,List<FileRef>>();
     
     for (KeyExtent ke : tabletsToTest) {
-      List<String> files = getTabletFiles(CredentialHelper.create(opts.principal, opts.getToken(), opts.instance), opts.getInstance(), tableId, ke);
+      List<FileRef> files = getTabletFiles(CredentialHelper.create(opts.principal, opts.getToken(), opts.instance), opts.getInstance(), tableId, ke);
       tabletFiles.put(ke, files);
     }
     
@@ -155,7 +155,7 @@ public class CollectTabletStats {
       ArrayList<Test> tests = new ArrayList<Test>();
       
       for (final KeyExtent ke : tabletsToTest) {
-        final List<String> files = tabletFiles.get(ke);
+        final List<FileRef> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           public int runTest() throws Exception {
             return readFiles(fs, sconf.getConfiguration(), files, ke, columns);
@@ -174,7 +174,7 @@ public class CollectTabletStats {
       ArrayList<Test> tests = new ArrayList<Test>();
       
       for (final KeyExtent ke : tabletsToTest) {
-        final List<String> files = tabletFiles.get(ke);
+        final List<FileRef> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           public int runTest() throws Exception {
             return readFilesUsingIterStack(fs, sconf, files, opts.auths, ke, columns, false);
@@ -191,7 +191,7 @@ public class CollectTabletStats {
       ArrayList<Test> tests = new ArrayList<Test>();
       
       for (final KeyExtent ke : tabletsToTest) {
-        final List<String> files = tabletFiles.get(ke);
+        final List<FileRef> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           public int runTest() throws Exception {
             return readFilesUsingIterStack(fs, sconf, files, opts.auths, ke, columns, true);
@@ -372,29 +372,23 @@ public class CollectTabletStats {
     return tabletsToTest;
   }
   
-  private static List<String> getTabletFiles(TCredentials token, Instance zki, String tableId, KeyExtent ke) {
-    List<String> files = new ArrayList<String>();
-    
-    for (String cq : MetadataTable.getDataFileSizes(ke, token).keySet()) {
-      files.add(ServerConstants.getTablesDir() + "/" + tableId + cq);
-    }
-    return files;
+  private static List<FileRef> getTabletFiles(TCredentials token, Instance zki, String tableId, KeyExtent ke) throws IOException {
+    return new ArrayList<FileRef>(MetadataTable.getDataFileSizes(ke, token).keySet());
   }
   
-  private static void reportHdfsBlockLocations(List<String> files) throws Exception {
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.get(conf);
+  private static void reportHdfsBlockLocations(List<FileRef> files) throws Exception {
+    VolumeManager fs = VolumeManagerImpl.get();
     
     System.out.println("\t\tFile block report : ");
-    for (String file : files) {
-      FileStatus status = fs.getFileStatus(new Path(file));
+    for (FileRef file : files) {
+      FileStatus status = fs.getFileStatus(file.path());
       
       if (status.isDir()) {
         // assume it is a map file
         status = fs.getFileStatus(new Path(file + "/data"));
       }
-      
-      BlockLocation[] locs = fs.getFileBlockLocations(status, 0, status.getLen());
+      FileSystem ns = fs.getFileSystemByPath(file.path());
+      BlockLocation[] locs = ns.getFileBlockLocations(status, 0, status.getLen());
       
       System.out.println("\t\t\tBlocks for : " + file);
       
@@ -433,14 +427,15 @@ public class CollectTabletStats {
     return visFilter;
   }
   
-  private static int readFiles(FileSystem fs, AccumuloConfiguration aconf, List<String> files, KeyExtent ke, String[] columns) throws Exception {
+  private static int readFiles(VolumeManager fs, AccumuloConfiguration aconf, List<FileRef> files, KeyExtent ke, String[] columns) throws Exception {
     
     int count = 0;
     
     HashSet<ByteSequence> columnSet = createColumnBSS(columns);
     
-    for (String file : files) {
-      FileSKVIterator reader = FileOperations.getInstance().openReader(file, false, fs, fs.getConf(), aconf);
+    for (FileRef file : files) {
+      FileSystem ns = fs.getFileSystemByPath(file.path());
+      FileSKVIterator reader = FileOperations.getInstance().openReader(file.path().toString(), false, ns, ns.getConf(), aconf);
       Range range = new Range(ke.getPrevEndRow(), false, ke.getEndRow(), true);
       reader.seek(range, columnSet, columnSet.size() == 0 ? false : true);
       while (reader.hasTop() && !range.afterEndKey(reader.getTopKey())) {
@@ -461,7 +456,7 @@ public class CollectTabletStats {
     return columnSet;
   }
   
-  private static int readFilesUsingIterStack(FileSystem fs, ServerConfiguration aconf, List<String> files, Authorizations auths, KeyExtent ke, String[] columns,
+  private static int readFilesUsingIterStack(VolumeManager fs, ServerConfiguration aconf, List<FileRef> files, Authorizations auths, KeyExtent ke, String[] columns,
       boolean useTableIterators)
       throws Exception {
     
@@ -469,8 +464,9 @@ public class CollectTabletStats {
     
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<SortedKeyValueIterator<Key,Value>>(files.size());
     
-    for (String file : files) {
-      readers.add(FileOperations.getInstance().openReader(file, false, fs, fs.getConf(), aconf.getConfiguration()));
+    for (FileRef file : files) {
+      FileSystem ns = fs.getFileSystemByPath(file.path());
+      readers.add(FileOperations.getInstance().openReader(file.path().toString(), false, ns, ns.getConf(), aconf.getConfiguration()));
     }
     
     List<IterInfo> emptyIterinfo = Collections.emptyList();

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java Mon Jun 24 21:34:20 2013
@@ -71,7 +71,6 @@ import org.apache.thrift.TException;
 
 import com.beust.jcommander.Parameter;
 
-
 /**
  * The purpose of this class is to server as fake tserver that is a data sink like /dev/null. NullTserver modifies the !METADATA location entries for a table to
  * point to it. This allows thrift performance to be measured by running any client code that writes to a table.
@@ -98,7 +97,7 @@ public class NullTserver {
     
     @Override
     public UpdateErrors closeUpdate(TInfo tinfo, long updateID) {
-      return new UpdateErrors(new HashMap<TKeyExtent,Long>(), new ArrayList<TConstraintViolationSummary>(), new HashMap<TKeyExtent, SecurityErrorCode>());
+      return new UpdateErrors(new HashMap<TKeyExtent,Long>(), new ArrayList<TConstraintViolationSummary>(), new HashMap<TKeyExtent,SecurityErrorCode>());
     }
     
     @Override
@@ -194,15 +193,8 @@ public class NullTserver {
       
     }
     
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface#removeLogs(org.apache.accumulo.trace.thrift.TInfo,
-     * org.apache.accumulo.core.security.thrift.Credentials, java.util.List)
-     */
     @Override
-    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
-    }
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {}
     
     @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
@@ -211,13 +203,13 @@ public class NullTserver {
   }
   
   static class Opts extends Help {
-    @Parameter(names={"-i", "--instance"}, description="instance name", required=true)
+    @Parameter(names = {"-i", "--instance"}, description = "instance name", required = true)
     String iname = null;
-    @Parameter(names={"-z", "--keepers"}, description="comma-separated list of zookeeper host:ports", required=true)
+    @Parameter(names = {"-z", "--keepers"}, description = "comma-separated list of zookeeper host:ports", required = true)
     String keepers = null;
-    @Parameter(names="--table", description="table to adopt", required=true)
+    @Parameter(names = "--table", description = "table to adopt", required = true)
     String tableName = null;
-    @Parameter(names="--port", description="port number to use")
+    @Parameter(names = "--port", description = "port number to use")
     int port = DefaultConfiguration.getInstance().getPort(Property.TSERV_CLIENTPORT);
   }
   
@@ -228,7 +220,7 @@ public class NullTserver {
     TransactionWatcher watcher = new TransactionWatcher();
     ThriftClientHandler tch = new ThriftClientHandler(HdfsZooInstance.getInstance(), watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
-    TServerUtils.startTServer(opts.port, processor, "NullTServer", "null tserver", 2, 1000, 10*1024*1024);
+    TServerUtils.startTServer(opts.port, processor, "NullTServer", "null tserver", 2, 1000, 10 * 1024 * 1024);
     
     InetSocketAddress addr = new InetSocketAddress(InetAddress.getLocalHost(), opts.port);
     

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/AddSplits.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/AddSplits.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/AddSplits.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/AddSplits.java Mon Jun 24 21:34:20 2013
@@ -22,10 +22,10 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -41,7 +41,7 @@ public class AddSplits extends Test {
     @SuppressWarnings("unchecked")
     List<String> tableNames = (List<String>) state.get("tables");
     tableNames = new ArrayList<String>(tableNames);
-    tableNames.add(Constants.METADATA_TABLE_NAME);
+    tableNames.add(MetadataTable.NAME);
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     TreeSet<Text> splits = new TreeSet<Text>();

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/BatchScan.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/BatchScan.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/BatchScan.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/BatchScan.java Mon Jun 24 21:34:20 2013
@@ -23,7 +23,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
@@ -33,6 +32,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.data.Key;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -50,7 +50,7 @@ public class BatchScan extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      BatchScanner bs = conn.createBatchScanner(tableName, Constants.NO_AUTHS, 3);
+      BatchScanner bs = conn.createBatchScanner(tableName, Authorizations.EMPTY, 3);
       List<Range> ranges = new ArrayList<Range>();
       for (int i = 0; i < rand.nextInt(2000) + 1; i++)
         ranges.add(new Range(String.format("%016x", (rand.nextLong() & 0x7fffffffffffffffl))));

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckBalance.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckBalance.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckBalance.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckBalance.java Mon Jun 24 21:34:20 2013
@@ -21,10 +21,11 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -35,16 +36,13 @@ public class CheckBalance extends Test {
   
   private static final String LAST_UNBALANCED_TIME = "lastUnbalancedTime";
   private static final String UNBALANCED_COUNT = "unbalancedCount";
-
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.test.randomwalk.Node#visit(org.apache.accumulo.test.randomwalk.State, java.util.Properties)
-   */
+  
   @Override
   public void visit(State state, Properties props) throws Exception {
     log.debug("checking balance");
     Map<String,Long> counts = new HashMap<String,Long>();
-    Scanner scanner = state.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
-    scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
+    Scanner scanner = state.getConnector().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    scanner.fetchColumnFamily(MetadataTable.CURRENT_LOCATION_COLUMN_FAMILY);
     for (Entry<Key,Value> entry : scanner) {
       String location = entry.getKey().getColumnQualifier().toString();
       Long count = counts.get(location);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/IsolatedScan.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/IsolatedScan.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/IsolatedScan.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/IsolatedScan.java Mon Jun 24 21:34:20 2013
@@ -21,7 +21,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.RowIterator;
@@ -30,6 +29,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
@@ -48,7 +48,7 @@ public class IsolatedScan extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      RowIterator iter = new RowIterator(new IsolatedScanner(conn.createScanner(tableName, Constants.NO_AUTHS)));
+      RowIterator iter = new RowIterator(new IsolatedScanner(conn.createScanner(tableName, Authorizations.EMPTY)));
       
       while (iter.hasNext()) {
         PeekingIterator<Entry<Key,Value>> row = new PeekingIterator<Entry<Key,Value>>(iter.next());

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java Mon Jun 24 21:34:20 2013
@@ -21,10 +21,10 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -40,7 +40,7 @@ public class Merge extends Test {
     @SuppressWarnings("unchecked")
     List<String> tableNames = (List<String>) state.get("tables");
     tableNames = new ArrayList<String>(tableNames);
-    tableNames.add(Constants.METADATA_TABLE_NAME);
+    tableNames.add(MetadataTable.NAME);
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
         
     List<Text> range = ConcurrentFixture.generateRange(rand);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ScanTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ScanTable.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ScanTable.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ScanTable.java Mon Jun 24 21:34:20 2013
@@ -22,7 +22,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -31,6 +30,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -48,7 +48,7 @@ public class ScanTable extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      Scanner scanner = conn.createScanner(tableName, Constants.NO_AUTHS);
+      Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
       Iterator<Entry<Key,Value>> iter = scanner.iterator();
       while (iter.hasNext()) {
         iter.next();

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/image/ScanMeta.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/image/ScanMeta.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/image/ScanMeta.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/image/ScanMeta.java Mon Jun 24 21:34:20 2013
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Properties;
 import java.util.UUID;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -79,7 +78,7 @@ public class ScanMeta extends Test {
     log.debug("Found " + hashes.size() + " hashes starting at " + uuid);
     
     // use batch scanner to verify all of these exist in index
-    BatchScanner indexScanner = conn.createBatchScanner(indexTableName, Constants.NO_AUTHS, 3);
+    BatchScanner indexScanner = conn.createBatchScanner(indexTableName, Authorizations.EMPTY, 3);
     ArrayList<Range> ranges = new ArrayList<Range>();
     for (Text row : hashes.keySet()) {
       ranges.add(new Range(row));

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java Mon Jun 24 21:34:20 2013
@@ -18,13 +18,13 @@ package org.apache.accumulo.test.randomw
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
@@ -57,7 +57,7 @@ public class CopyTool extends Configured
     job.setInputFormatClass(AccumuloInputFormat.class);
     AccumuloInputFormat.setConnectorInfo(job, args[0], new PasswordToken(args[1]));
     AccumuloInputFormat.setInputTableName(job, args[2]);
-    AccumuloInputFormat.setScanAuthorizations(job, Constants.NO_AUTHS);
+    AccumuloInputFormat.setScanAuthorizations(job, Authorizations.EMPTY);
     AccumuloInputFormat.setZooKeeperInstance(job, args[3], args[4]);
     
     job.setMapperClass(SeqMapClass.class);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerify.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerify.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerify.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerify.java Mon Jun 24 21:34:20 2013
@@ -19,12 +19,12 @@ package org.apache.accumulo.test.randomw
 import java.util.Properties;
 import java.util.Map.Entry;
 
-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.data.Key;
 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.CachedConfiguration;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
@@ -50,7 +50,7 @@ public class MapRedVerify extends Test {
       return;
     }
     
-    Scanner outputScanner = state.getConnector().createScanner(args[7], Constants.NO_AUTHS);
+    Scanner outputScanner = state.getConnector().createScanner(args[7], Authorizations.EMPTY);
     outputScanner.setRange(new Range());
     
     int count = 0;

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteSomeDocs.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteSomeDocs.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteSomeDocs.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteSomeDocs.java Mon Jun 24 21:34:20 2013
@@ -21,12 +21,12 @@ import java.util.Collections;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -50,7 +50,7 @@ public class DeleteSomeDocs extends Test
     
     String pattern = patterns.get(rand.nextInt(patterns.size()));
     BatchWriterConfig bwc = new BatchWriterConfig();
-    BatchDeleter ibd = state.getConnector().createBatchDeleter(indexTableName, Constants.NO_AUTHS, 8, bwc);
+    BatchDeleter ibd = state.getConnector().createBatchDeleter(indexTableName, Authorizations.EMPTY, 8, bwc);
     ibd.setRanges(Collections.singletonList(new Range()));
     
     IteratorSetting iterSettings = new IteratorSetting(100, RegExFilter.class);
@@ -62,7 +62,7 @@ public class DeleteSomeDocs extends Test
     
     ibd.close();
     
-    BatchDeleter dbd = state.getConnector().createBatchDeleter(dataTableName, Constants.NO_AUTHS, 8, bwc);
+    BatchDeleter dbd = state.getConnector().createBatchDeleter(dataTableName, Authorizations.EMPTY, 8, bwc);
     dbd.setRanges(Collections.singletonList(new Range()));
     
     iterSettings = new IteratorSetting(100, RegExFilter.class);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteWord.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteWord.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteWord.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/DeleteWord.java Mon Jun 24 21:34:20 2013
@@ -21,7 +21,6 @@ import java.util.Properties;
 import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
@@ -29,6 +28,7 @@ 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -50,7 +50,7 @@ public class DeleteWord extends Test {
     String wordToDelete = Insert.generateRandomWord(rand);
     
     // use index to find all documents containing word
-    Scanner scanner = state.getConnector().createScanner(indexTableName, Constants.NO_AUTHS);
+    Scanner scanner = state.getConnector().createScanner(indexTableName, Authorizations.EMPTY);
     scanner.fetchColumnFamily(new Text(wordToDelete));
     
     ArrayList<Range> documentsToDelete = new ArrayList<Range>();
@@ -60,7 +60,7 @@ public class DeleteWord extends Test {
     
     if (documentsToDelete.size() > 0) {
       // use a batch scanner to fetch all documents
-      BatchScanner bscanner = state.getConnector().createBatchScanner(docTableName, Constants.NO_AUTHS, 8);
+      BatchScanner bscanner = state.getConnector().createBatchScanner(docTableName, Authorizations.EMPTY, 8);
       bscanner.setRanges(documentsToDelete);
       
       BatchWriter ibw = state.getMultiTableBatchWriter().getBatchWriter(indexTableName);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Grep.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Grep.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Grep.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Grep.java Mon Jun 24 21:34:20 2013
@@ -23,7 +23,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Key;
@@ -31,6 +30,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -52,7 +52,7 @@ public class Grep extends Test {
       words[i] = new Text(Insert.generateRandomWord(rand));
     }
     
-    BatchScanner bs = state.getConnector().createBatchScanner(indexTableName, Constants.NO_AUTHS, 16);
+    BatchScanner bs = state.getConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 16);
     IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class.getName());
     IntersectingIterator.setColumnFamilies(ii, words);
     bs.addScanIterator(ii);
@@ -66,7 +66,7 @@ public class Grep extends Test {
     
     bs.close();
     
-    bs = state.getConnector().createBatchScanner(dataTableName, Constants.NO_AUTHS, 16);
+    bs = state.getConnector().createBatchScanner(dataTableName, Authorizations.EMPTY, 16);
     
     for (int i = 0; i < words.length; i++) {
       IteratorSetting more = new IteratorSetting(20 + i, "ii" + i, RegExFilter.class);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Reindex.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Reindex.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Reindex.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Reindex.java Mon Jun 24 21:34:20 2013
@@ -20,12 +20,12 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -42,7 +42,7 @@ public class Reindex extends Test {
     
     ShardFixture.createIndexTable(this.log, state, "_tmp", rand);
     
-    Scanner scanner = state.getConnector().createScanner(docTableName, Constants.NO_AUTHS);
+    Scanner scanner = state.getConnector().createScanner(docTableName, Authorizations.EMPTY);
     BatchWriter tbw = state.getConnector().createBatchWriter(tmpIndexTableName, new BatchWriterConfig());
     
     int count = 0;

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Search.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Search.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Search.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/Search.java Mon Jun 24 21:34:20 2013
@@ -23,7 +23,6 @@ import java.util.Properties;
 import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
@@ -31,6 +30,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -68,7 +68,7 @@ public class Search extends Test {
     
     log.debug("Looking up terms " + searchTerms + " expect to find " + docID);
     
-    BatchScanner bs = state.getConnector().createBatchScanner(indexTableName, Constants.NO_AUTHS, 10);
+    BatchScanner bs = state.getConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 10);
     IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(ii, columns);
     bs.addScanIterator(ii);
@@ -90,7 +90,7 @@ public class Search extends Test {
   }
   
   static Entry<Key,Value> findRandomDocument(State state, String dataTableName, Random rand) throws Exception {
-    Scanner scanner = state.getConnector().createScanner(dataTableName, Constants.NO_AUTHS);
+    Scanner scanner = state.getConnector().createScanner(dataTableName, Authorizations.EMPTY);
     scanner.setBatchSize(1);
     scanner.setRange(new Range(Integer.toString(rand.nextInt(0xfffffff), 16), null));
     

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/VerifyIndex.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/VerifyIndex.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/VerifyIndex.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/VerifyIndex.java Mon Jun 24 21:34:20 2013
@@ -20,11 +20,11 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Properties;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -37,8 +37,8 @@ public class VerifyIndex extends Test {
     String tmpIndexTableName = indexTableName + "_tmp";
     
     // scan new and old index and verify identical
-    Scanner indexScanner1 = state.getConnector().createScanner(tmpIndexTableName, Constants.NO_AUTHS);
-    Scanner indexScanner2 = state.getConnector().createScanner(indexTableName, Constants.NO_AUTHS);
+    Scanner indexScanner1 = state.getConnector().createScanner(tmpIndexTableName, Authorizations.EMPTY);
+    Scanner indexScanner2 = state.getConnector().createScanner(indexTableName, Authorizations.EMPTY);
     
     Iterator<Entry<Key,Value>> iter = indexScanner2.iterator();
     

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/AuditMessageTest.java Mon Jun 24 21:34:20 2013
@@ -21,11 +21,22 @@ import static org.junit.Assert.assertTru
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.*;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+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.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;
@@ -66,8 +77,7 @@ public class AuditMessageTest {
   private final String THIRD_TEST_TABLE_NAME = "pears";
   private 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.
   private static String lastAuditTimestamp;
   private Connector auditConnector;
@@ -93,7 +103,6 @@ public class AuditMessageTest {
     logWriters = accumulo.getLogWriters();
   }
   
-
   /**
    * Returns a List of Audit messages that have been grep'd out of the MiniAccumuloCluster output.
    * 
@@ -118,7 +127,7 @@ public class AuditMessageTest {
         try {
           while (it.hasNext()) {
             String line = it.nextLine();
-            if (line.matches(".* \\[" + Constants.AUDITLOG + "\\s*\\].*")) {
+            if (line.matches(".* \\[" + AuditedSecurityOperation.AUDITLOG + "\\s*\\].*")) {
               // Only include the message if startTimestamp is null. or the message occurred after the startTimestamp value
               if ((lastAuditTimestamp == null) || (line.substring(0, 23).compareTo(lastAuditTimestamp) > 0))
                 result.add(line);
@@ -406,7 +415,8 @@ public class AuditMessageTest {
       auditConnector.tableOperations().rename(OLD_TEST_TABLE_NAME, NEW_TEST_TABLE_NAME);
     } catch (AccumuloSecurityException ex) {}
     try {
-      auditConnector.tableOperations().clone(OLD_TEST_TABLE_NAME, NEW_TEST_TABLE_NAME, true, Collections.<String,String>emptyMap(), Collections.<String>emptySet());
+      auditConnector.tableOperations().clone(OLD_TEST_TABLE_NAME, NEW_TEST_TABLE_NAME, true, Collections.<String,String> emptyMap(),
+          Collections.<String> emptySet());
     } catch (AccumuloSecurityException ex) {}
     try {
       auditConnector.tableOperations().delete(OLD_TEST_TABLE_NAME);
@@ -480,11 +490,11 @@ public class AuditMessageTest {
     assertEquals(1, findAuditMessage(auditMessages, String.format(AuditedSecurityOperation.CREATE_USER_AUDIT_TEMPLATE, "root", "")).size());
     
   }
-
+  
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();
-
+    
     // Comment this out to have a look at the logs, they will be in /tmp/junit*
     folder.delete();
   }

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/MetaSplitTest.java Mon Jun 24 21:34:20 2013
@@ -21,12 +21,12 @@ import static org.junit.Assert.assertEqu
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.Constants;
 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;
 import org.apache.hadoop.io.Text;
@@ -61,7 +61,7 @@ public class MetaSplitTest {
     for (String point : points) {
       splits.add(new Text(point));
     }
-    opts.addSplits(Constants.METADATA_TABLE_NAME, splits);
+    opts.addSplits(MetadataTable.NAME, splits);
   }
   
   @Test(timeout = 60000)
@@ -72,18 +72,18 @@ public class MetaSplitTest {
     for (int i = 1; i <= 10; i++) {
       opts.create("" + i);
     }
-    opts.merge(Constants.METADATA_TABLE_NAME, new Text("01"), new Text("02"));
-    assertEquals(2, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
+    opts.merge(MetadataTable.NAME, new Text("01"), new Text("02"));
+    assertEquals(1, opts.listSplits(MetadataTable.NAME).size());
     addSplits(opts, "4 5 6 7 8".split(" "));
-    assertEquals(7, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
-    opts.merge(Constants.METADATA_TABLE_NAME, new Text("6"), new Text("9"));
-    assertEquals(5, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
+    assertEquals(6, opts.listSplits(MetadataTable.NAME).size());
+    opts.merge(MetadataTable.NAME, new Text("6"), new Text("9"));
+    assertEquals(4, opts.listSplits(MetadataTable.NAME).size());
     addSplits(opts, "44 55 66 77 88".split(" "));
-    assertEquals(10, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
-    opts.merge(Constants.METADATA_TABLE_NAME, new Text("5"), new Text("7"));
-    assertEquals(7, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
-    opts.merge(Constants.METADATA_TABLE_NAME, null, null);
-    assertEquals(1, opts.listSplits(Constants.METADATA_TABLE_NAME).size());
+    assertEquals(9, opts.listSplits(MetadataTable.NAME).size());
+    opts.merge(MetadataTable.NAME, new Text("5"), new Text("7"));
+    assertEquals(6, opts.listSplits(MetadataTable.NAME).size());
+    opts.merge(MetadataTable.NAME, null, null);
+    assertEquals(0, opts.listSplits(MetadataTable.NAME).size());
   }
   
 }

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/ShellServerTest.java Mon Jun 24 21:34:20 2013
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
@@ -75,7 +76,7 @@ public class ShellServerTest {
       sb.setLength(0);
     }
   }
-
+  
   public static class StringInputStream extends InputStream {
     private String source = "";
     private int offset = 0;
@@ -152,9 +153,9 @@ public class ShellServerTest {
     MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
     cluster = new MiniAccumuloCluster(cfg);
     cluster.start();
-
+    
     System.setProperty("HOME", folder.getRoot().getAbsolutePath());
-
+    
     // start the shell
     output = new TestOutputStream();
     input = new StringInputStream();
@@ -173,14 +174,14 @@ public class ShellServerTest {
     // give the tracer some time to start
     UtilWaitThread.sleep(1000);
   }
-
+  
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     cluster.stop();
     traceProcess.destroy();
     folder.delete();
   }
-
+  
   @Test(timeout = 30000)
   public void exporttableImporttable() throws Exception {
     // exporttable / importtable
@@ -189,10 +190,10 @@ public class ShellServerTest {
     exec("addsplits row5", true);
     exec("config -t t -s table.split.threshold=345M", true);
     exec("offline t", true);
-    String export = folder.newFolder().toString();
+    String export = "file://" + folder.newFolder().toString();
     exec("exporttable -t t " + export, true);
     DistCp cp = newDistCp();
-    String import_ = folder.newFolder().toString();
+    String import_ = "file://" + folder.newFolder().toString();
     cp.run(new String[] {"-f", export + "/distcp.txt", import_});
     exec("importtable t2 " + import_, true);
     exec("config -t t2 -np", true, "345M", true);
@@ -249,7 +250,7 @@ public class ShellServerTest {
     exec("execfile " + file.getAbsolutePath(), true, Constants.VERSION, true);
     
   }
-
+  
   @Test(timeout = 30000)
   public void egrep() throws Exception {
     // egrep
@@ -307,7 +308,7 @@ public class ShellServerTest {
     exec("scan", true, "row1", true);
     exec("droptable -f t", true);
     exec("deleteuser xyzzy", false, "delete yourself", true);
-    input.set(secret+"\n"+secret+"\n");
+    input.set(secret + "\n" + secret + "\n");
     exec("user root", true);
     exec("revoke -u xyzzy -s System.CREATE_TABLE", true);
     exec("revoke -u xyzzy -s System.GOOFY", false);
@@ -537,7 +538,8 @@ public class ShellServerTest {
     exec("deletetable -f t", true);
   }
   
-  @Test//(timeout = 30000)
+  @Test
+  // (timeout = 30000)
   public void help() throws Exception {
     exec("help -np", true, "Help Commands", true);
     exec("?", true, "Help Commands", true);
@@ -650,10 +652,12 @@ public class ShellServerTest {
     exec("getsplits", true, "z", false);
     exec("deletetable -f t");
     exec("getsplits -t !METADATA", true);
-    assertEquals(3, output.get().split("\n").length);
+    assertEquals(2, output.get().split("\n").length);
+    exec("getsplits -t !!ROOT", true);
+    assertEquals(1, output.get().split("\n").length);
     exec("merge --all -t !METADATA");
     exec("getsplits -t !METADATA", true);
-    assertEquals(2, output.get().split("\n").length);
+    assertEquals(1, output.get().split("\n").length);
   }
   
   @Test(timeout = 30000)
@@ -698,11 +702,12 @@ public class ShellServerTest {
     exec("insert c cf cq value", true);
     exec("insert d cf cq value", true);
     Thread thread = new Thread() {
+      @Override
       public void run() {
         try {
           ZooKeeperInstance instance = new ZooKeeperInstance(cluster.getConfig().getInstanceName(), cluster.getConfig().getZooKeepers());
           Connector connector = instance.getConnector("root", new PasswordToken(secret));
-          Scanner s = connector.createScanner("t", Constants.NO_AUTHS);
+          Scanner s = connector.createScanner("t", Authorizations.EMPTY);
           for (@SuppressWarnings("unused")
           Entry<Key,Value> kv : s)
             ;

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/TestAccumuloSplitRecovery.java Mon Jun 24 21:34:20 2013
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertEqu
 
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -33,7 +32,9 @@ 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.security.TablePermission;
+import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.hadoop.io.Text;
@@ -70,9 +71,9 @@ public class TestAccumuloSplitRecovery {
   
   boolean isOffline(String tablename, Connector connector) throws TableNotFoundException {
     String tableId = connector.tableOperations().tableIdMap().get(tablename);
-    Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     scanner.setRange(new Range(new Text(tableId + ";"), new Text(tableId + "<")));
-    scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
+    scanner.fetchColumnFamily(MetadataTable.CURRENT_LOCATION_COLUMN_FAMILY);
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {
       return false;
@@ -100,32 +101,32 @@ public class TestAccumuloSplitRecovery {
         UtilWaitThread.sleep(200);
       
       // poke a partial split into the !METADATA table
-      connector.securityOperations().grantTablePermission("root", Constants.METADATA_TABLE_NAME, TablePermission.WRITE);
+      connector.securityOperations().grantTablePermission("root", MetadataTable.NAME, TablePermission.WRITE);
       String tableId = connector.tableOperations().tableIdMap().get(TABLE);
       
       KeyExtent extent = new KeyExtent(new Text(tableId), null, new Text("b"));
       Mutation m = extent.getPrevRowUpdateMutation();
       
-      Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(0.5).getBytes()));
-      Constants.METADATA_OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(null));
-      bw = connector.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
+      MetadataTable.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(0.5).getBytes()));
+      MetadataTable.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(null));
+      bw = connector.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
       bw.addMutation(m);
       
       if (tn == 1) {
         
         bw.flush();
         
-        Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+        Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
         scanner.setRange(extent.toMetadataRange());
-        scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
+        scanner.fetchColumnFamily(MetadataTable.DATAFILE_COLUMN_FAMILY);
         
         KeyExtent extent2 = new KeyExtent(new Text(tableId), new Text("b"), null);
         m = extent2.getPrevRowUpdateMutation();
-        Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value("/t2".getBytes()));
-        Constants.METADATA_TIME_COLUMN.put(m, new Value("M0".getBytes()));
+        MetadataTable.DIRECTORY_COLUMN.put(m, new Value("/t2".getBytes()));
+        MetadataTable.TIME_COLUMN.put(m, new Value("M0".getBytes()));
         
         for (Entry<Key,Value> entry : scanner) {
-          m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, entry.getKey().getColumnQualifier(), entry.getValue());
+          m.put(MetadataTable.DATAFILE_COLUMN_FAMILY, entry.getKey().getColumnQualifier(), entry.getValue());
         }
         
         bw.addMutation(m);
@@ -136,7 +137,7 @@ public class TestAccumuloSplitRecovery {
       connector.tableOperations().online(TABLE);
       
       // verify the tablets went online
-      Scanner scanner = connector.createScanner(TABLE, Constants.NO_AUTHS);
+      Scanner scanner = connector.createScanner(TABLE, Authorizations.EMPTY);
       int i = 0;
       String expected[] = {"a", "b", "c"};
       for (Entry<Key,Value> entry : scanner) {

Modified: accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java Mon Jun 24 21:34:20 2013
@@ -20,7 +20,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -36,6 +35,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -124,12 +124,12 @@ public class RegExTest {
   
   private void runTest(Range range, String rowRegEx, String cfRegEx, String cqRegEx, String valRegEx, int expected) throws Exception {
     
-    Scanner s = conn.createScanner("ret", Constants.NO_AUTHS);
+    Scanner s = conn.createScanner("ret", Authorizations.EMPTY);
     s.setRange(range);
     setRegexs(s, rowRegEx, cfRegEx, cqRegEx, valRegEx);
     runTest(s, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);
     
-    BatchScanner bs = conn.createBatchScanner("ret", Constants.NO_AUTHS, 1);
+    BatchScanner bs = conn.createBatchScanner("ret", Authorizations.EMPTY, 1);
     bs.setRanges(Collections.singletonList(range));
     setRegexs(bs, rowRegEx, cfRegEx, cqRegEx, valRegEx);
     runTest(bs, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);

Modified: accumulo/branches/ACCUMULO-CURATOR/test/system/auto/TestUtils.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/system/auto/TestUtils.py?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/system/auto/TestUtils.py (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/system/auto/TestUtils.py Mon Jun 24 21:34:20 2013
@@ -457,6 +457,8 @@ class TestUtilsMixin:
         out,err = handle.communicate()
         self.assert_(handle.returncode==0)
         for line in out.split('\n') :
+            if line.find("=>") < 0:
+                continue
             left, right = line.split("=>")
             left = left.strip()
             right = right.strip()

Modified: accumulo/branches/ACCUMULO-CURATOR/test/system/auto/run.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/system/auto/run.py?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/system/auto/run.py (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/system/auto/run.py Mon Jun 24 21:34:20 2013
@@ -242,6 +242,8 @@ def main():
                       help='Do not clean up at the end of the test.')
     parser.add_option('-s', '--start', dest='start', default=None, 
                       help='Start the test list at the given test name')
+    parser.add_option('-x', '--xml', dest='xmlreport', default=False, action='store_true',
+                      help='Output tests results to xml (jenkins conpatible)')
     
     allTests = getTests()
     options = parseArguments(parser, allTests)
@@ -269,7 +271,12 @@ def main():
        print "ZOOKEEPER_HOME needs to be set"
        sys.exit(1)
 
-    runner = TestRunner()
+    if options.xmlreport:
+        import xmlrunner
+        runner = xmlrunner.XMLTestRunner(output='test-reports')
+    else:    
+        runner = TestRunner()
+
     
     suite = unittest.TestSuite()
     map(suite.addTest, filtered)

Modified: accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/examples.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/examples.py?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/examples.py (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/examples.py Mon Jun 24 21:34:20 2013
@@ -78,14 +78,14 @@ class Examples(TestUtilsMixin, unittest.
                      '--dataTable', 'dataTable',
                      '--vis', visibility,
                      '--chunkSize', 100000,
-                     ACCUMULO_HOME+"/fate")
+                     ACCUMULO_HOME+"/test")
         self.comment("  searching for a file")
         handle = self.runOn('localhost', [self.accumulo_sh(), 'org.apache.accumulo.examples.simple.dirlist.QueryUtil',
                                           '-i', INSTANCE_NAME, '-z', ZOOKEEPERS, '-u', ROOT, '-p', ROOT_PASSWORD,
-                                          '-t', 'indexTable', '--auths', auths, '--search', '--path', 'Fate.java'])
+                                          '-t', 'indexTable', '--auths', auths, '--search', '--path', 'examples.py'])
         out, err = handle.communicate()
         self.assert_(handle.returncode == 0)
-        self.assert_(out.find('accumulo/fate/Fate.java') >= 0)
+        self.assert_(out.find('test/system/auto/simple/examples.py') >= 0)
         self.comment("  found file at " + out)
 
     

Modified: accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/readwrite.py
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/readwrite.py?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/readwrite.py (original)
+++ accumulo/branches/ACCUMULO-CURATOR/test/system/auto/simple/readwrite.py Mon Jun 24 21:34:20 2013
@@ -157,7 +157,7 @@ class SunnyLG(SunnyDayTest):
         handle = self.runOn(self.masterHost(),
                             [self.accumulo_sh(),
                              'org.apache.accumulo.core.file.rfile.PrintInfo',
-                             dir + '/tables/1/default_tablet/F0000000.rf'])
+                             dir + '/tables/1/default_tablet/F0000001.rf'])
         out, err = handle.communicate()
         self.assert_(handle.returncode == 0)
         self.assert_(out.find('Locality group         : g1') >= 0)

Modified: accumulo/branches/ACCUMULO-CURATOR/trace/src/main/java/org/apache/accumulo/trace/instrument/receivers/ZooSpanClient.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/trace/src/main/java/org/apache/accumulo/trace/instrument/receivers/ZooSpanClient.java?rev=1496226&r1=1496225&r2=1496226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-CURATOR/trace/src/main/java/org/apache/accumulo/trace/instrument/receivers/ZooSpanClient.java (original)
+++ accumulo/branches/ACCUMULO-CURATOR/trace/src/main/java/org/apache/accumulo/trace/instrument/receivers/ZooSpanClient.java Mon Jun 24 21:34:20 2013
@@ -71,22 +71,12 @@ public class ZooSpanClient extends SendS
     zoo.getChildren(path, true);
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see trace.instrument.receivers.AsyncSpanReceiver#flush()
-   */
   @Override
   public void flush() {
     if (!hosts.isEmpty())
       super.flush();
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see trace.instrument.receivers.AsyncSpanReceiver#sendSpans()
-   */
   @Override
   void sendSpans() {
     if (hosts.isEmpty()) {
@@ -101,7 +91,7 @@ public class ZooSpanClient extends SendS
       super.sendSpans();
     }
   }
-
+  
   synchronized private void updateHosts(String path, List<String> children) {
     log.debug("Scanning trace hosts in zookeeper: " + path);
     try {