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/17 22:12:07 UTC

svn commit: r1493916 [4/5] - in /accumulo/branches/ACCUMULO-118: ./ assemble/ assemble/src/main/assemblies/ conf/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf...

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Mon Jun 17 20:12:04 2013
@@ -131,7 +131,6 @@ import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 
-
 /*
  * We need to be able to have the master tell a tabletServer to
  * close this file, and the tablet server to handle all pending client reads
@@ -162,7 +161,7 @@ public class Tablet {
   enum MinorCompactionReason {
     USER, SYSTEM, CLOSE
   }
-
+  
   public class CommitSession {
     
     private int seq;
@@ -444,11 +443,10 @@ public class Tablet {
   private volatile long numEntries;
   private volatile long numEntriesInMemory;
   
-
   // a count of the amount of data read by the iterators
   private AtomicLong scannedCount = new AtomicLong(0);
   private Rate scannedRate = new Rate(0.2);
-
+  
   private ConfigurationObserver configObserver;
   
   private TabletServer tabletServer;
@@ -666,7 +664,7 @@ public class Tablet {
       if (extent.isRootTablet()) {
         throw new IllegalArgumentException("Can not import files to root tablet");
       }
-
+      
       synchronized (bulkFileImportLock) {
         TCredentials auths = SecurityConstants.getSystemCredentials();
         Connector conn;
@@ -693,11 +691,11 @@ public class Tablet {
               dfv.setTime(bulkTime);
             }
           }
-
+          
           synchronized (timeLock) {
             if (bulkTime > persistedTime)
               persistedTime = bulkTime;
-          
+            
             MetadataTable.updateTabletDataFile(tid, extent, paths, tabletTime.getMetadataValue(persistedTime), auths, tabletServer.getLock());
           }
         }
@@ -1117,9 +1115,9 @@ public class Tablet {
     } else {
       
       Text rowName = extent.getMetadataEntry();
-        
+      
       ScannerImpl mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), SecurityConstants.getSystemCredentials(), Constants.METADATA_TABLE_ID,
-          Constants.NO_AUTHS);
+          Authorizations.EMPTY);
       
       // Commented out because when no data file is present, each tablet will scan through metadata table and return nothing
       // reduced batch size to improve performance
@@ -1292,6 +1290,7 @@ public class Tablet {
         constraintChecker.set(new ConstraintChecker(getTableConfiguration()));
       }
       
+      @Override
       public void propertiesChanged() {
         reloadConstraints();
         
@@ -1302,6 +1301,7 @@ public class Tablet {
         }
       }
       
+      @Override
       public void propertyChanged(String prop) {
         if (prop.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey()))
           reloadConstraints();
@@ -1316,6 +1316,7 @@ public class Tablet {
         
       }
       
+      @Override
       public void sessionExpired() {
         log.debug("Session expired, no longer updating per table props...");
       }
@@ -1337,6 +1338,7 @@ public class Tablet {
           absPaths.add(ref.path().toString());
         
         tabletServer.recover(this.tabletServer.getFileSystem(), this, logEntries, absPaths, new MutationReceiver() {
+          @Override
           public void receive(Mutation m) {
             // LogReader.printMutation(m);
             Collection<ColumnUpdate> muts = m.getUpdates();
@@ -1391,7 +1393,7 @@ public class Tablet {
       // TODO this could hang, causing other tablets to fail to load - ACCUMULO-1292
       AccumuloVFSClassLoader.getContextManager().getClassLoader(contextName);
     }
-
+    
     // do this last after tablet is completely setup because it
     // could cause major compaction to start
     datafileManager = new DatafileManager(datafiles);
@@ -1474,6 +1476,7 @@ public class Tablet {
       super(new Key(k), Arrays.copyOf(v.get(), v.get().length));
     }
     
+    @Override
     public String toString() {
       return key.toString() + "=" + getValue();
     }
@@ -1614,7 +1617,7 @@ public class Tablet {
     ScanDataSource dataSource = new ScanDataSource(authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag);
     
     LookupResult result = null;
-
+    
     try {
       SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource);
       result = lookup(iter, ranges, columns, results, maxResultSize);
@@ -1660,7 +1663,7 @@ public class Tablet {
     boolean endOfTabletReached = false;
     while (iter.hasTop()) {
       
-      value = (Value) iter.getTopValue();
+      value = iter.getTopValue();
       key = iter.getTopKey();
       
       KVEntry kvEntry = new KVEntry(key, value); // copies key and value
@@ -2023,7 +2026,7 @@ public class Tablet {
       if (statsIterator != null) {
         statsIterator.report();
       }
-
+      
     }
     
     public void interrupt() {
@@ -2108,6 +2111,7 @@ public class Tablet {
       this.mincReason = mincReason;
     }
     
+    @Override
     public void run() {
       minorCompactionWaitingToStart = false;
       minorCompactionInProgress = true;
@@ -2219,7 +2223,7 @@ public class Tablet {
       // don't bother trying to get flush id if closed... could be closed after this check but that is ok... just trying to cut down on uneeded log messages....
       return false;
     }
-
+    
     // get the flush id before the new memmap is made available for write
     long flushId;
     try {
@@ -2245,7 +2249,7 @@ public class Tablet {
     mct.run();
     return true;
   }
-
+  
   boolean initiateMinorCompaction(long flushId, MinorCompactionReason mincReason) {
     MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
     if (mct == null)
@@ -2332,7 +2336,7 @@ public class Tablet {
       throw new RuntimeException(e);
     }
   }
-
+  
   Pair<Long,List<IteratorSetting>> getCompactionID() throws NoNodeException {
     try {
       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
@@ -2342,7 +2346,7 @@ public class Tablet {
       long compactID = Long.parseLong(tokens[0]);
       
       CompactionIterators iters = new CompactionIterators();
-
+      
       if (tokens.length > 1) {
         Hex hex = new Hex();
         ByteArrayInputStream bais = new ByteArrayInputStream(hex.decode(tokens[1].split("=")[1].getBytes()));
@@ -2361,7 +2365,6 @@ public class Tablet {
           iters = new CompactionIterators();
         }
       }
-
       
       return new Pair<Long,List<IteratorSetting>>(compactID, iters.getIterators());
     } catch (InterruptedException e) {
@@ -2439,7 +2442,7 @@ public class Tablet {
       constraintChecker.compareAndSet(cc, ncc);
     }
   }
-
+  
   public CommitSession prepareMutationsForCommit(TservConstraintEnv cenv, List<Mutation> mutations) throws TConstraintViolationException {
     
     ConstraintChecker cc = constraintChecker.get();
@@ -2594,7 +2597,7 @@ public class Tablet {
           log.error(e.toString());
         }
       }
-
+      
       if (!saveState || tabletMemory.getMemTable().getNumEntries() == 0) {
         return;
       }
@@ -2716,21 +2719,21 @@ public class Tablet {
       Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> fileLog = MetadataTable.getFileAndLogEntries(SecurityConstants.getSystemCredentials(), extent);
       
       if (fileLog.getFirst().size() != 0) {
-        String msg = "Closed tablet " + extent + " has walog entries in !METADATA " + fileLog.getFirst();
+        String msg = "Closed tablet " + extent + " has walog entries in " + Constants.METADATA_TABLE_NAME + " " + fileLog.getFirst();
         log.error(msg);
         throw new RuntimeException(msg);
       }
       
       if (extent.isRootTablet()) {
         if (!fileLog.getSecond().keySet().equals(datafileManager.getDatafileSizes().keySet())) {
-          String msg = "Data file in !METADATA differ from in memory data " + extent + "  " + fileLog.getSecond().keySet() + "  "
+          String msg = "Data file in " + Constants.METADATA_TABLE_NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond().keySet() + "  "
               + datafileManager.getDatafileSizes().keySet();
           log.error(msg);
           throw new RuntimeException(msg);
         }
       } else {
         if (!fileLog.getSecond().equals(datafileManager.getDatafileSizes())) {
-          String msg = "Data file in !METADATA differ from in memory data " + extent + "  " + fileLog.getSecond() + "  "
+          String msg = "Data file in " + Constants.METADATA_TABLE_NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond() + "  "
               + datafileManager.getDatafileSizes();
           log.error(msg);
           throw new RuntimeException(msg);
@@ -2774,6 +2777,7 @@ public class Tablet {
       this.reason = reason;
     }
     
+    @Override
     public void run() {
       CompactionStats majCStats = null;
       
@@ -2794,7 +2798,7 @@ public class Tablet {
           if (reason == MajorCompactionReason.NORMAL && needsMajorCompaction(reason))
             initiateMajorCompaction(reason);
         }
-
+        
       } catch (RuntimeException E) {
         failed = true;
       } finally {
@@ -2807,7 +2811,7 @@ public class Tablet {
       }
     }
     
-    // We used to synchronize on the Tablet before fetching this information, 
+    // We used to synchronize on the Tablet before fetching this information,
     // but this method is called by the compaction queue thread to re-order the compactions.
     // The compaction queue holds a lock during this sort.
     // A tablet lock can be held while putting itself on the queue, so we can't lock the tablet
@@ -2841,7 +2845,7 @@ public class Tablet {
     }
     
     majorCompactionQueued.add(reason);
-
+    
     tabletResources.executeMajorCompaction(getExtent(), new CompactionRunner(reason));
     
     return false;
@@ -2934,7 +2938,7 @@ public class Tablet {
         Key first = pair.getFirst();
         Key last = pair.getSecond();
         // If first and last are null, it's an empty file. Add it to the compact set so it goes away.
-        if ( (first == null && last == null) || !this.extent.contains(first.getRow()) || !this.extent.contains(last.getRow())) {
+        if ((first == null && last == null) || !this.extent.contains(first.getRow()) || !this.extent.contains(last.getRow())) {
           result.put(file, entry.getValue().getSize());
         }
       }
@@ -3175,10 +3179,10 @@ public class Tablet {
               return majCStats;
           }
         }
-
+        
         compactionIterators = compactionId.getSecond();
       }
-
+      
       // need to handle case where only one file is being major compacted
       while (filesToCompact.size() > 0) {
         
@@ -3216,7 +3220,7 @@ public class Tablet {
           copy.keySet().retainAll(smallestFiles);
           
           log.debug("Starting MajC " + extent + " (" + reason + ") " + copy.keySet() + " --> " + compactTmpName + "  " + compactionIterators);
-
+          
           // always propagate deletes, unless last batch
           Compactor compactor = new Compactor(conf, fs, copy, null, compactTmpName, filesToCompact.size() == 0 ? propogateDeletes : true, acuTableConf, extent,
               cenv, compactionIterators, reason);
@@ -3303,7 +3307,7 @@ public class Tablet {
       
       majorCompactionInProgress = true;
     }
-
+    
     try {
       majCStats = _majorCompact(reason);
       if (reason == MajorCompactionReason.CHOP) {
@@ -3360,7 +3364,7 @@ public class Tablet {
     
     this.numEntries = numEntries;
   }
-
+  
   public long getNumEntries() {
     return numEntries;
   }
@@ -3463,8 +3467,8 @@ public class Tablet {
         splitPoint = findSplitRow(datafileManager.getFiles());
       else {
         Text tsp = new Text(sp);
-        splitPoint = new SplitRowSpec(FileUtil.estimatePercentageLTE(fs, tabletServer.getSystemConfiguration(), extent.getPrevEndRow(),
-            extent.getEndRow(), datafileManager.getFiles(), tsp), tsp);
+        splitPoint = new SplitRowSpec(FileUtil.estimatePercentageLTE(fs, tabletServer.getSystemConfiguration(), extent.getPrevEndRow(), extent.getEndRow(),
+            datafileManager.getFiles(), tsp), tsp);
       }
       
       if (splitPoint == null || splitPoint.row == null) {
@@ -3501,7 +3505,7 @@ public class Tablet {
       // finishes.... therefore split could propogate load flags for a finished bulk load... there is a special iterator
       // on the !METADATA table to clean up this type of garbage
       Map<FileRef,Long> bulkLoadedFiles = MetadataTable.getBulkFilesLoaded(SecurityConstants.getSystemCredentials(), extent);
-
+      
       MetadataTable.splitTablet(high, extent.getPrevEndRow(), splitRatio, SecurityConstants.getSystemCredentials(), tabletServer.getLock());
       MetadataTable.addNewTablet(low, lowDirectory, tabletServer.getTabletSession(), lowDatafileSizes, bulkLoadedFiles,
           SecurityConstants.getSystemCredentials(), time, lastFlushID, lastCompactID, tabletServer.getLock());
@@ -3545,7 +3549,7 @@ public class Tablet {
   public double scanRate() {
     return scannedRate.rate();
   }
-
+  
   public long totalQueries() {
     return this.queryCount;
   }
@@ -3627,7 +3631,7 @@ public class Tablet {
     }
     return result;
   }
-
+  
   private Set<String> beginClearingUnusedLogs() {
     Set<String> doomed = new HashSet<String>();
     
@@ -3789,8 +3793,8 @@ public class Tablet {
     
     if (updateMetadata) {
       try {
-      // if multiple threads were allowed to update this outside of a sync block, then it would be
-      // a race condition
+        // if multiple threads were allowed to update this outside of a sync block, then it would be
+        // a race condition
         MetadataTable.updateTabletCompactID(extent, compactionId, SecurityConstants.getSystemCredentials(), tabletServer.getLock());
       } finally {
         synchronized (this) {

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Mon Jun 17 20:12:04 2013
@@ -882,7 +882,8 @@ public class TabletServer extends Abstra
         Map<FileRef, MapFileInfo> fileRefMap = new HashMap<FileRef, MapFileInfo>();
         for (Entry<String,MapFileInfo> mapping : fileMap.entrySet()) {
           org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(mapping.getKey());
-          fileRefMap.put(new FileRef(mapping.getKey(), ns.makeQualified(new Path(mapping.getKey()))), mapping.getValue());
+          Path path = ns.makeQualified(new Path(mapping.getKey()));
+          fileRefMap.put(new FileRef(path.toString(), path), mapping.getValue());
         }
         
         Tablet importTablet = onlineTablets.get(new KeyExtent(tke));
@@ -1271,7 +1272,8 @@ public class TabletServer extends Abstra
         } else if (keyExtent.isRootTablet()) {
           throw new IllegalArgumentException("Cannot batch query root tablet with other tablets " + threadPoolExtent + " " + keyExtent);
         } else if (keyExtent.isMeta() && !threadPoolExtent.isMeta()) {
-          throw new IllegalArgumentException("Cannot batch query !METADATA and non !METADATA tablets " + threadPoolExtent + " " + keyExtent);
+          throw new IllegalArgumentException("Cannot batch query " + Constants.METADATA_TABLE_NAME + " and non " + Constants.METADATA_TABLE_NAME + " tablets "
+              + threadPoolExtent + " " + keyExtent);
         }
         
       }
@@ -2044,12 +2046,6 @@ public class TabletServer extends Abstra
       
     }
     
-    /*
-     * (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 {
       String myname = getClientAddressString();
@@ -2104,7 +2100,7 @@ public class TabletServer extends Abstra
         }
       }
     }
-        
+    
     @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       try {
@@ -2634,8 +2630,8 @@ public class TabletServer extends Abstra
       if (address == null) {
         return null;
       }
-      MasterClientService.Client client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), address,
-          Property.GENERAL_RPC_TIMEOUT, getSystemConfiguration());
+      MasterClientService.Client client = ThriftUtil.getClient(new MasterClientService.Client.Factory(), address, Property.GENERAL_RPC_TIMEOUT,
+          getSystemConfiguration());
       // log.info("Listener API to master has been opened");
       return client;
     } catch (Exception e) {
@@ -2889,7 +2885,7 @@ public class TabletServer extends Abstra
         Constants.METADATA_SPLIT_RATIO_COLUMN, Constants.METADATA_OLD_PREV_ROW_COLUMN, Constants.METADATA_TIME_COLUMN});
     
     ScannerImpl scanner = new ScannerImpl(HdfsZooInstance.getInstance(), SecurityConstants.getSystemCredentials(), Constants.METADATA_TABLE_ID,
-        Constants.NO_AUTHS);
+        Authorizations.EMPTY);
     scanner.setRange(extent.toMetadataRange());
     
     TreeMap<Key,Value> tkv = new TreeMap<Key,Value>();

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java Mon Jun 17 20:12:04 2013
@@ -175,11 +175,6 @@ public class DfsLogger {
     }
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.accumulo.server.tabletserver.log.IRemoteLogger#equals(java.lang.Object)
-   */
   @Override
   public boolean equals(Object obj) {
     // filename is unique
@@ -190,11 +185,6 @@ public class DfsLogger {
     return false;
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.accumulo.server.tabletserver.log.IRemoteLogger#hashCode()
-   */
   @Override
   public int hashCode() {
     // filename is unique
@@ -267,18 +257,24 @@ public class DfsLogger {
         logFile = fs.create(logPath, true, 0, replication, blockSize);
       
       try {
-        // sync: send data to datanodes
-        sync = logFile.getClass().getMethod("sync");
+        NoSuchMethodException e = null;
         try {
-          // hsych: send data to datanodes and sync the data to disk
-          sync = logFile.getClass().getMethod("hsync");
+          // sync: send data to datanodes
+          sync = logFile.getClass().getMethod("sync");
         } catch (NoSuchMethodException ex) {
+          e = ex;
         }
+        try {
+          // hsync: send data to datanodes and sync the data to disk
+          sync = logFile.getClass().getMethod("hsync");
+          e = null;
+        } catch (NoSuchMethodException ex) {}
+        if (e != null)
+          throw new RuntimeException(e);
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
       
-      
       // Initialize the crypto operations.
       @SuppressWarnings("deprecation")
       org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory.getCryptoModule(conf
@@ -310,13 +306,13 @@ public class DfsLogger {
       key.tserverSession = filename;
       key.filename = filename;
       write(key, EMPTY);
-      logFile.sync();
+      sync.invoke(logFile);
       log.debug("Got new write-ahead log: " + this);
-    } catch (IOException ex) {
+    } catch (Exception ex) {
       if (logFile != null)
         logFile.close();
       logFile = null;
-      throw ex;
+      throw new IOException(ex);
     }
     
     Thread t = new Daemon(new LogSyncingTask());
@@ -324,11 +320,6 @@ public class DfsLogger {
     t.start();
   }
   
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.accumulo.server.tabletserver.log.IRemoteLogger#toString()
-   */
   @Override
   public String toString() {
     return getLogger() + "/" + getFileName();
@@ -380,10 +371,10 @@ public class DfsLogger {
     key.tablet = tablet;
     try {
       write(key, EMPTY);
-      logFile.sync();
-    } catch (IOException ex) {
+      sync.invoke(logFile);
+    } catch (Exception ex) {
       log.error(ex);
-      throw ex;
+      throw new IOException(ex);
     }
   }
   

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/AddFilesWithMissingEntries.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/AddFilesWithMissingEntries.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/AddFilesWithMissingEntries.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/AddFilesWithMissingEntries.java Mon Jun 17 20:12:04 2013
@@ -21,7 +21,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.Scanner;
@@ -31,7 +30,9 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.PartialKey;
 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.server.ServerConstants;
+import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -46,17 +47,13 @@ public class AddFilesWithMissingEntries 
   static final Logger log = Logger.getLogger(AddFilesWithMissingEntries.class);
   
   public static class Opts extends ClientOpts {
-    @Parameter(names="-update", description="Make changes to the !METADATA table to include missing files")
+    @Parameter(names = "-update", description = "Make changes to the " + Constants.METADATA_TABLE_NAME + " table to include missing files")
     boolean update = false;
   }
   
-  
   /**
-   * A utility to add files to the !METADATA table that are not listed in the root tablet.  
-   * This is a recovery tool for someone who knows what they are doing.  It might be better to 
-   * save off files, and recover your instance by re-initializing and importing the existing files.
-   *  
-   * @param args
+   * A utility to add files to the {@value Constants#METADATA_TABLE_NAME} table that are not listed in the root tablet. This is a recovery tool for someone who
+   * knows what they are doing. It might be better to save off files, and recover your instance by re-initializing and importing the existing files.
    */
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
@@ -65,7 +62,7 @@ public class AddFilesWithMissingEntries 
     
     final Key rootTableEnd = new Key(Constants.ROOT_TABLET_EXTENT.getEndRow());
     final Range range = new Range(rootTableEnd.followingKey(PartialKey.ROW), true, Constants.METADATA_RESERVED_KEYSPACE_START_KEY, false);
-    final Scanner scanner = opts.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    final Scanner scanner = opts.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(range);
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.get(conf);
@@ -108,7 +105,8 @@ public class AddFilesWithMissingEntries 
     writer.close();
   }
   
-  private static int addUnknownFiles(FileSystem fs, String directory, Set<String> knownFiles, KeyExtent ke, MultiTableBatchWriter writer, boolean update) throws Exception {
+  private static int addUnknownFiles(FileSystem fs, String directory, Set<String> knownFiles, KeyExtent ke, MultiTableBatchWriter writer, boolean update)
+      throws Exception {
     int count = 0;
     final String tableId = ke.getTableId().toString();
     final Text row = ke.getMetadataEntry();

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java Mon Jun 17 20:12:04 2013
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.server.cli.ClientOpts;
@@ -105,7 +106,7 @@ public class CheckForMetadataProblems {
     if (opts.offline) {
       scanner = new OfflineMetadataScanner(ServerConfiguration.getSystemConfiguration(opts.getInstance()), fs);
     } else {
-      scanner =  opts.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+      scanner = opts.getConnector().createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     }
     
     scanner.setRange(Constants.METADATA_KEYSPACE);
@@ -180,14 +181,13 @@ public class CheckForMetadataProblems {
   }
   
   static class Opts extends ClientOpts {
-    @Parameter(names="--fix", description="best-effort attempt to fix problems found")
+    @Parameter(names = "--fix", description = "best-effort attempt to fix problems found")
     boolean fix = false;
     
-    @Parameter(names="--offline", description="perform the check on the files directly")
+    @Parameter(names = "--offline", description = "perform the check on the files directly")
     boolean offline = false;
   }
   
-  
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(CheckForMetadataProblems.class.getName(), args);

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Mon Jun 17 20:12:04 2013
@@ -409,7 +409,7 @@ public class Initialize {
       if (opts.clearInstanceName) {
         exists = false;
         break;
-      } else if ((boolean) (exists = ZooReaderWriter.getInstance().exists(instanceNamePath))) {
+      } else if (exists = ZooReaderWriter.getInstance().exists(instanceNamePath)) {
         String decision = getConsoleReader().readLine("Instance name \"" + instanceName + "\" exists. Delete existing entry from zookeeper? [Y/N] : ");
         if (decision == null)
           System.exit(0);
@@ -421,7 +421,7 @@ public class Initialize {
     } while (exists);
     return instanceNamePath;
   }
-
+  
   private static byte[] getRootPassword(Opts opts) throws IOException {
     if (opts.cliPassword != null) {
       return opts.cliPassword.getBytes();
@@ -429,7 +429,8 @@ public class Initialize {
     String rootpass;
     String confirmpass;
     do {
-      rootpass = getConsoleReader().readLine("Enter initial password for " + DEFAULT_ROOT_USER + " (this may not be applicable for your security setup): ", '*');
+      rootpass = getConsoleReader()
+          .readLine("Enter initial password for " + DEFAULT_ROOT_USER + " (this may not be applicable for your security setup): ", '*');
       if (rootpass == null)
         System.exit(0);
       confirmpass = getConsoleReader().readLine("Confirm initial password for " + DEFAULT_ROOT_USER + ": ", '*');
@@ -466,8 +467,8 @@ public class Initialize {
   
   private static void setMetadataReplication(int replication, String reason) throws IOException {
     String rep = getConsoleReader().readLine(
-        "Your HDFS replication " + reason
-        + " is not compatible with our default !METADATA replication of 5. What do you want to set your !METADATA replication to? (" + replication + ") ");
+        "Your HDFS replication " + reason + " is not compatible with our default " + Constants.METADATA_TABLE_NAME
+            + " replication of 5. What do you want to set your " + Constants.METADATA_TABLE_NAME + " replication to? (" + replication + ") ");
     if (rep == null || rep.length() == 0)
       rep = Integer.toString(replication);
     else
@@ -498,7 +499,7 @@ public class Initialize {
   public static void main(String[] args) {
     Opts opts = new Opts();
     opts.parseArgs(Initialize.class.getName(), args);
-        
+    
     try {
       SecurityUtil.serverLogin();
       Configuration conf = CachedConfiguration.getInstance();

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java Mon Jun 17 20:12:04 2013
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.client.C
 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.server.cli.ClientOpts;
 import org.apache.accumulo.server.fs.FileSystem;
 import org.apache.accumulo.server.fs.FileSystemImpl;
@@ -42,7 +43,7 @@ public class LocalityCheck {
     
     FileSystem fs = FileSystemImpl.get();
     Connector connector = opts.getConnector();
-    Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
     scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
     scanner.setRange(Constants.METADATA_KEYSPACE);

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java Mon Jun 17 20:12:04 2013
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.data.Par
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.server.util.FileUtil;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -337,7 +338,7 @@ public class MetadataTable extends org.a
   public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, TCredentials credentials) throws IOException {
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
     
-    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     mdScanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
     Text row = extent.getMetadataEntry();
     FileSystem fs = FileSystemImpl.get();
@@ -392,7 +393,7 @@ public class MetadataTable extends org.a
     Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
     update(credentials, zooLock, m);
   }
-
+  
   public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, TCredentials credentials, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     
@@ -482,12 +483,14 @@ public class MetadataTable extends org.a
     String prefix = Constants.METADATA_DELETE_FLAG_PREFIX;
     if (tableId.equals(Constants.METADATA_TABLE_ID))
       prefix = Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX;
-
-    if (pathToRemove.startsWith("../"))
-      pathToRemove = pathToRemove.substring(2);
-    else
-      pathToRemove = "/" + tableId + "/" + pathToRemove;
-
+    
+    if (!pathToRemove.contains(":")) {
+      if (pathToRemove.startsWith("../"))
+        pathToRemove = pathToRemove.substring(2);
+      else
+        pathToRemove = "/" + tableId + "/" + pathToRemove;
+    }
+    
     Path path = FileSystemImpl.get().getFullPath(ServerConstants.getTablesDirs(), pathToRemove);
     Mutation delFlag = new Mutation(new Text(prefix + path.toString()));
     delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
@@ -512,8 +515,8 @@ public class MetadataTable extends org.a
     
     // check to see if prev tablet exist in metadata tablet
     Key prevRowKey = new Key(new Text(KeyExtent.getMetadataEntry(table, metadataPrevEndRow)));
-
-    ScannerImpl scanner2 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    
+    ScannerImpl scanner2 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW)));
     
     FileSystem fs = FileSystemImpl.get();
@@ -523,10 +526,10 @@ public class MetadataTable extends org.a
       return new KeyExtent(metadataEntry, KeyExtent.decodePrevEndRow(oper));
     } else {
       log.info("Finishing incomplete split " + metadataEntry + " " + metadataPrevEndRow);
-
+      
       List<FileRef> highDatafilesToRemove = new ArrayList<FileRef>();
-
-      Scanner scanner3 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+      
+      Scanner scanner3 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
       Key rowKey = new Key(metadataEntry);
       
       SortedMap<FileRef,DataFileValue> origDatafileSizes = new TreeMap<FileRef,DataFileValue>();
@@ -543,13 +546,12 @@ public class MetadataTable extends org.a
       
       splitDatafiles(table, metadataPrevEndRow, splitRatio, new HashMap<FileRef,FileUtil.FileInfo>(), origDatafileSizes, lowDatafileSizes, highDatafileSizes,
           highDatafilesToRemove);
-    
+      
       MetadataTable.finishSplit(metadataEntry, highDatafileSizes, highDatafilesToRemove, credentials, lock);
       
       return new KeyExtent(metadataEntry, KeyExtent.encodePrevEndRow(metadataPrevEndRow));
     }
-
-
+    
   }
   
   public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<FileRef,FileUtil.FileInfo> firstAndLastRows,
@@ -637,7 +639,7 @@ public class MetadataTable extends org.a
   }
   
   public static void deleteTable(String tableId, boolean insertDeletes, TCredentials credentials, ZooLock lock) throws AccumuloException, IOException {
-    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     Text tableIdText = new Text(tableId);
     BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, new BatchWriterConfig().setMaxMemory(1000000)
         .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
@@ -809,7 +811,7 @@ public class MetadataTable extends org.a
       }
       
     } else {
-      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
       scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
       scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
       scanner.setRange(extent.toMetadataRange());
@@ -871,8 +873,8 @@ public class MetadataTable extends org.a
   private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     String root = getZookeeperLogLocation();
-    // there's a little race between getting the children and fetching 
-    // the data.  The log can be removed in between.
+    // there's a little race between getting the children and fetching
+    // the data. The log can be removed in between.
     while (true) {
       result.clear();
       for (String child : zoo.getChildren(root)) {
@@ -889,7 +891,7 @@ public class MetadataTable extends org.a
   }
   
   private static Scanner getTabletLogScanner(TCredentials credentials, KeyExtent extent) {
-    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
     Text start = extent.getMetadataEntry();
     Key endKey = new Key(start, Constants.METADATA_LOG_COLUMN_FAMILY);
@@ -907,7 +909,7 @@ public class MetadataTable extends org.a
       rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
       try {
         Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), CredentialHelper.extractToken(creds))
-            .createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+            .createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
         log.info("Setting range to " + Constants.NON_ROOT_METADATA_KEYSPACE);
         scanner.setRange(Constants.NON_ROOT_METADATA_KEYSPACE);
         scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
@@ -1000,7 +1002,7 @@ public class MetadataTable extends org.a
   }
   
   private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
-    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
+    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY));
     mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
     mscanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
     mscanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
@@ -1145,7 +1147,7 @@ public class MetadataTable extends org.a
     }
     
     // delete the clone markers and create directory entries
-    Scanner mscanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner mscanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
     mscanner.fetchColumnFamily(Constants.METADATA_CLONED_COLUMN_FAMILY);
     
@@ -1170,7 +1172,7 @@ public class MetadataTable extends org.a
   }
   
   public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
-    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
+    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY));
     mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
     mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
     BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
@@ -1190,7 +1192,7 @@ public class MetadataTable extends org.a
     List<FileRef> result = new ArrayList<FileRef>();
     try {
       FileSystem fs = FileSystemImpl.get();
-      Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
+      Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY));
       mscanner.setRange(extent.toMetadataRange());
       mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
       for (Entry<Key,Value> entry : mscanner) {
@@ -1214,7 +1216,7 @@ public class MetadataTable extends org.a
     Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
     
     FileSystem fs = FileSystemImpl.get();
-    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner.setRange(new Range(metadataRow));
     scanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
     for (Entry<Key,Value> entry : scanner) {
@@ -1239,23 +1241,23 @@ public class MetadataTable extends org.a
     
     update(SecurityConstants.getSystemCredentials(), m);
   }
-
+  
   public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
     // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
-    Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
+    Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner.setRange(new Range(Constants.METADATA_DELETES_KEYSPACE));
     for (Entry<Key,Value> entry : scanner) {
       String row = entry.getKey().getRow().toString();
-      if (row.startsWith(Constants.METADATA_DELETE_FLAG_PREFIX + "/" + Constants.METADATA_TABLE_ID)) {
+      if (row.startsWith(Constants.METADATA_DELETE_FLAG_PREFIX)) {
         String filename = row.substring(Constants.METADATA_DELETE_FLAG_PREFIX.length());
         // add the new entry first
         log.info("Moving " + filename + " marker to the root tablet");
         Mutation m = new Mutation(Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX + filename);
-        m.put(new byte[]{}, new byte[]{}, new byte[]{});
+        m.put(new byte[] {}, new byte[] {}, new byte[] {});
         update(creds, m);
         // remove the old entry
         m = new Mutation(entry.getKey().getRow());
-        m.putDelete(new byte[]{}, new byte[]{});
+        m.putDelete(new byte[] {}, new byte[] {});
         update(creds, m);
       } else {
         break;

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Mon Jun 17 20:12:04 2013
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.server.ServerConstants;
@@ -79,7 +80,7 @@ public class OfflineMetadataScanner exte
     DeletingIterator delIter = new DeletingIterator(multiIterator, false);
     ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
     ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, columns);
-    VisibilityFilter visFilter = new VisibilityFilter(colFilter, Constants.NO_AUTHS, new byte[0]);
+    VisibilityFilter visFilter = new VisibilityFilter(colFilter, Authorizations.EMPTY, new byte[0]);
     
     visFilter.seek(r, LocalityGroupUtil.EMPTY_CF_SET, false);
     

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java Mon Jun 17 20:12:04 2013
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.conf.Def
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.NumUtil;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.cli.ClientOpts;
@@ -152,7 +153,7 @@ public class TableDiskUsage {
     for (String tableId : tableIds) {
       Scanner mdScanner = null;
       try {
-        mdScanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+        mdScanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
       } catch (TableNotFoundException e) {
         throw new RuntimeException(e);
       }

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java Mon Jun 17 20:12:04 2013
@@ -29,7 +29,6 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
@@ -45,6 +44,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
@@ -90,8 +90,7 @@ public class VerifyTabletAssignments {
     
     Connector conn = opts.getConnector();
     Instance inst = conn.getInstance();
-    MetadataTable.getEntries(conn.getInstance(), CredentialHelper.create(opts.principal, opts.getToken(), opts.instance), tableName, false,
-        locations, tablets);
+    MetadataTable.getEntries(conn.getInstance(), CredentialHelper.create(opts.principal, opts.getToken(), opts.instance), tableName, false, locations, tablets);
     
     final HashSet<KeyExtent> failures = new HashSet<KeyExtent>();
     
@@ -151,8 +150,8 @@ public class VerifyTabletAssignments {
     }
   }
   
-  private static void checkTabletServer(AccumuloConfiguration conf, TCredentials st, Entry<String,List<KeyExtent>> entry,
-      HashSet<KeyExtent> failures) throws ThriftSecurityException, TException, NoSuchScanIDException {
+  private static void checkTabletServer(AccumuloConfiguration conf, TCredentials st, Entry<String,List<KeyExtent>> entry, HashSet<KeyExtent> failures)
+      throws ThriftSecurityException, TException, NoSuchScanIDException {
     TabletClientService.Iface client = ThriftUtil.getTServerClient(entry.getKey(), conf);
     
     Map<TKeyExtent,List<TRange>> batch = new TreeMap<TKeyExtent,List<TRange>>();
@@ -187,8 +186,8 @@ public class VerifyTabletAssignments {
     Map<String,Map<String,String>> emptyMapSMapSS = Collections.emptyMap();
     List<IterInfo> emptyListIterInfo = Collections.emptyList();
     List<TColumn> emptyListColumn = Collections.emptyList();
-    InitialMultiScan is = client.startMultiScan(tinfo, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS, Constants.NO_AUTHS.getAuthorizationsBB(),
-        false);
+    InitialMultiScan is = client.startMultiScan(tinfo, st, batch, emptyListColumn, emptyListIterInfo, emptyMapSMapSS,
+        Authorizations.EMPTY.getAuthorizationsBB(), false);
     if (is.result.more) {
       MultiScanResult result = client.continueMultiScan(tinfo, is.scanID);
       checkFailures(entry.getKey(), failures, result);

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java Mon Jun 17 20:12:04 2013
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.client.s
 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.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.fs.FileSystem;
@@ -109,7 +110,7 @@ public class TestConfirmDeletes {
     TCredentials credential = CredentialHelper.create("root", new PasswordToken(new byte[0]), "instance");
     
     Scanner scanner = instance.getConnector(credential.getPrincipal(), CredentialHelper.extractToken(credential)).createScanner(Constants.METADATA_TABLE_NAME,
-        Constants.NO_AUTHS);
+        Authorizations.EMPTY);
     int count = 0;
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java Mon Jun 17 20:12:04 2013
@@ -35,6 +35,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.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.master.state.Assignment;
@@ -137,7 +138,7 @@ public class TestMergeState {
     Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, newState);
     
     // unassign the tablets
-    BatchDeleter deleter = connector.createBatchDeleter("!METADATA", Constants.NO_AUTHS, 1000, new BatchWriterConfig());
+    BatchDeleter deleter = connector.createBatchDeleter("!METADATA", Authorizations.EMPTY, 1000, new BatchWriterConfig());
     deleter.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
     deleter.setRanges(Collections.singletonList(new Range()));
     deleter.delete();

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/CloneTest.java Mon Jun 17 20:12:04 2013
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 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.io.Text;
 
 public class CloneTest extends TestCase {
@@ -100,7 +101,7 @@ public class CloneTest extends TestCase 
     
     assertEquals(0, rc);
     
-    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
     
     HashSet<String> files = new HashSet<String>();
@@ -139,7 +140,7 @@ public class CloneTest extends TestCase 
     
     assertEquals(0, rc);
     
-    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
     
     HashSet<String> files = new HashSet<String>();
@@ -187,7 +188,7 @@ public class CloneTest extends TestCase 
     
     assertEquals(0, rc);
     
-    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
     
     HashSet<String> files = new HashSet<String>();
@@ -255,7 +256,7 @@ public class CloneTest extends TestCase 
     
     assertEquals(0, rc);
     
-    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
     
     HashSet<String> files = new HashSet<String>();
@@ -319,7 +320,7 @@ public class CloneTest extends TestCase 
     
     assertEquals(0, rc);
     
-    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+    Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
     
     HashSet<String> files = new HashSet<String>();

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java Mon Jun 17 20:12:04 2013
@@ -32,6 +32,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.server.util.TabletIterator.TabletDeletedException;
 import org.apache.hadoop.io.Text;
 
@@ -42,13 +43,13 @@ public class TabletIteratorTest extends 
     private Connector conn;
     
     public TestTabletIterator(Connector conn) throws Exception {
-      super(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS), Constants.METADATA_KEYSPACE, true, true);
+      super(conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY), Constants.METADATA_KEYSPACE, true, true);
       this.conn = conn;
     }
     
     protected void resetScanner() {
       try {
-        Scanner ds = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+        Scanner ds = conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
         Text tablet = new KeyExtent(new Text("0"), new Text("m"), null).getMetadataEntry();
         ds.setRange(new Range(tablet, true, tablet, true));
         

Propchange: accumulo/branches/ACCUMULO-118/src/
------------------------------------------------------------------------------
  Reverse-merged /accumulo/trunk/src:r1478775
  Merged /accumulo/trunk/src:r1486112-1486135,1486203-1488406,1488408-1489580,1489976-1489981,1490337-1493757
  Merged /accumulo/branches/1.4/src:r1382578-1382612,1382614-1388119,1388121-1388628,1388630-1393867,1393869-1396064,1396066-1396571,1396573-1396615,1396617-1396757,1396759-1396771,1396773-1397047,1397049-1397112,1397114-1397116,1397118-1397175,1397177-1397188,1397190-1397382,1397384-1397699,1397701-1397920,1397922-1398285,1398287-1398307,1398309-1398392,1398394-1398398,1398400-1398437,1398439-1398513,1398515-1398800,1398802-1399210,1399212-1399716,1399718-1400975,1400977-1402570,1402572-1402681,1402683-1407300,1407302-1423031,1423033-1423628,1423630-1427863,1427865-1450270,1450272-1451699,1451701-1455609,1455611-1455638,1455640-1455979,1455981-1461547,1461549-1465686,1465688-1466198,1466200-1466258,1466260-1466626,1466628-1468530,1468532-1468957,1468959-1480032,1480034-1482988,1482990-1483953,1483955-1492947
  Merged /incubator/accumulo/trunk/src:r1178656-1201898
  Merged /incubator/accumulo/branches/1.4/src:r1201899-1201901
  Merged /accumulo/branches/1.5/src:r1484971-1484975,1484977-1484982,1484986-1485411,1485413-1485414,1485416-1485661,1485663-1492948

Propchange: accumulo/branches/ACCUMULO-118/start/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Mon Jun 17 20:12:04 2013
@@ -1,5 +1,3 @@
+.*
 target
-.classpath
-.project
-.settings
 accumulo-start.iml

Modified: accumulo/branches/ACCUMULO-118/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java (original)
+++ accumulo/branches/ACCUMULO-118/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java Mon Jun 17 20:12:04 2013
@@ -25,7 +25,10 @@ import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -45,6 +48,7 @@ public class AccumuloClassLoader {
   
   public static final String CLASSPATH_PROPERTY_NAME = "general.classpaths";
   
+  /* @formatter:off */
   public static final String ACCUMULO_CLASSPATH_VALUE = 
       "$ACCUMULO_HOME/conf,\n" + 
           "$ACCUMULO_HOME/lib/[^.].*.jar,\n" + 
@@ -57,6 +61,10 @@ public class AccumuloClassLoader {
           "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar,\n" +
           "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar,\n"
           ;
+  /* @formatter:on */
+  
+  public static final String MAVEN_PROJECT_BASEDIR_PROPERTY_NAME = "general.maven.project.basedir";
+  public static final String DEFAULT_MAVEN_PROJECT_BASEDIR_VALUE = "";
   
   private static String SITE_CONF;
   
@@ -208,6 +216,8 @@ public class AccumuloClassLoader {
       return new ArrayList<URL>();
     String[] cps = replaceEnvVars(cp, System.getenv()).split(",");
     ArrayList<URL> urls = new ArrayList<URL>();
+    for (String classpath : getMavenClasspaths())
+      addUrl(classpath, urls);
     for (String classpath : cps) {
       if (!classpath.startsWith("#")) {
         addUrl(classpath, urls);
@@ -216,6 +226,27 @@ public class AccumuloClassLoader {
     return urls;
   }
   
+  private static Set<String> getMavenClasspaths() {
+    String baseDirname = AccumuloClassLoader.getAccumuloString(MAVEN_PROJECT_BASEDIR_PROPERTY_NAME, DEFAULT_MAVEN_PROJECT_BASEDIR_VALUE);
+    if (baseDirname == null || baseDirname.trim().isEmpty())
+      return Collections.emptySet();
+    Set<String> paths = new TreeSet<String>();
+    findMavenTargetClasses(paths, new File(baseDirname.trim()), 0);
+    return paths;
+  }
+  
+  private static void findMavenTargetClasses(Set<String> paths, File file, int depth) {
+    if (depth > 3)
+      return;
+    if (file.isDirectory()) {
+      File[] children = file.listFiles();
+      for (File child : children)
+        findMavenTargetClasses(paths, child, depth + 1);
+    } else if ("pom.xml".equals(file.getName())) {
+      paths.add(file.getParentFile().getAbsolutePath() + File.separator + "target" + File.separator + "classes");
+    }
+  }
+  
   public static synchronized ClassLoader getClassLoader() throws IOException {
     if (classloader == null) {
       ArrayList<URL> urls = findAccumuloURLs();

Propchange: accumulo/branches/ACCUMULO-118/test/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Mon Jun 17 20:12:04 2013
@@ -1,5 +1,3 @@
+.*
 target
-.classpath
-.project
-.settings
 accumulo-test.iml

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java Mon Jun 17 20:12:04 2013
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.KeyExtent;
 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.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.hadoop.io.Text;
@@ -55,12 +56,13 @@ public class QueryMetadataTable {
       this.row = row;
     }
     
+    @Override
     public void run() {
       try {
         KeyExtent extent = new KeyExtent(row, (Text) null);
         
         Connector connector = HdfsZooInstance.getInstance().getConnector(principal, token);
-        Scanner mdScanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
+        Scanner mdScanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
         Text row = extent.getMetadataEntry();
         
         mdScanner.setRange(new Range(row));
@@ -84,9 +86,9 @@ public class QueryMetadataTable {
   }
   
   static class Opts extends ClientOpts {
-    @Parameter(names="--numQueries", description="number of queries to run")
+    @Parameter(names = "--numQueries", description = "number of queries to run")
     int numQueries = 1;
-    @Parameter(names="--numThreads", description="number of threads used to run the queries")
+    @Parameter(names = "--numThreads", description = "number of threads used to run the queries")
     int numThreads = 1;
   }
   

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousWalk.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousWalk.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousWalk.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousWalk.java Mon Jun 17 20:12:04 2013
@@ -26,15 +26,14 @@ import java.util.Map.Entry;
 import java.util.Random;
 import java.util.zip.CRC32;
 
-import org.apache.accumulo.trace.instrument.Span;
-import org.apache.accumulo.trace.instrument.Trace;
-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.trace.instrument.Span;
+import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,7 +42,6 @@ import org.apache.hadoop.io.Text;
 import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
 
-
 public class ContinuousWalk {
   
   static public class Opts extends ContinuousQuery.Opts {
@@ -57,7 +55,8 @@ public class ContinuousWalk {
         }
       }
     }
-    @Parameter(names="--authsFile", description="read the authorities to use from a file")
+    
+    @Parameter(names = "--authsFile", description = "read the authorities to use from a file")
     RandomAuths randomAuths = new RandomAuths();
   }
   
@@ -74,12 +73,12 @@ public class ContinuousWalk {
     private List<Authorizations> auths;
     
     RandomAuths() {
-      auths = Collections.singletonList(Constants.NO_AUTHS);
+      auths = Collections.singletonList(Authorizations.EMPTY);
     }
     
     RandomAuths(String file) throws IOException {
       if (file == null) {
-        auths = Collections.singletonList(Constants.NO_AUTHS);
+        auths = Collections.singletonList(Authorizations.EMPTY);
         return;
       }
       
@@ -101,7 +100,7 @@ public class ContinuousWalk {
       return auths.get(r.nextInt(auths.size()));
     }
   }
-
+  
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(ContinuousWalk.class.getName(), args);

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/AddSplitTest.java Mon Jun 17 20:12:04 2013
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -35,6 +34,7 @@ import org.apache.accumulo.core.client.T
 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.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -99,7 +99,7 @@ public class AddSplitTest extends Functi
   }
   
   private void verifyData(long ts) throws Exception {
-    Scanner scanner = getConnector().createScanner("foo", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("foo", Authorizations.EMPTY);
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
     

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincTest.java Mon Jun 17 20:12:04 2013
@@ -21,7 +21,6 @@ import java.util.List;
 import java.util.Map;
 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.Scanner;
@@ -29,6 +28,7 @@ import org.apache.accumulo.core.conf.Pro
 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.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -69,7 +69,7 @@ public class BadIteratorMincTest extends
     checkRFiles("foo", 1, 1, 0, 0);
     
     // try to scan table
-    Scanner scanner = getConnector().createScanner("foo", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("foo", Authorizations.EMPTY);
     
     int count = 0;
     for (@SuppressWarnings("unused")
@@ -115,7 +115,7 @@ public class BadIteratorMincTest extends
     
     // this should not hang
     getConnector().tableOperations().delete("foo");
-
+    
   }
   
 }

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitTest.java Mon Jun 17 20:12:04 2013
@@ -25,7 +25,6 @@ import java.util.Map;
 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.BatchWriterConfig;
@@ -34,6 +33,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.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -95,7 +95,7 @@ public class BatchScanSplitTest extends 
     
     // logger.setLevel(Level.TRACE);
     
-    BatchScanner bs = getConnector().createBatchScanner("bss", Constants.NO_AUTHS, 4);
+    BatchScanner bs = getConnector().createBatchScanner("bss", Authorizations.EMPTY, 4);
     
     HashMap<Text,Value> found = new HashMap<Text,Value>();
     

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushTest.java Mon Jun 17 20:12:04 2013
@@ -25,7 +25,6 @@ import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 import java.util.Random;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -37,6 +36,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.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -73,7 +73,7 @@ public class BatchWriterFlushTest extend
   private void runLatencyTest() throws Exception {
     // should automatically flush after 3 seconds
     BatchWriter bw = getConnector().createBatchWriter("bwlt", new BatchWriterConfig().setMaxLatency(2000, TimeUnit.MILLISECONDS));
-    Scanner scanner = getConnector().createScanner("bwlt", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("bwlt", Authorizations.EMPTY);
     
     Mutation m = new Mutation(new Text(String.format("r_%10d", 1)));
     m.put(new Text("cf"), new Text("cq"), new Value(("" + 1).getBytes()));
@@ -107,7 +107,7 @@ public class BatchWriterFlushTest extend
   
   private void runFlushTest() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, MutationsRejectedException, Exception {
     BatchWriter bw = getConnector().createBatchWriter("bwft", new BatchWriterConfig());
-    Scanner scanner = getConnector().createScanner("bwft", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("bwft", Authorizations.EMPTY);
     Random r = new Random();
     
     for (int i = 0; i < 4; i++) {

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java Mon Jun 17 20:12:04 2013
@@ -24,7 +24,6 @@ import java.util.Map;
 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.BatchWriterConfig;
@@ -35,6 +34,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 
 public class BloomFilterTest extends FunctionalTest {
@@ -75,7 +75,7 @@ public class BloomFilterTest extends Fun
     bw.close();
     getConnector().tableOperations().flush("bt4", null, null, true);
     
-    for (String table : new String[]{"bt1", "bt2", "bt3"}) {
+    for (String table : new String[] {"bt1", "bt2", "bt3"}) {
       getConnector().tableOperations().setProperty(table, Property.TABLE_INDEXCACHE_ENABLED.getKey(), "false");
       getConnector().tableOperations().setProperty(table, Property.TABLE_BLOCKCACHE_ENABLED.getKey(), "false");
       getConnector().tableOperations().flush(table, null, null, true);
@@ -122,7 +122,7 @@ public class BloomFilterTest extends Fun
     timeCheck(t3, tb3);
     
     // test querying for empty key
-    Scanner scanner = getConnector().createScanner("bt4", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("bt4", Authorizations.EMPTY);
     scanner.setRange(new Range(new Text("")));
     
     if (!scanner.iterator().next().getValue().toString().equals("foo1")) {
@@ -172,7 +172,7 @@ public class BloomFilterTest extends Fun
       ranges.add(range);
     }
     
-    BatchScanner bs = getConnector().createBatchScanner(table, Constants.NO_AUTHS, 3);
+    BatchScanner bs = getConnector().createBatchScanner(table, Authorizations.EMPTY, 3);
     bs.setRanges(ranges);
     
     long t1 = System.currentTimeMillis();

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/BulkFileTest.java Mon Jun 17 20:12:04 2013
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -31,6 +30,7 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.trace.TraceFileSystem;
 import org.apache.hadoop.conf.Configuration;
@@ -87,7 +87,7 @@ public class BulkFileTest extends Functi
   }
   
   private void verifyData(String table, int s, int e) throws Exception {
-    Scanner scanner = getConnector().createScanner(table, Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
     

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyTest.java Mon Jun 17 20:12:04 2013
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Map;
 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;
@@ -32,6 +31,7 @@ import org.apache.accumulo.core.conf.Pro
 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.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
@@ -43,7 +43,7 @@ public class ConcurrencyTest extends Fun
     Scanner scanner;
     
     ScanTask(Connector conn, long time) throws Exception {
-      scanner = conn.createScanner("cct", Constants.NO_AUTHS);
+      scanner = conn.createScanner("cct", Authorizations.EMPTY);
       IteratorSetting slow = new IteratorSetting(30, "slow", SlowIterator.class);
       slow.addOption("sleepTime", "" + time);
       scanner.addScanIterator(slow);

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/ConstraintTest.java Mon Jun 17 20:12:04 2013
@@ -35,6 +35,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.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -123,7 +124,7 @@ public class ConstraintTest extends Func
     }
     
     // verify mutation did not go through
-    Scanner scanner = getConnector().createScanner("ct", Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner("ct", Authorizations.EMPTY);
     scanner.setRange(new Range(new Text("r1")));
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
@@ -293,7 +294,7 @@ public class ConstraintTest extends Func
       throw new Exception("Did not see MutationsRejectedException");
     }
     
-    Scanner scanner = getConnector().createScanner(table, Constants.NO_AUTHS);
+    Scanner scanner = getConnector().createScanner(table, Authorizations.EMPTY);
     
     Iterator<Entry<Key,Value>> iter = scanner.iterator();
     

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java?rev=1493916&r1=1493915&r2=1493916&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/functional/CreateAndUseTest.java Mon Jun 17 20:12:04 2013
@@ -24,7 +24,6 @@ import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-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;
@@ -33,6 +32,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.hadoop.io.Text;
 
 public class CreateAndUseTest extends FunctionalTest {
@@ -79,7 +79,7 @@ public class CreateAndUseTest extends Fu
     bw.close();
     
     // verify data is there
-    Scanner scanner1 = getConnector().createScanner("t1", Constants.NO_AUTHS);
+    Scanner scanner1 = getConnector().createScanner("t1", Authorizations.EMPTY);
     
     int ei = 1;
     
@@ -102,7 +102,7 @@ public class CreateAndUseTest extends Fu
     // TEST 2 create a table and immediately scan it
     getConnector().tableOperations().create("t2");
     getConnector().tableOperations().addSplits("t2", splits);
-    Scanner scanner2 = getConnector().createScanner("t2", Constants.NO_AUTHS);
+    Scanner scanner2 = getConnector().createScanner("t2", Authorizations.EMPTY);
     int count = 0;
     for (Entry<Key,Value> entry : scanner2) {
       if (entry != null)
@@ -122,7 +122,7 @@ public class CreateAndUseTest extends Fu
     
     getConnector().tableOperations().create("t3");
     getConnector().tableOperations().addSplits("t3", splits);
-    BatchScanner bs = getConnector().createBatchScanner("t3", Constants.NO_AUTHS, 3);
+    BatchScanner bs = getConnector().createBatchScanner("t3", Authorizations.EMPTY, 3);
     bs.setRanges(ranges);
     count = 0;
     for (Entry<Key,Value> entry : bs) {