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 [9/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/jav...

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java Thu Feb 28 22:31:00 2013
@@ -36,11 +36,13 @@ 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.security.Authorizations;
+import org.apache.accumulo.core.security.CredentialHelper;
 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.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.tokens.PasswordToken;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
@@ -89,13 +91,23 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public void ping(Credential credentials) {
+  public void ping(TCredentials credentials) {
     // anybody can call this; no authentication check
     log.info("Master reports: I just got pinged!");
   }
   
   @Override
-  public boolean authenticateUser(TInfo tinfo, Credential credentials, Credential toAuth) throws ThriftSecurityException {
+  public boolean authenticate(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
+    try {
+      return security.authenticateUser(credentials, credentials);
+    } catch (ThriftSecurityException e) {
+      log.error(e);
+      throw e;
+    }
+  }
+  
+  @Override
+  public boolean authenticateUser(TInfo tinfo, TCredentials credentials, TCredentials toAuth) throws ThriftSecurityException {
     try {
       return security.authenticateUser(credentials, toAuth);
     } catch (ThriftSecurityException e) {
@@ -105,75 +117,78 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public void changeAuthorizations(TInfo tinfo, Credential credentials, String user, List<ByteBuffer> authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(TInfo tinfo, TCredentials credentials, String user, List<ByteBuffer> authorizations) throws ThriftSecurityException {
     security.changeAuthorizations(credentials, user, new Authorizations(authorizations));
   }
   
   @Override
-  public void changePassword(TInfo tinfo, Credential credentials, Credential toChange) throws ThriftSecurityException {
+  public void changeLocalUserPassword(TInfo tinfo, TCredentials credentials, String principal, ByteBuffer password) throws ThriftSecurityException {
+    PasswordToken token = new PasswordToken(password);
+    TCredentials toChange = CredentialHelper.createSquelchError(principal, token, credentials.instanceId);
     security.changePassword(credentials, toChange);
   }
   
   @Override
-  public void createUser(TInfo tinfo, Credential credentials, Credential newUser, List<ByteBuffer> authorizations)
-      throws ThriftSecurityException {
-    security.createUser(credentials, newUser, new Authorizations(authorizations));
+  public void createLocalUser(TInfo tinfo, TCredentials credentials, String principal, ByteBuffer password) throws ThriftSecurityException {
+    PasswordToken token = new PasswordToken(password);
+    TCredentials newUser = CredentialHelper.createSquelchError(principal, token, credentials.instanceId);
+    security.createUser(credentials, newUser, new Authorizations());
   }
   
   @Override
-  public void dropUser(TInfo tinfo, Credential credentials, String user) throws ThriftSecurityException {
+  public void dropLocalUser(TInfo tinfo, TCredentials credentials, String user) throws ThriftSecurityException {
     security.dropUser(credentials, user);
   }
   
   @Override
-  public List<ByteBuffer> getUserAuthorizations(TInfo tinfo, Credential credentials, String user) throws ThriftSecurityException {
+  public List<ByteBuffer> getUserAuthorizations(TInfo tinfo, TCredentials credentials, String user) throws ThriftSecurityException {
     return security.getUserAuthorizations(credentials, user).getAuthorizationsBB();
   }
   
   @Override
-  public void grantSystemPermission(TInfo tinfo, Credential credentials, String user, byte permission) throws ThriftSecurityException {
+  public void grantSystemPermission(TInfo tinfo, TCredentials credentials, String user, byte permission) throws ThriftSecurityException {
     security.grantSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
-  public void grantTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
+  public void grantTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     security.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
-  public void revokeSystemPermission(TInfo tinfo, Credential credentials, String user, byte permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(TInfo tinfo, TCredentials credentials, String user, byte permission) throws ThriftSecurityException {
     security.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
   
   @Override
-  public void revokeTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
+  public void revokeTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     security.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission));
   }
   
   @Override
-  public boolean hasSystemPermission(TInfo tinfo, Credential credentials, String user, byte sysPerm) throws ThriftSecurityException {
+  public boolean hasSystemPermission(TInfo tinfo, TCredentials credentials, String user, byte sysPerm) throws ThriftSecurityException {
     return security.hasSystemPermission(credentials, user, SystemPermission.getPermissionById(sysPerm));
   }
   
   @Override
-  public boolean hasTablePermission(TInfo tinfo, Credential credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
+  public boolean hasTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte tblPerm) throws ThriftSecurityException,
       ThriftTableOperationException {
     String tableId = checkTableId(tableName, TableOperation.PERMISSION);
     return security.hasTablePermission(credentials, user, tableId, TablePermission.getPermissionById(tblPerm));
   }
   
   @Override
-  public Set<String> listUsers(TInfo tinfo, Credential credentials) throws ThriftSecurityException {
+  public Set<String> listLocalUsers(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
     return security.listUsers(credentials);
   }
   
-  static private Map<String,String> conf(Credential credentials, AccumuloConfiguration conf) throws TException {
+  static private Map<String,String> conf(TCredentials credentials, AccumuloConfiguration conf) throws TException {
     security.authenticateUser(credentials, credentials);
-
+    
     Map<String,String> result = new HashMap<String,String>();
     for (Entry<String,String> entry : conf) {
       // TODO: do we need to send any instance information?
@@ -187,7 +202,7 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public Map<String,String> getConfiguration(TInfo tinfo, Credential credentials, ConfigurationType type) throws TException {
+  public Map<String,String> getConfiguration(TInfo tinfo, TCredentials credentials, ConfigurationType type) throws TException {
     switch (type) {
       case CURRENT:
         return conf(credentials, new ServerConfiguration(instance).getConfiguration());
@@ -200,16 +215,16 @@ public class ClientServiceHandler implem
   }
   
   @Override
-  public Map<String,String> getTableConfiguration(TInfo tinfo, Credential credentials, String tableName) throws TException, ThriftTableOperationException {
+  public Map<String,String> getTableConfiguration(TInfo tinfo, TCredentials credentials, String tableName) throws TException, ThriftTableOperationException {
     String tableId = checkTableId(tableName, null);
     return conf(credentials, new ServerConfiguration(instance).getTableConfiguration(tableId));
   }
   
   @Override
-  public List<String> bulkImportFiles(TInfo tinfo, final Credential tikw, final long tid, final String tableId, final List<String> files,
+  public List<String> bulkImportFiles(TInfo tinfo, final TCredentials tikw, final long tid, final String tableId, final List<String> files,
       final String errorDir, final boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, TException {
     try {
-      final Credential credentials = new Credential(tikw);
+      final TCredentials credentials = new TCredentials(tikw);
       if (!security.hasSystemPermission(credentials, credentials.getPrincipal(), SystemPermission.SYSTEM))
         throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       return transactionWatcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, new Callable<List<String>>() {
@@ -232,7 +247,7 @@ public class ClientServiceHandler implem
   
   @SuppressWarnings({"rawtypes", "unchecked"})
   @Override
-  public boolean checkClass(TInfo tinfo, Credential credentials, String className, String interfaceMatch) throws TException {
+  public boolean checkClass(TInfo tinfo, TCredentials credentials, String className, String interfaceMatch) throws TException {
     ClassLoader loader = getClass().getClassLoader();
     Class shouldMatch;
     try {

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java Thu Feb 28 22:31:00 2013
@@ -34,10 +34,10 @@ import org.apache.accumulo.core.conf.Pro
 import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
-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.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.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.StringUtil;
@@ -129,7 +129,9 @@ public class HdfsZooInstance implements 
   
   private static synchronized void _getInstanceID() {
     if (instanceId == null) {
-      instanceId = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      @SuppressWarnings("deprecation")
+      String instanceIdFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      instanceId = instanceIdFromFile;
     }
   }
   
@@ -150,19 +152,20 @@ public class HdfsZooInstance implements 
   
   @Override
   // Not really deprecated, just not for client use
-  public Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+  public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
   }
   
-  @SuppressWarnings("deprecation")
-  public Connector getConnector(Credential cred) throws AccumuloException, AccumuloSecurityException {
+  @Override
+  @Deprecated
+  public Connector getConnector(TCredentials cred) throws AccumuloException, AccumuloSecurityException {
     return new ConnectorImpl(this, cred);
   }
   
   @Override
   // Not really deprecated, just not for client use
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken().setPassword(pass));
+    return getConnector(user, new PasswordToken(pass));
   }
   
   @Override
@@ -198,6 +201,7 @@ public class HdfsZooInstance implements 
     System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
   }
   
+  @Deprecated
   @Override
   public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
     return getConnector(auth.user, auth.getPassword());

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java Thu Feb 28 22:31:00 2013
@@ -57,12 +57,13 @@ public class ZooConfiguration extends Ac
     return instance;
   }
   
-  @SuppressWarnings("deprecation")
   synchronized public static ZooConfiguration getInstance(AccumuloConfiguration parent) {
     if (instance == null) {
       propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
-      instanceId = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      @SuppressWarnings("deprecation")
+      String deprecatedInstanceIdFromHdfs = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      instanceId = deprecatedInstanceIdFromHdfs;
     }
     return instance;
   }
@@ -87,6 +88,7 @@ public class ZooConfiguration extends Ac
     return value;
   }
   
+  @Override
   public String get(Property property) {
     if (Property.isFixedZooPropertyKey(property)) {
       if (fixedProps.containsKey(property.getKey())) {

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java Thu Feb 28 22:31:00 2013
@@ -60,7 +60,7 @@ import org.apache.accumulo.core.gc.thrif
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.SecurityUtil;
-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.NamingThreadFactory;
 import org.apache.accumulo.core.util.ServerServices;
@@ -120,7 +120,7 @@ public class SimpleGarbageCollector impl
   
   private static final Logger log = Logger.getLogger(SimpleGarbageCollector.class);
     
-  private Credential credentials;
+  private TCredentials credentials;
   private long gcStartDelay;
   private boolean checkForBulkProcessingFiles;
   private FileSystem fs;
@@ -180,7 +180,7 @@ public class SimpleGarbageCollector impl
     this.address = address;
   }
 
-  public void init(FileSystem fs, Instance instance, Credential credentials, boolean noTrash) throws IOException {
+  public void init(FileSystem fs, Instance instance, TCredentials credentials, boolean noTrash) throws IOException {
     this.fs = TraceFileSystem.wrap(fs);
     this.credentials = credentials;
     this.instance = instance;
@@ -700,7 +700,7 @@ public class SimpleGarbageCollector impl
   }
   
   @Override
-  public GCStatus getStatus(TInfo info, Credential credentials) {
+  public GCStatus getStatus(TInfo info, TCredentials credentials) {
     return status;
   }
 }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/Master.java Thu Feb 28 22:31:00 2013
@@ -78,8 +78,8 @@ import org.apache.accumulo.core.master.t
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.master.thrift.TabletSplit;
 import org.apache.accumulo.core.security.SecurityUtil;
-import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -228,15 +228,14 @@ public class Master implements LiveTServ
   static final boolean X = true;
   static final boolean _ = false;
   static final boolean transitionOK[][] = {
-      //                            INITIAL HAVE_LOCK SAFE_MODE NORMAL UNLOAD_META UNLOAD_ROOT STOP
-      /* INITIAL */                 {X,     X,        _,        _,     _,          _,          X},
-      /* HAVE_LOCK */               {_,     X,        X,        X,     _,          _,          X},
-      /* SAFE_MODE */               {_,     _,        X,        X,     X,          _,          X},
-      /* NORMAL */                  {_,     _,        X,        X,     X,          _,          X},
-      /* UNLOAD_METADATA_TABLETS */ {_,     _,        X,        X,     X,          X,          X},
-      /* UNLOAD_ROOT_TABLET */      {_,     _,        _,        X,     _,          X,          X},
-      /* STOP */                    {_,     _,        _,        _,     _,          _,          X}
-  };
+      // INITIAL HAVE_LOCK SAFE_MODE NORMAL UNLOAD_META UNLOAD_ROOT STOP
+      /* INITIAL */{X, X, _, _, _, _, X},
+      /* HAVE_LOCK */{_, X, X, X, _, _, X},
+      /* SAFE_MODE */{_, _, X, X, X, _, X},
+      /* NORMAL */{_, _, X, X, X, _, X},
+      /* UNLOAD_METADATA_TABLETS */{_, _, X, X, X, X, X},
+      /* UNLOAD_ROOT_TABLET */{_, _, _, X, _, X, X},
+      /* STOP */{_, _, _, _, _, _, X}};
   
   synchronized private void setMasterState(MasterState newState) {
     if (state.equals(newState))
@@ -276,11 +275,11 @@ public class Master implements LiveTServ
         log.info("Upgrading zookeeper");
         
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-
+        
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
-
+        
         for (String id : Tables.getIdToNameMap(instance).keySet()) {
-
+          
           zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(),
               NodeExistsPolicy.SKIP);
         }
@@ -290,7 +289,7 @@ public class Master implements LiveTServ
       }
     }
   }
-
+  
   private final AtomicBoolean upgradeMetadataRunning = new AtomicBoolean(false);
   
   private final ServerConfiguration serverConfig;
@@ -481,7 +480,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public long initiateFlush(TInfo tinfo, Credential c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
+    public long initiateFlush(TInfo tinfo, TCredentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
       security.canFlush(c, tableId);
       
       String zTablePath = Constants.ZROOT + "/" + getConfiguration().getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId
@@ -508,7 +507,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void waitForFlush(TInfo tinfo, Credential c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
+    public void waitForFlush(TInfo tinfo, TCredentials c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
         throws ThriftSecurityException, ThriftTableOperationException, TException {
       security.canFlush(c, tableId);
       
@@ -619,7 +618,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public MasterMonitorInfo getMasterStats(TInfo info, Credential credentials) throws ThriftSecurityException, TException {
+    public MasterMonitorInfo getMasterStats(TInfo info, TCredentials credentials) throws ThriftSecurityException, TException {
       final MasterMonitorInfo result = new MasterMonitorInfo();
       
       result.tServerInfo = new ArrayList<TabletServerStatus>();
@@ -652,7 +651,7 @@ public class Master implements LiveTServ
       return result;
     }
     
-    private void alterTableProperty(Credential c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
+    private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
         ThriftTableOperationException {
       final String tableId = checkTableId(tableName, op);
       if (!security.canAlterTable(c, tableId))
@@ -671,25 +670,25 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void removeTableProperty(TInfo info, Credential credentials, String tableName, String property) throws ThriftSecurityException,
+    public void removeTableProperty(TInfo info, TCredentials credentials, String tableName, String property) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
       alterTableProperty(credentials, tableName, property, null, TableOperation.REMOVE_PROPERTY);
     }
     
     @Override
-    public void setTableProperty(TInfo info, Credential credentials, String tableName, String property, String value) throws ThriftSecurityException,
+    public void setTableProperty(TInfo info, TCredentials credentials, String tableName, String property, String value) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
       alterTableProperty(credentials, tableName, property, value, TableOperation.SET_PROPERTY);
     }
     
     @Override
-    public void shutdown(TInfo info, Credential c, boolean stopTabletServers) throws ThriftSecurityException, TException {
+    public void shutdown(TInfo info, TCredentials c, boolean stopTabletServers) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       Master.this.shutdown(stopTabletServers);
     }
     
     @Override
-    public void shutdownTabletServer(TInfo info, Credential c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
+    public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer, boolean force) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       final InetSocketAddress addr = AddressUtil.parseAddress(tabletServer, Property.TSERV_CLIENTPORT);
@@ -710,7 +709,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void reportSplitExtent(TInfo info, Credential credentials, String serverName, TabletSplit split) throws TException {
+    public void reportSplitExtent(TInfo info, TCredentials credentials, String serverName, TabletSplit split) throws TException {
       KeyExtent oldTablet = new KeyExtent(split.oldTablet);
       if (migrations.remove(oldTablet) != null) {
         log.info("Canceled migration of " + split.oldTablet);
@@ -724,9 +723,8 @@ public class Master implements LiveTServ
       log.warn("Got a split from a server we don't recognize: " + serverName);
     }
     
-    
     @Override
-    public void reportTabletStatus(TInfo info, Credential credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) throws TException {
+    public void reportTabletStatus(TInfo info, TCredentials credentials, String serverName, TabletLoadState status, TKeyExtent ttablet) throws TException {
       KeyExtent tablet = new KeyExtent(ttablet);
       
       switch (status) {
@@ -754,7 +752,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void setMasterGoalState(TInfo info, Credential c, MasterGoalState state) throws ThriftSecurityException, TException {
+    public void setMasterGoalState(TInfo info, TCredentials c, MasterGoalState state) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       Master.this.setMasterGoalState(state);
@@ -771,7 +769,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void removeSystemProperty(TInfo info, Credential c, String property) throws ThriftSecurityException, TException {
+    public void removeSystemProperty(TInfo info, TCredentials c, String property) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       try {
@@ -784,7 +782,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void setSystemProperty(TInfo info, Credential c, String property, String value) throws ThriftSecurityException, TException {
+    public void setSystemProperty(TInfo info, TCredentials c, String property, String value) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
       
       try {
@@ -796,20 +794,20 @@ public class Master implements LiveTServ
       }
     }
     
-    private void authenticate(Credential c) throws ThriftSecurityException {
+    private void authenticate(TCredentials c) throws ThriftSecurityException {
       if (!security.authenticateUser(c, c))
         throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
       
     }
     
     @Override
-    public long beginTableOperation(TInfo tinfo, Credential credentials) throws ThriftSecurityException, TException {
+    public long beginTableOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       authenticate(credentials);
       return fate.startTransaction();
     }
     
     @Override
-    public void executeTableOperation(TInfo tinfo, Credential c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
+    public void executeTableOperation(TInfo tinfo, TCredentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
         List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, TException {
       authenticate(c);
       
@@ -1011,7 +1009,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public String waitForTableOperation(TInfo tinfo, Credential credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
+    public String waitForTableOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
         TException {
       authenticate(credentials);
       
@@ -1035,7 +1033,7 @@ public class Master implements LiveTServ
     }
     
     @Override
-    public void finishTableOperation(TInfo tinfo, Credential credentials, long opid) throws ThriftSecurityException, TException {
+    public void finishTableOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, TException {
       authenticate(credentials);
       fate.delete(opid);
     }
@@ -2042,7 +2040,7 @@ public class Master implements LiveTServ
     getMasterLock(zroot + Constants.ZMASTER_LOCK);
     
     recoveryManager = new RecoveryManager(this);
-
+    
     TableManager.getInstance().addObserver(this);
     
     StatusThread statusThread = new StatusThread();
@@ -2088,7 +2086,7 @@ public class Master implements LiveTServ
       }
     });
     
-    Credential systemAuths = SecurityConstants.getSystemCredentials();
+    TCredentials systemAuths = SecurityConstants.getSystemCredentials();
     final TabletStateStore stores[] = {new ZooTabletStateStore(new ZooStore(zroot)), new RootTabletStateStore(instance, systemAuths, this),
         new MetaDataStateStore(instance, systemAuths, this)};
     watchers.add(new TabletGroupWatcher(stores[2], null));

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java Thu Feb 28 22:31:00 2013
@@ -28,7 +28,7 @@ import org.apache.accumulo.core.client.M
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 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.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.hadoop.io.Text;
@@ -42,9 +42,9 @@ public class MetaDataStateStore extends 
   
   final protected Instance instance;
   final protected CurrentState state;
-  final protected Credential auths;
+  final protected TCredentials auths;
   
-  public MetaDataStateStore(Instance instance, Credential auths, CurrentState state) {
+  public MetaDataStateStore(Instance instance, TCredentials auths, CurrentState state) {
     this.instance = instance;
     this.state = state;
     this.auths = auths;

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java Thu Feb 28 22:31:00 2013
@@ -38,7 +38,7 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 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.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
@@ -48,7 +48,7 @@ public class MetaDataTableScanner implem
   BatchScanner mdScanner;
   Iterator<Entry<Key,Value>> iter;
   
-  public MetaDataTableScanner(Instance instance, Credential auths, Range range, CurrentState state) {
+  public MetaDataTableScanner(Instance instance, TCredentials auths, Range range, CurrentState state) {
     // scan over metadata table, looking for tablets in the wrong state based on the live servers and online tables
     try {
       Connector connector = instance.getConnector(auths.getPrincipal(), CredentialHelper.extractToken(auths));
@@ -78,7 +78,7 @@ public class MetaDataTableScanner implem
     scanner.addScanIterator(tabletChange);
   }
   
-  public MetaDataTableScanner(Instance instance, Credential auths, Range range) {
+  public MetaDataTableScanner(Instance instance, TCredentials auths, Range range) {
     this(instance, auths, range, null);
   }
   

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java Thu Feb 28 22:31:00 2013
@@ -20,11 +20,11 @@ import java.util.Iterator;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
 public class RootTabletStateStore extends MetaDataStateStore {
   
-  public RootTabletStateStore(Instance instance, Credential auths, CurrentState state) {
+  public RootTabletStateStore(Instance instance, TCredentials auths, CurrentState state) {
     super(instance, auths, state);
   }
   

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java Thu Feb 28 22:31:00 2013
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.Authorizations;
 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.ThriftSecurityException;
 import org.apache.accumulo.server.security.handler.Authenticator;
 import org.apache.accumulo.server.security.handler.Authorizor;
@@ -49,16 +49,16 @@ public class AuditedSecurityOperation ex
     return instance;
   }
   
-  private void audit(Credential credentials, ThriftSecurityException ex, String template, Object... args) {
+  private void audit(TCredentials credentials, ThriftSecurityException ex, String template, Object... args) {
     log.log(AuditLevel.AUDIT, "Error: authenticated operation failed: " + credentials.getPrincipal() + ": " + String.format(template, args));
   }
   
-  private void audit(Credential credentials, String template, Object... args) {
+  private void audit(TCredentials credentials, String template, Object... args) {
     log.log(AuditLevel.AUDIT, "Using credentials " + credentials.getPrincipal() + ": " + String.format(template, args));
   }
   
   @Override
-  public boolean authenticateUser(Credential credentials, Credential toAuth) throws ThriftSecurityException {
+  public boolean authenticateUser(TCredentials credentials, TCredentials toAuth) throws ThriftSecurityException {
     try {
       boolean result = super.authenticateUser(credentials, toAuth);
       audit(credentials, result ? "authenticated" : "failed authentication");
@@ -71,7 +71,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Authorizations getUserAuthorizations(Credential credentials, String user) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(TCredentials credentials, String user) throws ThriftSecurityException {
     try {
       Authorizations result = super.getUserAuthorizations(credentials, user);
       audit(credentials, "got authorizations for %s", user);
@@ -85,7 +85,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Authorizations getUserAuthorizations(Credential credentials) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(TCredentials credentials) throws ThriftSecurityException {
     try {
       return getUserAuthorizations(credentials, credentials.getPrincipal());
     } catch (ThriftSecurityException ex) {
@@ -95,7 +95,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void changeAuthorizations(Credential credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(TCredentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
     try {
       super.changeAuthorizations(credentials, user, authorizations);
       audit(credentials, "changed authorizations for %s to %s", user, authorizations);
@@ -107,7 +107,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void changePassword(Credential credentials, Credential newInfo) throws ThriftSecurityException {
+  public void changePassword(TCredentials credentials, TCredentials newInfo) throws ThriftSecurityException {
     try {
       super.changePassword(credentials, newInfo);
       audit(credentials, "changed password for %s", newInfo.getPrincipal());
@@ -119,7 +119,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void createUser(Credential credentials, Credential newUser, Authorizations authorizations) throws ThriftSecurityException {
+  public void createUser(TCredentials credentials, TCredentials newUser, Authorizations authorizations) throws ThriftSecurityException {
     try {
       super.createUser(credentials, newUser, authorizations);
       audit(credentials, "createUser");
@@ -131,7 +131,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void dropUser(Credential credentials, String user) throws ThriftSecurityException {
+  public void dropUser(TCredentials credentials, String user) throws ThriftSecurityException {
     try {
       super.dropUser(credentials, user);
       audit(credentials, "dropUser");
@@ -143,7 +143,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void grantSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void grantSystemPermission(TCredentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       super.grantSystemPermission(credentials, user, permission);
       audit(credentials, "granted permission %s for %s", permission, user);
@@ -155,7 +155,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void grantTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public void grantTablePermission(TCredentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       super.grantTablePermission(credentials, user, table, permission);
       audit(credentials, "granted permission %s on table %s for %s", permission, table, user);
@@ -167,7 +167,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void revokeSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(TCredentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       super.revokeSystemPermission(credentials, user, permission);
       audit(credentials, "revoked permission %s for %s", permission, user);
@@ -179,7 +179,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void revokeTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public void revokeTablePermission(TCredentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       super.revokeTablePermission(credentials, user, table, permission);
       audit(credentials, "revoked permission %s on table %s for %s", permission, table, user);
@@ -191,7 +191,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public boolean hasSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public boolean hasSystemPermission(TCredentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     try {
       boolean result = super.hasSystemPermission(credentials, user, permission);
       audit(credentials, "checked permission %s on %s", permission, user);
@@ -204,7 +204,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public boolean hasTablePermission(Credential credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
+  public boolean hasTablePermission(TCredentials credentials, String user, String table, TablePermission permission) throws ThriftSecurityException {
     try {
       boolean result = super.hasTablePermission(credentials, user, table, permission);
       audit(credentials, "checked permission %s on table %s for %s", permission, table, user);
@@ -217,7 +217,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public Set<String> listUsers(Credential credentials) throws ThriftSecurityException {
+  public Set<String> listUsers(TCredentials credentials) throws ThriftSecurityException {
     try {
       Set<String> result = super.listUsers(credentials);
       audit(credentials, "listUsers");
@@ -230,7 +230,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void deleteTable(Credential credentials, String table) throws ThriftSecurityException {
+  public void deleteTable(TCredentials credentials, String table) throws ThriftSecurityException {
     try {
       super.deleteTable(credentials, table);
       audit(credentials, "deleted table %s", table);
@@ -242,7 +242,7 @@ public class AuditedSecurityOperation ex
   }
   
   @Override
-  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
+  public void initializeSecurity(TCredentials credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
     super.initializeSecurity(credentials, principal, token);
     log.info("Initialized root user with username: " + principal + " at the request of user " + credentials.getPrincipal());
   }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java Thu Feb 28 22:31:00 2013
@@ -27,9 +27,9 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
 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.AuthenticationToken;
 import org.apache.accumulo.core.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.commons.codec.binary.Base64;
@@ -40,23 +40,24 @@ public class SecurityConstants {
   static Logger log = Logger.getLogger(SecurityConstants.class);
   
   public static final String SYSTEM_PRINCIPAL = "!SYSTEM";
-  private static final SecurityToken SYSTEM_TOKEN = makeSystemPassword();
-  private static final Credential systemCredentials = CredentialHelper.createSquelchError(SYSTEM_PRINCIPAL, SYSTEM_TOKEN, HdfsZooInstance.getInstance().getInstanceID());
+  private static final AuthenticationToken SYSTEM_TOKEN = makeSystemPassword();
+  private static final TCredentials systemCredentials = CredentialHelper.createSquelchError(SYSTEM_PRINCIPAL, SYSTEM_TOKEN, HdfsZooInstance.getInstance()
+      .getInstanceID());
   public static byte[] confChecksum = null;
   
-  public static SecurityToken getSystemToken() {
+  public static AuthenticationToken getSystemToken() {
     return SYSTEM_TOKEN;
   }
   
-  public static Credential getSystemCredentials() {
+  public static TCredentials getSystemCredentials() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION);
     }
     return systemCredentials;
   }
-
-  private static SecurityToken makeSystemPassword() {
+  
+  private static AuthenticationToken makeSystemPassword() {
     int wireVersion = Constants.WIRE_VERSION;
     byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes(Constants.UTF8);
     try {
@@ -78,7 +79,7 @@ public class SecurityConstants {
       // ByteArrayOutputStream; crash hard
       // if this happens
     }
-    return new PasswordToken().setPassword(Base64.encodeBase64(bytes.toByteArray()));
+    return new PasswordToken(Base64.encodeBase64(bytes.toByteArray()));
   }
   
   private static byte[] getSystemConfigChecksum() throws NoSuchAlgorithmException {

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java Thu Feb 28 22:31:00 2013
@@ -22,17 +22,17 @@ import org.apache.accumulo.core.Constant
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 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.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.security.handler.Authenticator;
 import org.apache.accumulo.server.security.handler.Authorizor;
@@ -71,26 +71,23 @@ public class SecurityOperation {
     return instance;
   }
   
-  @SuppressWarnings("deprecation")
   protected static Authorizor getAuthorizor(String instanceId, boolean initialize) {
-    Authorizor toRet = Master.createInstanceFromPropertyName(AccumuloConfiguration.getSiteConfiguration(), Property.INSTANCE_SECURITY_AUTHORIZOR,
+    Authorizor toRet = Master.createInstanceFromPropertyName(ServerConfiguration.getSiteConfiguration(), Property.INSTANCE_SECURITY_AUTHORIZOR,
         Authorizor.class, ZKAuthorizor.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
   
-  @SuppressWarnings("deprecation")
   protected static Authenticator getAuthenticator(String instanceId, boolean initialize) {
-    Authenticator toRet = Master.createInstanceFromPropertyName(AccumuloConfiguration.getSiteConfiguration(), Property.INSTANCE_SECURITY_AUTHENTICATOR,
+    Authenticator toRet = Master.createInstanceFromPropertyName(ServerConfiguration.getSiteConfiguration(), Property.INSTANCE_SECURITY_AUTHENTICATOR,
         Authenticator.class, ZKAuthenticator.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
   
-  @SuppressWarnings("deprecation")
   protected static PermissionHandler getPermHandler(String instanceId, boolean initialize) {
-    PermissionHandler toRet = Master.createInstanceFromPropertyName(AccumuloConfiguration.getSiteConfiguration(),
-        Property.INSTANCE_SECURITY_PERMISSION_HANDLER, PermissionHandler.class, ZKPermHandler.getInstance());
+    PermissionHandler toRet = Master.createInstanceFromPropertyName(ServerConfiguration.getSiteConfiguration(), Property.INSTANCE_SECURITY_PERMISSION_HANDLER,
+        PermissionHandler.class, ZKPermHandler.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
@@ -116,7 +113,7 @@ public class SecurityOperation {
           + " do not play nice with eachother. Please choose authentication and authorization mechanisms that are compatible with one another.");
   }
   
-  public void initializeSecurity(Credential credentials, String rootPrincipal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
+  public void initializeSecurity(TCredentials credentials, String rootPrincipal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
     authenticate(credentials);
     
     if (!credentials.getPrincipal().equals(SecurityConstants.SYSTEM_PRINCIPAL))
@@ -139,7 +136,7 @@ public class SecurityOperation {
     return rootUserName;
   }
   
-  private void authenticate(Credential credentials) throws ThriftSecurityException {
+  private void authenticate(TCredentials credentials) throws ThriftSecurityException {
     if (!credentials.getInstanceId().equals(HdfsZooInstance.getInstance().getInstanceID()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.INVALID_INSTANCEID);
     
@@ -150,7 +147,7 @@ public class SecurityOperation {
     }
     
     try {
-      SecurityToken token = reassembleToken(credentials);
+      AuthenticationToken token = reassembleToken(credentials);
       if (!authenticator.authenticateUser(credentials.getPrincipal(), token)) {
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
       }
@@ -160,20 +157,20 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canAskAboutUser(Credential credentials, String user) throws ThriftSecurityException {
+  public boolean canAskAboutUser(TCredentials credentials, String user) throws ThriftSecurityException {
     // Authentication done in canPerformSystemActions
     if (!(canPerformSystemActions(credentials) || credentials.getPrincipal().equals(user)))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return true;
   }
   
-  public boolean authenticateUser(Credential credentials, Credential toAuth) throws ThriftSecurityException {
+  public boolean authenticateUser(TCredentials credentials, TCredentials toAuth) throws ThriftSecurityException {
     canAskAboutUser(credentials, toAuth.getPrincipal());
     // User is already authenticated from canAskAboutUser, this gets around issues with !SYSTEM user
     if (credentials.equals(toAuth))
       return true;
     try {
-      SecurityToken token = reassembleToken(toAuth);
+      AuthenticationToken token = reassembleToken(toAuth);
       return authenticator.authenticateUser(toAuth.getPrincipal(), token);
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
@@ -183,17 +180,17 @@ public class SecurityOperation {
   /**
    * @param toAuth
    * @return
-   * @throws AccumuloSecurityException 
+   * @throws AccumuloSecurityException
    */
-  private SecurityToken reassembleToken(Credential toAuth) throws AccumuloSecurityException {
-    String tokenClass = toAuth.getTokenClass();
+  private AuthenticationToken reassembleToken(TCredentials toAuth) throws AccumuloSecurityException {
+    String tokenClass = toAuth.getTokenClassName();
     if (authenticator.validTokenClass(tokenClass)) {
       return CredentialHelper.extractToken(toAuth);
     }
     throw new AccumuloSecurityException(toAuth.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
   }
-
-  public Authorizations getUserAuthorizations(Credential credentials, String user) throws ThriftSecurityException {
+  
+  public Authorizations getUserAuthorizations(TCredentials credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
     
     targetUserExists(user);
@@ -212,7 +209,7 @@ public class SecurityOperation {
     }
   }
   
-  public Authorizations getUserAuthorizations(Credential credentials) throws ThriftSecurityException {
+  public Authorizations getUserAuthorizations(TCredentials credentials) throws ThriftSecurityException {
     return getUserAuthorizations(credentials, credentials.getPrincipal());
   }
   
@@ -262,7 +259,7 @@ public class SecurityOperation {
   }
   
   // some people just aren't allowed to ask about other users; here are those who can ask
-  private boolean canAskAboutOtherUsers(Credential credentials, String user) throws ThriftSecurityException {
+  private boolean canAskAboutOtherUsers(TCredentials credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
     return credentials.getPrincipal().equals(user) || hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false)
         || hasSystemPermission(credentials.getPrincipal(), SystemPermission.CREATE_USER, false)
@@ -282,17 +279,17 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canScan(Credential credentials, String table) throws ThriftSecurityException {
+  public boolean canScan(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), table, TablePermission.READ, true);
   }
   
-  public boolean canWrite(Credential credentials, String table) throws ThriftSecurityException {
+  public boolean canWrite(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), table, TablePermission.WRITE, true);
   }
   
-  public boolean canSplitTablet(Credential credentials, String table) throws ThriftSecurityException {
+  public boolean canSplitTablet(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false)
@@ -302,90 +299,90 @@ public class SecurityOperation {
   /**
    * This is the check to perform any system action. This includes tserver's loading of a tablet, shutting the system down, or altering system properties.
    */
-  public boolean canPerformSystemActions(Credential credentials) throws ThriftSecurityException {
+  public boolean canPerformSystemActions(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.SYSTEM, false);
   }
   
-  public boolean canFlush(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canFlush(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canAlterTable(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canAlterTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false)
         || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false);
   }
   
-  public boolean canCreateTable(Credential c) throws ThriftSecurityException {
+  public boolean canCreateTable(TCredentials c) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_TABLE, false);
   }
   
-  public boolean canRenameTable(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canRenameTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canCloneTable(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canCloneTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_TABLE, false)
         && hasTablePermission(c.getPrincipal(), tableId, TablePermission.READ, false);
   }
   
-  public boolean canDeleteTable(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canDeleteTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.DROP_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.DROP_TABLE, false);
   }
   
-  public boolean canOnlineOfflineTable(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canMerge(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canMerge(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false);
   }
   
-  public boolean canDeleteRange(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canDeleteRange(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.SYSTEM, false) || hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false);
   }
   
-  public boolean canBulkImport(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canBulkImport(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c.getPrincipal(), tableId, TablePermission.BULK_IMPORT, false);
   }
   
-  public boolean canCompact(Credential c, String tableId) throws ThriftSecurityException {
+  public boolean canCompact(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.ALTER_TABLE, false)
         || hasTablePermission(c.getPrincipal(), tableId, TablePermission.WRITE, false);
   }
   
-  public boolean canChangeAuthorizations(Credential c, String user) throws ThriftSecurityException {
+  public boolean canChangeAuthorizations(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     if (user.equals(SecurityConstants.SYSTEM_PRINCIPAL))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_USER, false);
   }
   
-  public boolean canChangePassword(Credential c, String user) throws ThriftSecurityException {
+  public boolean canChangePassword(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     if (user.equals(SecurityConstants.SYSTEM_PRINCIPAL))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return c.getPrincipal().equals(user) || hasSystemPermission(c.getPrincipal(), SystemPermission.ALTER_USER, false);
   }
   
-  public boolean canCreateUser(Credential c, String user) throws ThriftSecurityException {
+  public boolean canCreateUser(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     
     // don't allow creating a user with the same name as system user
@@ -395,7 +392,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.CREATE_USER, false);
   }
   
-  public boolean canDropUser(Credential c, String user) throws ThriftSecurityException {
+  public boolean canDropUser(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     
     // can't delete root or system users
@@ -405,7 +402,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.DROP_USER, false);
   }
   
-  public boolean canGrantSystem(Credential c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+  public boolean canGrantSystem(TCredentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -419,7 +416,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.GRANT, false);
   }
   
-  public boolean canGrantTable(Credential c, String user, String table) throws ThriftSecurityException {
+  public boolean canGrantTable(TCredentials c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -430,7 +427,7 @@ public class SecurityOperation {
         || hasTablePermission(c.getPrincipal(), table, TablePermission.GRANT, false);
   }
   
-  public boolean canRevokeSystem(Credential c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
+  public boolean canRevokeSystem(TCredentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system or root user
@@ -444,7 +441,7 @@ public class SecurityOperation {
     return hasSystemPermission(c.getPrincipal(), SystemPermission.GRANT, false);
   }
   
-  public boolean canRevokeTable(Credential c, String user, String table) throws ThriftSecurityException {
+  public boolean canRevokeTable(TCredentials c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     
     // can't modify system user
@@ -455,7 +452,7 @@ public class SecurityOperation {
         || hasTablePermission(c.getPrincipal(), table, TablePermission.GRANT, false);
   }
   
-  public void changeAuthorizations(Credential credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
+  public void changeAuthorizations(TCredentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
     if (!canChangeAuthorizations(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -469,11 +466,11 @@ public class SecurityOperation {
     }
   }
   
-  public void changePassword(Credential credentials, Credential toChange) throws ThriftSecurityException {
+  public void changePassword(TCredentials credentials, TCredentials toChange) throws ThriftSecurityException {
     if (!canChangePassword(credentials, toChange.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
-      SecurityToken token = reassembleToken(toChange);
+      AuthenticationToken token = reassembleToken(toChange);
       authenticator.changePassword(toChange.getPrincipal(), token);
       log.info("Changed password for user " + toChange.getPrincipal() + " at the request of user " + credentials.getPrincipal());
     } catch (AccumuloSecurityException e) {
@@ -481,11 +478,11 @@ public class SecurityOperation {
     }
   }
   
-  public void createUser(Credential credentials, Credential newUser, Authorizations authorizations) throws ThriftSecurityException {
+  public void createUser(TCredentials credentials, TCredentials newUser, Authorizations authorizations) throws ThriftSecurityException {
     if (!canCreateUser(credentials, newUser.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
-      SecurityToken token = reassembleToken(newUser);
+      AuthenticationToken token = reassembleToken(newUser);
       authenticator.createUser(newUser.getPrincipal(), token);
       authorizor.initUser(newUser.getPrincipal());
       permHandle.initUser(newUser.getPrincipal());
@@ -497,7 +494,7 @@ public class SecurityOperation {
     }
   }
   
-  public void dropUser(Credential credentials, String user) throws ThriftSecurityException {
+  public void dropUser(TCredentials credentials, String user) throws ThriftSecurityException {
     if (!canDropUser(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
@@ -510,7 +507,7 @@ public class SecurityOperation {
     }
   }
   
-  public void grantSystemPermission(Credential credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
+  public void grantSystemPermission(TCredentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canGrantSystem(credentials, user, permissionById))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -524,7 +521,7 @@ public class SecurityOperation {
     }
   }
   
-  public void grantTablePermission(Credential c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
+  public void grantTablePermission(TCredentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canGrantTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -540,7 +537,7 @@ public class SecurityOperation {
     }
   }
   
-  public void revokeSystemPermission(Credential credentials, String user, SystemPermission permission) throws ThriftSecurityException {
+  public void revokeSystemPermission(TCredentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     if (!canRevokeSystem(credentials, user, permission))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -555,7 +552,7 @@ public class SecurityOperation {
     }
   }
   
-  public void revokeTablePermission(Credential c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
+  public void revokeTablePermission(TCredentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canRevokeTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     
@@ -572,20 +569,19 @@ public class SecurityOperation {
     }
   }
   
-  public boolean hasSystemPermission(Credential credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
+  public boolean hasSystemPermission(TCredentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasSystemPermission(user, permissionById, false);
   }
   
-  public boolean hasTablePermission(Credential credentials, String user, String tableId, TablePermission permissionById)
-      throws ThriftSecurityException {
+  public boolean hasTablePermission(TCredentials credentials, String user, String tableId, TablePermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasTablePermission(user, tableId, permissionById, false);
   }
   
-  public Set<String> listUsers(Credential credentials) throws ThriftSecurityException {
+  public Set<String> listUsers(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     try {
       return authenticator.listUsers();
@@ -594,7 +590,7 @@ public class SecurityOperation {
     }
   }
   
-  public void deleteTable(Credential credentials, String tableId) throws ThriftSecurityException {
+  public void deleteTable(TCredentials credentials, String tableId) throws ThriftSecurityException {
     if (!canDeleteTable(credentials, tableId))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
@@ -607,12 +603,12 @@ public class SecurityOperation {
     }
   }
   
-  public boolean canExport(Credential credentials, String tableId) throws ThriftSecurityException {
+  public boolean canExport(TCredentials credentials, String tableId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.getPrincipal(), tableId, TablePermission.READ, false);
   }
   
-  public boolean canImport(Credential credentials) throws ThriftSecurityException {
+  public boolean canImport(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials.getPrincipal(), SystemPermission.CREATE_TABLE, false);
   }

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java Thu Feb 28 22:31:00 2013
@@ -19,9 +19,9 @@ package org.apache.accumulo.server.secur
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-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.security.tokens.SecurityToken;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 
 /**
  * This interface is used for the system which will be used for authenticating a user. If the implementation does not support configuration through Accumulo, it
@@ -34,17 +34,17 @@ public interface Authenticator extends o
   
   public boolean validSecurityHandlers(Authorizor auth, PermissionHandler pm);
   
-  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(TCredentials credentials, String principal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException;
   
-  public boolean authenticateUser(String principal, SecurityToken token) throws AccumuloSecurityException;
+  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloSecurityException;
   
   public Set<String> listUsers() throws AccumuloSecurityException;
   
-  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException;
+  public void createUser(String principal, AuthenticationToken token) throws AccumuloSecurityException;
   
   public void dropUser(String user) throws AccumuloSecurityException;
   
-  public void changePassword(String principal, SecurityToken token) throws AccumuloSecurityException;
+  public void changePassword(String principal, AuthenticationToken token) throws AccumuloSecurityException;
   
   public boolean userExists(String user) throws AccumuloSecurityException;
   

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java Thu Feb 28 22:31:00 2013
@@ -18,7 +18,7 @@ package org.apache.accumulo.server.secur
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 
 /**
@@ -40,7 +40,7 @@ public interface Authorizor {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(Credential credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(TCredentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to change the authorizations for the user

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java Thu Feb 28 22:31:00 2013
@@ -20,9 +20,9 @@ import java.util.Collections;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.security.tokens.NullToken;
-import org.apache.accumulo.core.security.tokens.SecurityToken;
+import org.apache.accumulo.core.security.tokens.AuthenticationToken;
 
 /**
  * This is an Authenticator implementation that doesn't actually do any security. Use at your own risk.
@@ -40,12 +40,12 @@ public class InsecureAuthenticator exten
   }
   
   @Override
-  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException {
+  public void initializeSecurity(TCredentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
     return;
   }
   
   @Override
-  public boolean authenticateUser(String principal, SecurityToken token) {
+  public boolean authenticateUser(String principal, AuthenticationToken token) {
     return true;
   }
   
@@ -55,7 +55,7 @@ public class InsecureAuthenticator exten
   }
   
   @Override
-  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException {
+  public void createUser(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     return;
   }
   
@@ -65,7 +65,7 @@ public class InsecureAuthenticator exten
   }
   
   @Override
-  public void changePassword(String user, SecurityToken token) throws AccumuloSecurityException {
+  public void changePassword(String user, AuthenticationToken token) throws AccumuloSecurityException {
     return;
   }
 

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java Thu Feb 28 22:31:00 2013
@@ -20,7 +20,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;
 
 /**
  * This is a Permission Handler implementation that doesn't actually do any security. Use at your own risk.
@@ -47,7 +47,7 @@ public class InsecurePermHandler impleme
    * @see org.apache.accumulo.server.security.handler.PermissionHandler#initializeSecurity(java.lang.String)
    */
   @Override
-  public void initializeSecurity(Credential token, String rootuser) throws AccumuloSecurityException {
+  public void initializeSecurity(TCredentials token, String rootuser) throws AccumuloSecurityException {
     return;
   }
   

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java Thu Feb 28 22:31:00 2013
@@ -20,7 +20,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.ThriftSecurityException;
 
 /**
@@ -42,7 +42,7 @@ public interface PermissionHandler {
   /**
    * Used to initialize security for the root user
    */
-  public void initializeSecurity(Credential credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
+  public void initializeSecurity(TCredentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
   
   /**
    * Used to get the system permission for the user

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java Thu Feb 28 22:31:00 2013
@@ -22,10 +22,10 @@ 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.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 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.core.security.tokens.AuthenticationToken;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -57,7 +57,7 @@ public final class ZKAuthenticator exten
   }
   
   @Override
-  public void initializeSecurity(Credential credentials, String principal, byte[] token) throws AccumuloSecurityException {
+  public void initializeSecurity(TCredentials credentials, String principal, byte[] token) throws AccumuloSecurityException {
     try {
       // remove old settings from zookeeper first, if any
       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -105,7 +105,7 @@ public final class ZKAuthenticator exten
    * Creates a user with no permissions whatsoever
    */
   @Override
-  public void createUser(String principal, SecurityToken token) throws AccumuloSecurityException {
+  public void createUser(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     try {
       if (!(token instanceof PasswordToken))
         throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
@@ -143,7 +143,7 @@ public final class ZKAuthenticator exten
   }
   
   @Override
-  public void changePassword(String principal, SecurityToken token) throws AccumuloSecurityException {
+  public void changePassword(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
@@ -182,7 +182,7 @@ public final class ZKAuthenticator exten
   }
   
   @Override
-  public boolean authenticateUser(String principal, SecurityToken token) throws AccumuloSecurityException {
+  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;

Modified: accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java?rev=1451401&r1=1451400&r2=1451401&view=diff
==============================================================================
--- accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java (original)
+++ accumulo/branches/1.5/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java Thu Feb 28 22:31:00 2013
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.security.Authorizations;
 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;
@@ -73,7 +73,7 @@ public class ZKAuthorizor implements Aut
   }
   
   @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