You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/02/28 23:31:03 UTC

svn commit: r1451401 [10/11] - in /accumulo/branches/1.5: ./ assemble/ core/ core/src/main/java/org/apache/accumulo/core/cli/ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/ja...

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java Thu Feb 28 22:31:00 2013
@@ -28,7 +28,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -223,7 +223,7 @@ public class ZKPermHandler implements Pe
   }
   
   @Override
-  public void initializeSecurity(Credential itw, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(TCredentials itw, String rootuser) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     
     // create the root user with all system privileges, no table privileges, and no record-level authorizations

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Thu Feb 28 22:31:00 2013
@@ -81,7 +81,7 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.MetadataTable.DataFileValue;
@@ -686,7 +686,7 @@ public class Tablet {
       }
       
       synchronized (bulkFileImportLock) {
-        Credential auths = SecurityConstants.getSystemCredentials();
+        TCredentials auths = SecurityConstants.getSystemCredentials();
         Connector conn;
         try {
           conn = HdfsZooInstance.getInstance().getConnector(auths.getPrincipal(), CredentialHelper.extractToken(auths));
@@ -851,7 +851,7 @@ public class Tablet {
         // need to write to !METADATA before writing to walog, when things are done in the reverse order
         // data could be lost... the minor compaction start even should be written before the following metadata
         // write is made
-        Credential creds = SecurityConstants.getSystemCredentials();
+        TCredentials creds = SecurityConstants.getSystemCredentials();
         
         synchronized (timeLock) {
           if (commitSession.getMaxCommittedTime() > persistedTime)
@@ -2278,7 +2278,7 @@ public class Tablet {
       }
       
       if (updateMetadata) {
-        Credential creds = SecurityConstants.getSystemCredentials();
+        TCredentials creds = SecurityConstants.getSystemCredentials();
         // if multiple threads were allowed to update this outside of a sync block, then it would be
         // a race condition
         MetadataTable.updateTabletFlushID(extent, tableFlushID, creds, tabletServer.getLock());

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Thu Feb 28 22:31:00 2013
@@ -104,7 +104,7 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
@@ -579,12 +579,12 @@ public class TabletServer extends Abstra
   
   static class TservConstraintEnv implements Environment {
     
-    private Credential credentials;
+    private TCredentials credentials;
     private SecurityOperation security;
     private Authorizations auths;
     private KeyExtent ke;
     
-    TservConstraintEnv(SecurityOperation secOp, Credential credentials) {
+    TservConstraintEnv(SecurityOperation secOp, TCredentials credentials) {
       this.security = secOp;
       this.credentials = credentials;
     }
@@ -720,7 +720,7 @@ public class TabletServer extends Abstra
     Map<KeyExtent,Long> failures = new HashMap<KeyExtent,Long>();
     HashMap<KeyExtent,SecurityErrorCode> authFailures = new HashMap<KeyExtent,SecurityErrorCode>();
     public Violations violations;
-    public Credential credentials;
+    public TCredentials credentials;
     public long totalUpdates = 0;
     public long flushTime = 0;
     Stat prepareTimes = new Stat();
@@ -877,7 +877,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, Credential credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
+    public List<TKeyExtent> bulkImport(TInfo tinfo, TCredentials credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime)
         throws ThriftSecurityException {
 
       if (!security.canPerformSystemActions(credentials))
@@ -1095,7 +1095,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialScan startScan(TInfo tinfo, Credential credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
+    public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated)
         throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException {
       
@@ -1246,7 +1246,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public InitialMultiScan startMultiScan(TInfo tinfo, Credential credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
+    public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) throws ThriftSecurityException {
       // find all of the tables that need to be scanned
       HashSet<String> tables = new HashSet<String>();
@@ -1370,7 +1370,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, Credential credentials) throws ThriftSecurityException {
+    public long startUpdate(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
       // Make sure user is real
       
       security.authenticateUser(credentials, credentials);
@@ -1643,7 +1643,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void update(TInfo tinfo, Credential credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
+    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
         ConstraintViolationException, ThriftSecurityException {
 
       if (!security.canWrite(credentials, new String(tkeyExtent.getTable())))
@@ -1692,7 +1692,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void splitTablet(TInfo tinfo, Credential credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
+    public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint)
         throws NotServingTabletException, ThriftSecurityException {
       
       String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table));
@@ -1719,12 +1719,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletServerStatus getTabletServerStatus(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
+    public TabletServerStatus getTabletServerStatus(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return getStats(sessionManager.getActiveScansPerTable());
     }
     
     @Override
-    public List<TabletStats> getTabletStats(TInfo tinfo, Credential credentials, String tableId) throws ThriftSecurityException, TException {
+    public List<TabletStats> getTabletStats(TInfo tinfo, TCredentials credentials, String tableId) throws ThriftSecurityException, TException {
       TreeMap<KeyExtent,Tablet> onlineTabletsCopy;
       synchronized (onlineTablets) {
         onlineTabletsCopy = new TreeMap<KeyExtent,Tablet>(onlineTablets);
@@ -1750,7 +1750,7 @@ public class TabletServer extends Abstra
     
     private ZooCache masterLockCache = new ZooCache();
     
-    private void checkPermission(Credential credentials, String lock, boolean requiresSystemPermission, final String request)
+    private void checkPermission(TCredentials credentials, String lock, boolean requiresSystemPermission, final String request)
         throws ThriftSecurityException {
       if (requiresSystemPermission) {
         boolean fatal = false;
@@ -1814,7 +1814,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void loadTablet(TInfo tinfo, Credential credentials, String lock, final TKeyExtent textent) {
+    public void loadTablet(TInfo tinfo, TCredentials credentials, String lock, final TKeyExtent textent) {
       
       try {
         checkPermission(credentials, lock, true, "loadTablet");
@@ -1881,7 +1881,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void unloadTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent, boolean save) {
+    public void unloadTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent textent, boolean save) {
       try {
         checkPermission(credentials, lock, true, "unloadTablet");
       } catch (ThriftSecurityException e) {
@@ -1895,7 +1895,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flush(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
+    public void flush(TInfo tinfo, TCredentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) {
       try {
         checkPermission(credentials, lock, true, "flush");
       } catch (ThriftSecurityException e) {
@@ -1932,7 +1932,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void flushTablet(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent) throws TException {
+    public void flushTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent textent) throws TException {
       try {
         checkPermission(credentials, lock, true, "flushTablet");
       } catch (ThriftSecurityException e) {
@@ -1952,7 +1952,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void halt(TInfo tinfo, Credential credentials, String lock) throws ThriftSecurityException {
+    public void halt(TInfo tinfo, TCredentials credentials, String lock) throws ThriftSecurityException {
       
         checkPermission(credentials, lock, true, "halt");
       
@@ -1972,7 +1972,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void fastHalt(TInfo info, Credential credentials, String lock) {
+    public void fastHalt(TInfo info, TCredentials credentials, String lock) {
       try {
         halt(info, credentials, lock);
       } catch (Exception e) {
@@ -1981,12 +1981,12 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public TabletStats getHistoricalStats(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
+    public TabletStats getHistoricalStats(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return statsKeeper.getTabletStats();
     }
     
     @Override
-    public List<ActiveScan> getActiveScans(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
+    public List<ActiveScan> getActiveScans(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, true, "getScans");
       } catch (ThriftSecurityException e) {
@@ -1998,7 +1998,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void chop(TInfo tinfo, Credential credentials, String lock, TKeyExtent textent) throws TException {
+    public void chop(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent textent) throws TException {
       try {
         checkPermission(credentials, lock, true, "chop");
       } catch (ThriftSecurityException e) {
@@ -2015,7 +2015,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public void compact(TInfo tinfo, Credential credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
+    public void compact(TInfo tinfo, TCredentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow)
         throws TException {
       try {
         checkPermission(credentials, lock, true, "compact");
@@ -2057,7 +2057,7 @@ public class TabletServer extends Abstra
      * org.apache.accumulo.core.security.thrift.Credentials, java.util.List)
      */
     @Override
-    public void removeLogs(TInfo tinfo, Credential credentials, List<String> filenames) throws TException {
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
       String myname = getClientAddressString();
       myname = myname.replace(':', '+');
       Path logDir = new Path(Constants.getWalDirectory(acuConf), myname);
@@ -2107,7 +2107,7 @@ public class TabletServer extends Abstra
     }
     
     @Override
-    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
+    public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, true, "getActiveCompactions");
       } catch (ThriftSecurityException e) {

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java Thu Feb 28 22:31:00 2013
@@ -40,8 +40,6 @@ import org.apache.accumulo.core.conf.Acc
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.crypto.CryptoModule;
-import org.apache.accumulo.core.security.crypto.CryptoModuleFactory;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.server.logger.LogFileKey;
@@ -80,7 +78,7 @@ public class DfsLogger {
     Set<TServerInstance> getCurrentTServers();
   }
   
-  private LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new LinkedBlockingQueue<DfsLogger.LogWork>();
+  private final LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new LinkedBlockingQueue<DfsLogger.LogWork>();
   
   private final Object closeLock = new Object();
   
@@ -151,7 +149,7 @@ public class DfsLogger {
   }
   
   public static class LoggerOperation {
-    private LogWork work;
+    private final LogWork work;
     
     public LoggerOperation(LogWork work) {
       this.work = work;
@@ -201,7 +199,7 @@ public class DfsLogger {
     return getFileName().hashCode();
   }
   
-  private ServerResources conf;
+  private final ServerResources conf;
   private FSDataOutputStream logFile;
   private DataOutputStream encryptingLogFile = null;
   private Path logPath;
@@ -217,7 +215,7 @@ public class DfsLogger {
     this.logPath = new Path(Constants.getWalDirectory(conf.getConfiguration()), filename);
   }
   
-  public static FSDataInputStream readHeader(FileSystem fs, Path path, Map<String, String> opts) throws IOException {
+  public static FSDataInputStream readHeader(FileSystem fs, Path path, Map<String,String> opts) throws IOException {
     FSDataInputStream file = fs.open(path);
     try {
       byte[] magic = LOG_FILE_HEADER_V2.getBytes();
@@ -252,7 +250,7 @@ public class DfsLogger {
       FileSystem fs = conf.getFileSystem();
       short replication = (short) conf.getConfiguration().getCount(Property.TSERV_WAL_REPLICATION);
       if (replication == 0)
-        replication = (short) fs.getDefaultReplication();
+        replication = fs.getDefaultReplication();
       long blockSize = conf.getConfiguration().getMemoryInBytes(Property.TSERV_WAL_BLOCKSIZE);
       if (blockSize == 0)
         blockSize = (long) (conf.getConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE) * 1.1);
@@ -263,7 +261,8 @@ public class DfsLogger {
       
       // Initialize the crypto operations.
       @SuppressWarnings("deprecation")
-      CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(conf.getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
+      org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory.getCryptoModule(conf
+          .getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
       
       // Initialize the log file with a header and the crypto params used to set up this log file.
       logFile.write(LOG_FILE_HEADER_V2.getBytes());

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java Thu Feb 28 22:31:00 2013
@@ -34,8 +34,6 @@ import org.apache.accumulo.core.client.I
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.RecoveryStatus;
-import org.apache.accumulo.core.security.crypto.CryptoModule;
-import org.apache.accumulo.core.security.crypto.CryptoModuleFactory;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -104,7 +102,7 @@ public class LogSorter {
       String formerThreadName = Thread.currentThread().getName();
       int part = 0;
       try {
-
+        
         // the following call does not throw an exception if the file/dir does not exist
         fs.delete(new Path(destPath), true);
         
@@ -137,7 +135,8 @@ public class LogSorter {
           }
           
           @SuppressWarnings("deprecation")
-          CryptoModule cryptoOps = CryptoModuleFactory.getCryptoModule(cryptoModuleName);
+          org.apache.accumulo.core.security.crypto.CryptoModule cryptoOps = org.apache.accumulo.core.security.crypto.CryptoModuleFactory
+              .getCryptoModule(cryptoModuleName);
           @SuppressWarnings("deprecation")
           InputStream decryptingInputStream = cryptoOps.getDecryptingInputStream(input, cryptoOpts);
           
@@ -232,7 +231,7 @@ public class LogSorter {
   }
   
   ThreadPoolExecutor threadPool;
-  private Instance instance;
+  private final Instance instance;
   
   public LogSorter(Instance instance, FileSystem fs, AccumuloConfiguration conf) {
     this.instance = instance;

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java Thu Feb 28 22:31:00 2013
@@ -17,12 +17,12 @@
 package org.apache.accumulo.server.tabletserver.mastermessage;
 
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.thrift.TException;
 
 public interface MasterMessage {
   
-  void send(Credential info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException;
+  void send(TCredentials info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException;
   
 }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java Thu Feb 28 22:31:00 2013
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.TabletSplit;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
@@ -45,7 +45,7 @@ public class SplitReportMessage implemen
     extents.put(ne2, np2);
   }
   
-  public void send(Credential credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException {
+  public void send(TCredentials credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException {
     TabletSplit split = new TabletSplit();
     split.oldTablet = old_extent.toThrift();
     split.newTablets = Translator.translate(extents.keySet(), Translator.KET);

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java Thu Feb 28 22:31:00 2013
@@ -20,7 +20,7 @@ import org.apache.accumulo.trace.instrum
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.thrift.TException;
 
@@ -34,7 +34,7 @@ public class TabletStatusMessage impleme
     this.status = status;
   }
   
-  public void send(Credential auth, String serverName, Iface client) throws TException, ThriftSecurityException {
+  public void send(TCredentials auth, String serverName, Iface client) throws TException, ThriftSecurityException {
     client.reportTabletStatus(Tracer.traceInfo(), auth, serverName, status, extent.toThrift());
   }
 }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/trace/TraceFileSystem.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/trace/TraceFileSystem.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/trace/TraceFileSystem.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/trace/TraceFileSystem.java Thu Feb 28 22:31:00 2013
@@ -19,9 +19,9 @@ package org.apache.accumulo.server.trace
 import java.io.IOException;
 import java.net.URI;
 
+import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
@@ -266,14 +266,13 @@ public class TraceFileSystem extends Fil
     }
   }
   
-  @SuppressWarnings("deprecation")
   @Override
   public boolean isDirectory(Path f) throws IOException {
     Span span = Trace.start("isDirectory");
     if (Trace.isTracing())
       span.data("path", f.toString());
     try {
-      return impl.isDirectory(f);
+      return impl.getFileStatus(f).isDir();
     } finally {
       span.stop();
     }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/Admin.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/Admin.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/Admin.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/Admin.java Thu Feb 28 22:31:00 2013
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
@@ -81,7 +81,7 @@ public class Admin {
     Instance instance = opts.getInstance();
       
     try {
-      Credential creds;
+      TCredentials creds;
       if (opts.getToken() == null) {
         creds = SecurityConstants.getSystemCredentials();
       } else {
@@ -103,7 +103,7 @@ public class Admin {
     }
   }
   
-  private static void stopServer(Instance instance, final Credential credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
+  private static void stopServer(Instance instance, final TCredentials credentials, final boolean tabletServersToo) throws AccumuloException, AccumuloSecurityException {
     MasterClient.execute(HdfsZooInstance.getInstance(), new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
@@ -112,7 +112,7 @@ public class Admin {
     });
   }
   
-  private static void stopTabletServer(Instance instance, final Credential creds, List<String> servers, final boolean force) throws AccumuloException, AccumuloSecurityException {
+  private static void stopTabletServer(Instance instance, final TCredentials creds, List<String> servers, final boolean force) throws AccumuloException, AccumuloSecurityException {
     for (String server : servers) {
       InetSocketAddress address = AddressUtil.parseAddress(server, Property.TSERV_CLIENTPORT);
       final String finalServer = org.apache.accumulo.core.util.AddressUtil.toString(address);

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java Thu Feb 28 22:31:00 2013
@@ -59,7 +59,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -95,7 +95,7 @@ import org.apache.zookeeper.KeeperExcept
 public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
   
   private static final Text EMPTY_TEXT = new Text();
-  private static Map<Credential,Writer> metadata_tables = new HashMap<Credential,Writer>();
+  private static Map<TCredentials,Writer> metadata_tables = new HashMap<TCredentials,Writer>();
   private static final Logger log = Logger.getLogger(MetadataTable.class);
   
   private static final int SAVE_ROOT_TABLET_RETRIES = 3;
@@ -104,7 +104,7 @@ public class MetadataTable extends org.a
     
   }
   
-  public synchronized static Writer getMetadataTable(Credential credentials) {
+  public synchronized static Writer getMetadataTable(TCredentials credentials) {
     Writer metadataTable = metadata_tables.get(credentials);
     if (metadataTable == null) {
       metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID);
@@ -117,11 +117,11 @@ public class MetadataTable extends org.a
     Constants.METADATA_LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/").getBytes()));
   }
   
-  public static void update(Credential credentials, Mutation m) {
+  public static void update(TCredentials credentials, Mutation m) {
     update(credentials, null, m);
   }
   
-  public static void update(Credential credentials, ZooLock zooLock, Mutation m) {
+  public static void update(TCredentials credentials, ZooLock zooLock, Mutation m) {
     Writer t;
     t = getMetadataTable(credentials);
     if (zooLock != null)
@@ -155,7 +155,7 @@ public class MetadataTable extends org.a
    * @param flushId
    * 
    */
-  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, Credential credentials,
+  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, TCredentials credentials,
       Set<String> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
       if (unusedWalLogs != null) {
@@ -232,7 +232,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credential credentials, ZooLock zooLock) {
+  public static void updateTabletFlushID(KeyExtent extent, long flushID, TCredentials credentials, ZooLock zooLock) {
     if (!extent.isRootTablet()) {
       Mutation m = new Mutation(extent.getMetadataEntry());
       Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
@@ -240,7 +240,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credential credentials, ZooLock zooLock) {
+  public static void updateTabletCompactID(KeyExtent extent, long compactID, TCredentials credentials, ZooLock zooLock) {
     if (!extent.isRootTablet()) {
       Mutation m = new Mutation(extent.getMetadataEntry());
       Constants.METADATA_COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
@@ -248,7 +248,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, Credential credentials, ZooLock zooLock) {
+  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, TCredentials credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     byte[] tidBytes = Long.toString(tid).getBytes();
     
@@ -261,7 +261,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addTablet(KeyExtent extent, String path, Credential credentials, char timeType, ZooLock lock) {
+  public static void addTablet(KeyExtent extent, String path, TCredentials credentials, char timeType, ZooLock lock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -270,7 +270,7 @@ public class MetadataTable extends org.a
     update(credentials, lock, m);
   }
   
-  public static void updateTabletPrevEndRow(KeyExtent extent, Credential credentials) {
+  public static void updateTabletPrevEndRow(KeyExtent extent, TCredentials credentials) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     update(credentials, m);
   }
@@ -324,7 +324,7 @@ public class MetadataTable extends org.a
     return results;
   }
   
-  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, Credential credentials, Text startRow, SortedMap<Key,Value> results,
+  public static boolean getBatchFromRootTablet(AccumuloConfiguration conf, TCredentials credentials, Text startRow, SortedMap<Key,Value> results,
       SortedSet<Column> columns, boolean skipStartRow, int size) throws AccumuloSecurityException {
     while (true) {
       try {
@@ -355,7 +355,7 @@ public class MetadataTable extends org.a
     return false;
   }
   
-  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, Credential credentials) {
+  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, TCredentials credentials) {
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
     
     Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
@@ -379,7 +379,7 @@ public class MetadataTable extends org.a
   }
   
   public static void addNewTablet(KeyExtent extent, String path, TServerInstance location, Map<String,DataFileValue> datafileSizes,
-      Map<String,Long> bulkLoadedFiles, Credential credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
+      Map<String,Long> bulkLoadedFiles, TCredentials credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation();
     
     Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
@@ -406,7 +406,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credential credentials, ZooLock zooLock) {
+  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, TCredentials credentials, ZooLock zooLock) {
     Mutation m = extent.getPrevRowUpdateMutation(); //
     
     Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
@@ -416,7 +416,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credential credentials,
+  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
       ZooLock zooLock) {
     Mutation m = new Mutation(metadataEntry);
     Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
@@ -434,18 +434,18 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, Credential credentials,
+  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
       ZooLock zooLock) {
     finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
   }
   
   public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
-      DataFileValue size, Credential credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
+      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
     replaceDatafiles(extent, datafilesToDelete, scanFiles, path, compactionId, size, credentials, address, lastLocation, zooLock, true);
   }
   
   public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
-      DataFileValue size, Credential credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
+      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
     
     if (insertDeleteFlags) {
       // add delete flags for those paths before the data file reference is removed
@@ -477,7 +477,7 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, Credential credentials) {
+  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, TCredentials credentials) {
     
     String tableId = extent.getTableId().toString();
     
@@ -501,7 +501,7 @@ public class MetadataTable extends org.a
     return delFlag;
   }
   
-  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, Credential credentials, ZooLock zooLock) {
+  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, TCredentials credentials, ZooLock zooLock) {
     Mutation m = new Mutation(extent.getMetadataEntry());
     
     for (String pathToRemove : scanFiles)
@@ -510,16 +510,16 @@ public class MetadataTable extends org.a
     update(credentials, zooLock, m);
   }
   
-  public static void getTabletAndPrevTabletKeyValues(SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
+  public static void getTabletAndPrevTabletKeyValues(SortedMap<Key,Value> tkv, KeyExtent ke, List<ColumnFQ> columns, TCredentials credentials) {
     getTabletAndPrevTabletKeyValues(HdfsZooInstance.getInstance(), tkv, ke, columns, credentials);
   }
   
-  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, Credential credentials) {
+  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(KeyExtent ke, List<ColumnFQ> columns, TCredentials credentials) {
     return getTabletEntries(HdfsZooInstance.getInstance(), ke, columns, credentials);
   }
   
   private static KeyExtent fixSplit(Text table, Text metadataEntry, Text metadataPrevEndRow, Value oper, double splitRatio, TServerInstance tserver,
-      Credential credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
+      TCredentials credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
     if (metadataPrevEndRow == null)
       // something is wrong, this should not happen... if a tablet is split, it will always have a
       // prev end row....
@@ -615,7 +615,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, Credential credentials, ZooLock lock)
+  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, TCredentials credentials, ZooLock lock)
       throws AccumuloException {
     log.warn("Incomplete split " + metadataEntry + " attempting to fix");
     
@@ -659,7 +659,7 @@ public class MetadataTable extends org.a
     return fixSplit(table, metadataEntry, metadataPrevEndRow, oper, splitRatio, tserver, credentials, time.toString(), initFlushID, initCompactID, lock);
   }
   
-  public static void deleteTable(String tableId, boolean insertDeletes, Credential credentials, ZooLock lock) throws AccumuloException {
+  public static void deleteTable(String tableId, boolean insertDeletes, TCredentials credentials, ZooLock lock) throws AccumuloException {
     Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
     Text tableIdText = new Text(tableId);
     BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, new BatchWriterConfig().setMaxMemory(1000000)
@@ -772,7 +772,7 @@ public class MetadataTable extends org.a
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_WALOGS;
   }
   
-  public static void addLogEntry(Credential credentials, LogEntry entry, ZooLock zooLock) {
+  public static void addLogEntry(TCredentials credentials, LogEntry entry, ZooLock zooLock) {
     if (entry.extent.isRootTablet()) {
       String root = getZookeeperLogLocation();
       while (true) {
@@ -811,7 +811,7 @@ public class MetadataTable extends org.a
     return e;
   }
   
-  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(Credential credentials, KeyExtent extent) throws KeeperException,
+  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(TCredentials credentials, KeyExtent extent) throws KeeperException,
       InterruptedException, IOException {
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
@@ -854,7 +854,7 @@ public class MetadataTable extends org.a
     return new Pair<List<LogEntry>,SortedMap<String,DataFileValue>>(result, sizes);
   }
   
-  public static List<LogEntry> getLogEntries(Credential credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
+  public static List<LogEntry> getLogEntries(TCredentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
     log.info("Scanning logging entries for " + extent);
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
@@ -899,7 +899,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  private static Scanner getTabletLogScanner(Credential credentials, KeyExtent extent) {
+  private static Scanner getTabletLogScanner(TCredentials credentials, KeyExtent extent) {
     Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
     scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
     Text start = extent.getMetadataEntry();
@@ -914,7 +914,7 @@ public class MetadataTable extends org.a
     Iterator<LogEntry> rootTabletEntries = null;
     Iterator<Entry<Key,Value>> metadataEntries = null;
     
-    LogEntryIterator(Credential creds) throws IOException, KeeperException, InterruptedException {
+    LogEntryIterator(TCredentials creds) throws IOException, KeeperException, InterruptedException {
       rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
       try {
         Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), CredentialHelper.extractToken(creds))
@@ -946,7 +946,7 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Iterator<LogEntry> getLogEntries(Credential creds) throws IOException, KeeperException, InterruptedException {
+  public static Iterator<LogEntry> getLogEntries(TCredentials creds) throws IOException, KeeperException, InterruptedException {
     return new LogEntryIterator(creds);
   }
   
@@ -1211,11 +1211,11 @@ public class MetadataTable extends org.a
     }
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(Credential credentials, KeyExtent extent) {
+  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, KeyExtent extent) {
     return getBulkFilesLoaded(credentials, extent.getMetadataEntry());
   }
   
-  public static Map<String,Long> getBulkFilesLoaded(Credential credentials, Text metadataRow) {
+  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, Text metadataRow) {
     
     Map<String,Long> ret = new HashMap<String,Long>();
     

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java Thu Feb 28 22:31:00 2013
@@ -44,7 +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.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -150,7 +150,7 @@ public class VerifyTabletAssignments {
     }
   }
   
-  private static void checkTabletServer(AccumuloConfiguration conf, Credential st, Entry<String,List<KeyExtent>> entry,
+  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);
     

Modified: accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java (original)
+++ accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java Thu Feb 28 22:31:00 2013
@@ -32,7 +32,7 @@ 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.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.fs.FileSystem;
@@ -90,7 +90,7 @@ public class TestConfirmDeletes {
   }
   
   private void test1(String[] metadata, String[] deletes, int expectedInitial, int expected) throws Exception {
-    Credential auth = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
+    TCredentials auth = CredentialHelper.create("root", new PasswordToken(new byte[0]), "instance");
     
     Instance instance = new MockInstance();
     FileSystem fs = FileSystem.getLocal(CachedConfiguration.getInstance());
@@ -106,9 +106,10 @@ public class TestConfirmDeletes {
   }
   
   private void load(Instance instance, String[] metadata, String[] deletes) throws Exception {
-    Credential credential = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
+    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);
+    Scanner scanner = instance.getConnector(credential.getPrincipal(), CredentialHelper.extractToken(credential)).createScanner(Constants.METADATA_TABLE_NAME,
+        Constants.NO_AUTHS);
     int count = 0;
     for (@SuppressWarnings("unused")
     Entry<Key,Value> entry : scanner) {

Modified: accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java (original)
+++ accumulo/branches/1.5/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java Thu Feb 28 22:31:00 2013
@@ -35,7 +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.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.CurrentState;
@@ -85,7 +85,7 @@ public class TestMergeState {
     bw.addMutation(m);
     bw.close();
   }
-
+  
   @Test
   public void test() throws Exception {
     Instance instance = new MockInstance();
@@ -113,8 +113,8 @@ public class TestMergeState {
     
     // Read out the TabletLocationStates
     MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
-    Credential auths = CredentialHelper.create("root", new PasswordToken().setPassword(new byte[0]), "instance");
-
+    TCredentials auths = CredentialHelper.create("root", new PasswordToken(new byte[0]), "instance");
+    
     // Verify the tablet state: hosted, and count
     MetaDataStateStore metaDataStateStore = new MetaDataStateStore(instance, auths, state);
     int count = 0;
@@ -161,10 +161,10 @@ public class TestMergeState {
     m = tablet.getPrevRowUpdateMutation();
     Constants.METADATA_CHOPPED_COLUMN.put(m, new Value("junk".getBytes()));
     update(connector, m);
-
+    
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
-
+    
     // take it offline
     m = tablet.getPrevRowUpdateMutation();
     Collection<Collection<String>> walogs = Collections.emptyList();
@@ -173,9 +173,9 @@ public class TestMergeState {
     // now we can split
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.MERGING, stats.nextMergeState(connector, state));
-
+    
   }
-
+  
   /**
    * @param state
    * @param metaDataStateStore

Propchange: accumulo/branches/1.5/src/
------------------------------------------------------------------------------
  Merged /accumulo/branches/ACCUMULO-259-polishing/src:r1449464-1451386

Modified: accumulo/branches/1.5/start/src/main/java/org/apache/accumulo/start/classloader/vfs/providers/HdfsRandomAccessContent.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/start/src/main/java/org/apache/accumulo/start/classloader/vfs/providers/HdfsRandomAccessContent.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/start/src/main/java/org/apache/accumulo/start/classloader/vfs/providers/HdfsRandomAccessContent.java (original)
+++ accumulo/branches/1.5/start/src/main/java/org/apache/accumulo/start/classloader/vfs/providers/HdfsRandomAccessContent.java Thu Feb 28 22:31:00 2013
@@ -16,8 +16,10 @@
  */
 package org.apache.accumulo.start.classloader.vfs.providers;
 
+import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
 
 import org.apache.commons.vfs2.RandomAccessContent;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -25,338 +27,302 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Provides random access to content in an HdfsFileObject. Currently this only supports read operations. All write
- * operations throw an {@link UnsupportedOperationException}.
+ * Provides random access to content in an HdfsFileObject. Currently this only supports read operations. All write operations throw an
+ * {@link UnsupportedOperationException}.
  * 
  * @since 2.1
  */
-public class HdfsRandomAccessContent implements RandomAccessContent
-{
-    private final FileSystem fs;
-    private final Path path;
-    private final FSDataInputStream fis;
-
-    /**
-     * 
-     * @param path
-     *            A Hadoop Path
-     * @param fs
-     *            A Hadoop FileSystem
-     * @throws IOException
-     *             when the path cannot be processed.
-     */
-    public HdfsRandomAccessContent(final Path path, final FileSystem fs) throws IOException
-    {
-        this.fs = fs;
-        this.path = path;
-        this.fis = this.fs.open(this.path);
-    }
-
-    /**
-     * @see org.apache.commons.vfs2.RandomAccessContent#close()
-     */
-    @Override
-    public void close() throws IOException
-    {
-        this.fis.close();
-    }
-
-    /**
-     * @see org.apache.commons.vfs2.RandomAccessContent#getFilePointer()
-     */
-    @Override
-    public long getFilePointer() throws IOException
-    {
-        return this.fis.getPos();
-    }
-
-    /**
-     * @see org.apache.commons.vfs2.RandomAccessContent#getInputStream()
-     */
-    @Override
-    public InputStream getInputStream() throws IOException
-    {
-        return this.fis;
-    }
-
-    /**
-     * @see org.apache.commons.vfs2.RandomAccessContent#length()
-     */
-    @Override
-    public long length() throws IOException
-    {
-        return this.fs.getFileStatus(this.path).getLen();
-    }
-
-    /**
-     * @see java.io.DataInput#readBoolean()
-     */
-    @Override
-    public boolean readBoolean() throws IOException
-    {
-        return this.fis.readBoolean();
-    }
-
-    /**
-     * @see java.io.DataInput#readByte()
-     */
-    @Override
-    public byte readByte() throws IOException
-    {
-        return this.fis.readByte();
-    }
-
-    /**
-     * @see java.io.DataInput#readChar()
-     */
-    @Override
-    public char readChar() throws IOException
-    {
-        return this.fis.readChar();
-    }
-
-    /**
-     * @see java.io.DataInput#readDouble()
-     */
-    @Override
-    public double readDouble() throws IOException
-    {
-        return this.fis.readDouble();
-    }
-
-    /**
-     * @see java.io.DataInput#readFloat()
-     */
-    @Override
-    public float readFloat() throws IOException
-    {
-        return this.fis.readFloat();
-    }
-
-    /**
-     * @see java.io.DataInput#readFully(byte[])
-     */
-    @Override
-    public void readFully(final byte[] b) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataInput#readFully(byte[], int, int)
-     */
-    @Override
-    public void readFully(final byte[] b, final int off, final int len) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataInput#readInt()
-     */
-    @Override
-    public int readInt() throws IOException
-    {
-        return this.fis.readInt();
-    }
-
-    /**
-     * @see java.io.DataInput#readLine()
-     */
-    @Override
-    @SuppressWarnings("deprecation")
-    public String readLine() throws IOException
-    {
-        return this.fis.readLine();
-    }
-
-    /**
-     * @see java.io.DataInput#readLong()
-     */
-    @Override
-    public long readLong() throws IOException
-    {
-        return this.fis.readLong();
-    }
-
-    /**
-     * @see java.io.DataInput#readShort()
-     */
-    @Override
-    public short readShort() throws IOException
-    {
-        return this.fis.readShort();
-    }
-
-    /**
-     * @see java.io.DataInput#readUnsignedByte()
-     */
-    @Override
-    public int readUnsignedByte() throws IOException
-    {
-        return this.fis.readUnsignedByte();
-    }
-
-    /**
-     * @see java.io.DataInput#readUnsignedShort()
-     */
-    @Override
-    public int readUnsignedShort() throws IOException
-    {
-        return this.fis.readUnsignedShort();
-    }
-
-    /**
-     * @see java.io.DataInput#readUTF()
-     */
-    @Override
-    public String readUTF() throws IOException
-    {
-        return this.fis.readUTF();
-    }
-
-    /**
-     * @see org.apache.commons.vfs2.RandomAccessContent#seek(long)
-     */
-    @Override
-    public void seek(final long pos) throws IOException
-    {
-        this.fis.seek(pos);
-    }
-
-    /**
-     * @see java.io.DataInput#skipBytes(int)
-     */
-    @Override
-    public int skipBytes(final int n) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#write(byte[])
-     */
-    @Override
-    public void write(final byte[] b) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#write(byte[], int, int)
-     */
-    @Override
-    public void write(final byte[] b, final int off, final int len) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#write(int)
-     */
-    @Override
-    public void write(final int b) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeBoolean(boolean)
-     */
-    @Override
-    public void writeBoolean(final boolean v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeByte(int)
-     */
-    @Override
-    public void writeByte(final int v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeBytes(java.lang.String)
-     */
-    @Override
-    public void writeBytes(final String s) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeChar(int)
-     */
-    @Override
-    public void writeChar(final int v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeChars(java.lang.String)
-     */
-    @Override
-    public void writeChars(final String s) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeDouble(double)
-     */
-    @Override
-    public void writeDouble(final double v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeFloat(float)
-     */
-    @Override
-    public void writeFloat(final float v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeInt(int)
-     */
-    @Override
-    public void writeInt(final int v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeLong(long)
-     */
-    @Override
-    public void writeLong(final long v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeShort(int)
-     */
-    @Override
-    public void writeShort(final int v) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @see java.io.DataOutput#writeUTF(java.lang.String)
-     */
-    @Override
-    public void writeUTF(final String s) throws IOException
-    {
-        throw new UnsupportedOperationException();
-    }
-
+public class HdfsRandomAccessContent implements RandomAccessContent {
+  private final FileSystem fs;
+  private final Path path;
+  private final FSDataInputStream fis;
+  
+  /**
+   * 
+   * @param path
+   *          A Hadoop Path
+   * @param fs
+   *          A Hadoop FileSystem
+   * @throws IOException
+   *           when the path cannot be processed.
+   */
+  public HdfsRandomAccessContent(final Path path, final FileSystem fs) throws IOException {
+    this.fs = fs;
+    this.path = path;
+    this.fis = this.fs.open(this.path);
+  }
+  
+  /**
+   * @see org.apache.commons.vfs2.RandomAccessContent#close()
+   */
+  @Override
+  public void close() throws IOException {
+    this.fis.close();
+  }
+  
+  /**
+   * @see org.apache.commons.vfs2.RandomAccessContent#getFilePointer()
+   */
+  @Override
+  public long getFilePointer() throws IOException {
+    return this.fis.getPos();
+  }
+  
+  /**
+   * @see org.apache.commons.vfs2.RandomAccessContent#getInputStream()
+   */
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return this.fis;
+  }
+  
+  /**
+   * @see org.apache.commons.vfs2.RandomAccessContent#length()
+   */
+  @Override
+  public long length() throws IOException {
+    return this.fs.getFileStatus(this.path).getLen();
+  }
+  
+  /**
+   * @see java.io.DataInput#readBoolean()
+   */
+  @Override
+  public boolean readBoolean() throws IOException {
+    return this.fis.readBoolean();
+  }
+  
+  /**
+   * @see java.io.DataInput#readByte()
+   */
+  @Override
+  public byte readByte() throws IOException {
+    return this.fis.readByte();
+  }
+  
+  /**
+   * @see java.io.DataInput#readChar()
+   */
+  @Override
+  public char readChar() throws IOException {
+    return this.fis.readChar();
+  }
+  
+  /**
+   * @see java.io.DataInput#readDouble()
+   */
+  @Override
+  public double readDouble() throws IOException {
+    return this.fis.readDouble();
+  }
+  
+  /**
+   * @see java.io.DataInput#readFloat()
+   */
+  @Override
+  public float readFloat() throws IOException {
+    return this.fis.readFloat();
+  }
+  
+  /**
+   * @see java.io.DataInput#readFully(byte[])
+   */
+  @Override
+  public void readFully(final byte[] b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataInput#readFully(byte[], int, int)
+   */
+  @Override
+  public void readFully(final byte[] b, final int off, final int len) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataInput#readInt()
+   */
+  @Override
+  public int readInt() throws IOException {
+    return this.fis.readInt();
+  }
+  
+  /**
+   * @see java.io.DataInput#readLine()
+   */
+  @Override
+  public String readLine() throws IOException {
+    BufferedReader d = new BufferedReader(new InputStreamReader(this.fis));
+    return d.readLine();
+  }
+  
+  /**
+   * @see java.io.DataInput#readLong()
+   */
+  @Override
+  public long readLong() throws IOException {
+    return this.fis.readLong();
+  }
+  
+  /**
+   * @see java.io.DataInput#readShort()
+   */
+  @Override
+  public short readShort() throws IOException {
+    return this.fis.readShort();
+  }
+  
+  /**
+   * @see java.io.DataInput#readUnsignedByte()
+   */
+  @Override
+  public int readUnsignedByte() throws IOException {
+    return this.fis.readUnsignedByte();
+  }
+  
+  /**
+   * @see java.io.DataInput#readUnsignedShort()
+   */
+  @Override
+  public int readUnsignedShort() throws IOException {
+    return this.fis.readUnsignedShort();
+  }
+  
+  /**
+   * @see java.io.DataInput#readUTF()
+   */
+  @Override
+  public String readUTF() throws IOException {
+    return this.fis.readUTF();
+  }
+  
+  /**
+   * @see org.apache.commons.vfs2.RandomAccessContent#seek(long)
+   */
+  @Override
+  public void seek(final long pos) throws IOException {
+    this.fis.seek(pos);
+  }
+  
+  /**
+   * @see java.io.DataInput#skipBytes(int)
+   */
+  @Override
+  public int skipBytes(final int n) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#write(byte[])
+   */
+  @Override
+  public void write(final byte[] b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#write(byte[], int, int)
+   */
+  @Override
+  public void write(final byte[] b, final int off, final int len) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#write(int)
+   */
+  @Override
+  public void write(final int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeBoolean(boolean)
+   */
+  @Override
+  public void writeBoolean(final boolean v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeByte(int)
+   */
+  @Override
+  public void writeByte(final int v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeBytes(java.lang.String)
+   */
+  @Override
+  public void writeBytes(final String s) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeChar(int)
+   */
+  @Override
+  public void writeChar(final int v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeChars(java.lang.String)
+   */
+  @Override
+  public void writeChars(final String s) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeDouble(double)
+   */
+  @Override
+  public void writeDouble(final double v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeFloat(float)
+   */
+  @Override
+  public void writeFloat(final float v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeInt(int)
+   */
+  @Override
+  public void writeInt(final int v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeLong(long)
+   */
+  @Override
+  public void writeLong(final long v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeShort(int)
+   */
+  @Override
+  public void writeShort(final int v) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * @see java.io.DataOutput#writeUTF(java.lang.String)
+   */
+  @Override
+  public void writeUTF(final String s) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
 }

Modified: accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java (original)
+++ accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java Thu Feb 28 22:31:00 2013
@@ -35,7 +35,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.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.hadoop.io.Text;
@@ -43,7 +43,7 @@ import org.apache.hadoop.io.Text;
 import com.beust.jcommander.Parameter;
 
 public class QueryMetadataTable {
-  private static Credential credentials;
+  private static TCredentials credentials;
   
   static String location;
   

Modified: accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java (original)
+++ accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java Thu Feb 28 22:31:00 2013
@@ -21,9 +21,7 @@ import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.accumulo.core.cli.ScannerOpts;
-import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
@@ -33,14 +31,13 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.DistributedTrace;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 import com.beust.jcommander.Parameter;
 
-
 public class VerifyIngest {
   
   private static final Logger log = Logger.getLogger(VerifyIngest.class);
@@ -54,7 +51,7 @@ public class VerifyIngest {
   }
   
   public static class Opts extends TestIngest.Opts {
-    @Parameter(names="-useGet", description="fetches values one at a time, instead of scanning")
+    @Parameter(names = "-useGet", description = "fetches values one at a time, instead of scanning")
     boolean useGet = false;
   }
   

Modified: accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java (original)
+++ accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java Thu Feb 28 22:31:00 2013
@@ -21,8 +21,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.test.CreateRFiles;
@@ -64,8 +64,8 @@ public class BulkSplitOptimizationTest e
     
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
     fs.delete(new Path("/tmp/testmf"), true);
-    SecurityToken token = this.getToken();
-    CreateRFiles.main(new String[] { "--output", "tmp/testmf", "--numThreads", "8", "--start", "0", "--end", "100000", "--splits", "99"});
+    AuthenticationToken token = this.getToken();
+    CreateRFiles.main(new String[] {"--output", "tmp/testmf", "--numThreads", "8", "--start", "0", "--end", "100000", "--splits", "99"});
     
     bulkImport(fs, TABLE_NAME, "/tmp/testmf");
     
@@ -86,7 +86,7 @@ public class BulkSplitOptimizationTest e
     
     String passwd = "";
     if (token instanceof PasswordToken) {
-      passwd = new String(((PasswordToken)token).getPassword());
+      passwd = new String(((PasswordToken) token).getPassword());
     }
     VerifyIngest.main(new String[] {"--timestamp", "1", "--size", "50", "--random", "56", "--rows", "100000", "--start", "0", "--cols", "1", "-p", passwd});
     

Modified: accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java (original)
+++ accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java Thu Feb 28 22:31:00 2013
@@ -40,9 +40,8 @@ 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.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
@@ -54,7 +53,7 @@ import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
 
 public abstract class FunctionalTest {
-
+  
   public static Map<String,String> parseConfig(String... perTableConfigs) {
     
     TreeMap<String,String> config = new TreeMap<String,String>();
@@ -107,7 +106,7 @@ public abstract class FunctionalTest {
     
   }
   
-  private SecurityToken token = null;
+  private AuthenticationToken token = null;
   private String instanceName = "";
   private String principal = "";
   
@@ -119,11 +118,11 @@ public abstract class FunctionalTest {
     return principal;
   }
   
-  protected void setToken(SecurityToken token) {
+  protected void setToken(AuthenticationToken token) {
     this.token = token;
   }
   
-  protected SecurityToken getToken() {
+  protected AuthenticationToken getToken() {
     return token;
   }
   
@@ -239,27 +238,26 @@ public abstract class FunctionalTest {
   }
   
   static class Opts extends ClientOpts {
-    @Parameter(names="--classname", required=true, description="name of the class under test")
+    @Parameter(names = "--classname", required = true, description = "name of the class under test")
     String classname = null;
     
-    @Parameter(names="--opt", required=true, description="the options for test")
+    @Parameter(names = "--opt", required = true, description = "the options for test")
     String opt = null;
   }
   
-  
   public static void main(String[] args) throws Exception {
-    CredentialHelper.create("", new PasswordToken().setPassword(new byte[0]), "");
+    CredentialHelper.create("", new PasswordToken(new byte[0]), "");
     Opts opts = new Opts();
     opts.parseArgs(FunctionalTest.class.getName(), args);
     
     Class<? extends FunctionalTest> testClass = AccumuloVFSClassLoader.loadClass(opts.classname, FunctionalTest.class);
     FunctionalTest fTest = testClass.newInstance();
     
-    //fTest.setMaster(master);
+    // fTest.setMaster(master);
     fTest.setInstanceName(opts.instance);
     fTest.setPrincipal(opts.principal);
     fTest.setToken(opts.getToken());
-
+    
     if (opts.opt.equals("getConfig")) {
       Map<String,String> iconfig = fTest.getInitialConfig();
       System.out.println("{");
@@ -274,15 +272,15 @@ public abstract class FunctionalTest {
     } else if (opts.opt.equals("cleanup")) {
       fTest.cleanup();
     } else {
-    	printHelpAndExit("Unknown option: " + opts.opt);
+      printHelpAndExit("Unknown option: " + opts.opt);
     }
     
   }
-
+  
   static void printHelpAndExit(String message) {
-      System.out.println(message);
-      new JCommander(new Opts()).usage();
-      System.exit(1);
+    System.out.println(message);
+    new JCommander(new Opts()).usage();
+    System.exit(1);
   }
   
   static Mutation nm(String row, String cf, String cq, Value value) {

Modified: accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java (original)
+++ accumulo/branches/1.5/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java Thu Feb 28 22:31:00 2013
@@ -47,14 +47,13 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 public class PermissionsTest {
   private static final String TEST_USER = "test_user";
-  private static final SecurityToken TEST_PASS = new PasswordToken().setPassword("test_password".getBytes());
+  private static final PasswordToken TEST_PASS = new PasswordToken("test_password");
   
   public static class SystemPermissionsTest extends FunctionalTest {
     private static final Logger log = Logger.getLogger(SystemPermissionsTest.class);
@@ -78,7 +77,7 @@ public class PermissionsTest {
       verifyHasOnlyTheseSystemPermissions(getConnector(), getConnector().whoami(), SystemPermission.values());
       
       // create the test user
-      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS);
+      getConnector().securityOperations().createLocalUser(TEST_USER, TEST_PASS);
       Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS);
       verifyHasNoSystemPermissions(getConnector(), TEST_USER, SystemPermission.values());
       
@@ -169,29 +168,27 @@ public class PermissionsTest {
         case CREATE_USER:
           user = "__CREATE_USER_WITHOUT_PERM_TEST__";
           try {
-            test_user_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
+            test_user_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
             throw new IllegalStateException("Should NOT be able to create a user");
           } catch (AccumuloSecurityException e) {
-            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
-                || root_conn.securityOperations().authenticateUser(user, password.getBytes()))
+            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
               throw e;
           }
           break;
         case DROP_USER:
           user = "__DROP_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
+          root_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
           try {
-            test_user_conn.securityOperations().dropUser(user);
+            test_user_conn.securityOperations().dropLocalUser(user);
             throw new IllegalStateException("Should NOT be able to delete a user");
           } catch (AccumuloSecurityException e) {
-            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED
-                || !root_conn.securityOperations().authenticateUser(user, password.getBytes()))
+            if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
               throw e;
           }
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITHOUT_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
+          root_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
           try {
             test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
             throw new IllegalStateException("Should NOT be able to alter a user");
@@ -246,20 +243,20 @@ public class PermissionsTest {
           break;
         case CREATE_USER:
           user = "__CREATE_USER_WITH_PERM_TEST__";
-          test_user_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
-          if (!root_conn.securityOperations().authenticateUser(user, password.getBytes()))
+          test_user_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
+          if (!root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
             throw new IllegalStateException("Should be able to create a user");
           break;
         case DROP_USER:
           user = "__DROP_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
-          test_user_conn.securityOperations().dropUser(user);
-          if (root_conn.securityOperations().authenticateUser(user, password.getBytes()))
+          root_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
+          test_user_conn.securityOperations().dropLocalUser(user);
+          if (root_conn.securityOperations().authenticateUser(user, new PasswordToken(password)))
             throw new IllegalStateException("Should be able to delete a user");
           break;
         case ALTER_USER:
           user = "__ALTER_USER_WITH_PERM_TEST__";
-          root_conn.securityOperations().createUser(user, new PasswordToken().setPassword(password.getBytes()));
+          root_conn.securityOperations().createLocalUser(user, new PasswordToken(password));
           test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B"));
           if (root_conn.securityOperations().getUserAuthorizations(user).isEmpty())
             throw new IllegalStateException("Should be able to alter a user");
@@ -316,7 +313,7 @@ public class PermissionsTest {
     @Override
     public void run() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, MutationsRejectedException {
       // create the test user
-      getConnector().securityOperations().createUser(TEST_USER, TEST_PASS);
+      getConnector().securityOperations().createLocalUser(TEST_USER, TEST_PASS);
       Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS);
       
       // check for read-only access to metadata table