You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2013/02/09 08:57:39 UTC

svn commit: r1444337 [1/10] - in /accumulo/trunk: 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/java/org/apache/accumulo/core...

Author: vines
Date: Sat Feb  9 07:57:36 2013
New Revision: 1444337

URL: http://svn.apache.org/r1444337
Log:
ACCUMULO-1041 - Majority of the client interfaces tackled. CLI needs work, as does proxy. Javadocs are a mess, as are some variable names, but it's functional. Except mapreduce APIs need DistributedCache code reintegrated as discussed on JIRA


Modified:
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnRequiredTable.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/Instance.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/security/thrift/SecurityErrorCode.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateUserCommand.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java
    accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java
    accumulo/trunk/core/src/main/thrift/client.thrift
    accumulo/trunk/core/src/main/thrift/gc.thrift
    accumulo/trunk/core/src/main/thrift/master.thrift
    accumulo/trunk/core/src/main/thrift/security.thrift
    accumulo/trunk/core/src/main/thrift/tabletserver.thrift
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
    accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/Viewer.java
    accumulo/trunk/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
    accumulo/trunk/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
    accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/RootTabletStateStore.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authenticator.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecureAuthenticator.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/Admin.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
    accumulo/trunk/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
    accumulo/trunk/server/src/test/java/org/apache/accumulo/server/master/TestMergeState.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/TestIngest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/VisibilityTest.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateUser.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/AlterTablePerm.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/Authenticate.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/ChangePass.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/CreateUser.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/DropTable.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SecurityFixture.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/SetAuths.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
    accumulo/trunk/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
    accumulo/trunk/test/src/test/java/org/apache/accumulo/test/MiniAccumuloClusterTest.java

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnDefaultTable.java Sat Feb  9 07:57:36 2013
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.cli;
 
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -39,12 +40,12 @@ public class ClientOnDefaultTable extend
   }
   
   @Override
-  public void setAccumuloConfigs(Job job) {
+  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
     super.setAccumuloConfigs(job);
-    AccumuloInputFormat.setConnectorInfo(job, user, getPassword());
+    AccumuloInputFormat.setConnectorInfo(job, principal, getToken());
     AccumuloInputFormat.setInputTableName(job, getTableName());
     AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setConnectorInfo(job, user, getPassword());
+    AccumuloOutputFormat.setConnectorInfo(job, principal, getToken());
     AccumuloOutputFormat.setCreateTables(job, true);
     AccumuloOutputFormat.setDefaultTableName(job, getTableName());
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnRequiredTable.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnRequiredTable.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnRequiredTable.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOnRequiredTable.java Sat Feb  9 07:57:36 2013
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.cli;
 
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -28,12 +29,12 @@ public class ClientOnRequiredTable exten
   public String tableName = null;
   
   @Override
-  public void setAccumuloConfigs(Job job) {
+  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
     super.setAccumuloConfigs(job);
-    AccumuloInputFormat.setConnectorInfo(job, user, getPassword());
+    AccumuloInputFormat.setConnectorInfo(job, principal, getToken());
     AccumuloInputFormat.setInputTableName(job, tableName);
     AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setConnectorInfo(job, user, getPassword());
+    AccumuloOutputFormat.setConnectorInfo(job, principal, getToken());
     AccumuloOutputFormat.setCreateTables(job, true);
     AccumuloOutputFormat.setDefaultTableName(job, tableName);
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java Sat Feb  9 07:57:36 2013
@@ -16,13 +16,12 @@
  */
 package org.apache.accumulo.core.cli;
 
-import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 import java.util.UUID;
 
-import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -37,7 +36,11 @@ import org.apache.accumulo.core.conf.Def
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
+import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
@@ -74,7 +77,7 @@ public class ClientOpts extends Help {
     public byte[] value;
     
     public Password(String dfault) {
-      value = dfault.getBytes();
+      value = dfault.getBytes(Charset.forName("UTF-8"));
     }
     
     @Override
@@ -98,7 +101,7 @@ public class ClientOpts extends Help {
   }
   
   @Parameter(names = {"-u", "--user"}, description = "Connection user")
-  public String user = System.getProperty("user.name");
+  public String principal = System.getProperty("user.name");
   
   @Parameter(names = "-p", converter = PasswordConverter.class, description = "Connection password")
   public Password password = new Password("secret");
@@ -106,11 +109,14 @@ public class ClientOpts extends Help {
   @Parameter(names = "--password", converter = PasswordConverter.class, description = "Enter the connection password", password = true)
   public Password securePassword = null;
   
-  public byte[] getPassword() {
+  public SecurityToken getToken() {
+    PasswordToken pt = new PasswordToken();
     if (securePassword == null) {
-      return password.value;
+      if (password.value == null)
+        return null;
+      return pt.setPassword(password.value);
     }
-    return securePassword.value;
+    return pt.setPassword(securePassword.value);
   }
   
   @Parameter(names = {"-z", "--keepers"}, description = "Comma separated list of zookeeper hosts (host:port,host:port)")
@@ -197,14 +203,14 @@ public class ClientOpts extends Help {
   }
   
   public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
-    return getInstance().getConnector(this.user, this.getPassword());
+    return getInstance().getConnector(this.principal, this.getToken());
   }
   
-  public Credentials getCredentials() {
-    return new Credentials(user, ByteBuffer.wrap(getPassword()), getInstance().getInstanceID());
+  public Credential getCredentials() throws AccumuloSecurityException {
+    return CredentialHelper.create(principal, getToken(), getInstance().getInstanceID());
   }
   
-  public void setAccumuloConfigs(Job job) {
+  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
     AccumuloInputFormat.setZooKeeperInstance(job, instance, zookeepers);
     AccumuloOutputFormat.setZooKeeperInstance(job, instance, zookeepers);
   }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/Instance.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/Instance.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/Instance.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/Instance.java Sat Feb  9 07:57:36 2013
@@ -20,7 +20,9 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
+import org.apache.accumulo.core.security.thrift.Credential;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 
 /**
  * This class represents the information a client needs to know to connect to an instance of accumulo.
@@ -94,9 +96,10 @@ public interface Instance {
    *           when a generic exception occurs
    * @throws AccumuloSecurityException
    *           when a user's credentials are invalid
+   * @deprecated since 1.5, use {@link #getConnector(String, SecurityToken)}
    */
-  public abstract Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException;
-  
+  public abstract Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException;
+    
   /**
    * Returns a connection to accumulo.
    * 
@@ -147,4 +150,22 @@ public interface Instance {
    * @throws AccumuloException
    */
   public abstract String getAuthenticatorClassName() throws AccumuloException;
+
+  /**
+   * @param principal
+   * @param token
+   *  A known SecurityToken type. If uncertain which should be provided, use {@link #getAuthenticatorClassName()} and {@link Authenticator#login(Properties)}
+   * @return
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   */
+  Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * @param auth
+   * @return
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   */
+  Connector getConnector(Credential auth) throws AccumuloException, AccumuloSecurityException;
 }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java Sat Feb  9 07:57:36 2013
@@ -29,8 +29,12 @@ import org.apache.accumulo.core.conf.Acc
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
-import org.apache.accumulo.core.security.thrift.Credentials;
+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.ThriftSecurityException;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -218,10 +222,19 @@ public class ZooKeeperInstance implement
   }
   
   // Suppress deprecation, ConnectorImpl is deprecated to warn clients against using.
+  @Override
+  public Connector getConnector(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
+  }
+  
   @SuppressWarnings("deprecation")
+  public Connector getConnector(Credential credential) throws AccumuloException, AccumuloSecurityException {
+    return new ConnectorImpl(this, credential);
+  }
+  
   @Override
-  public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(this, user, pass);
+  public Connector getConnector(String principal, byte[] pass) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(principal, new PasswordToken().setPassword(pass));
   }
   
   private AccumuloConfiguration conf = null;
@@ -288,8 +301,8 @@ public class ZooKeeperInstance implement
   }
   
   @Override
-  public Connector getConnector(Credentials auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.getPrincipal(), auth.getToken());
+  public Connector getConnector(AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
+    return getConnector(auth.user, auth.password);
   }
   
   @Override

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java Sat Feb  9 07:57:36 2013
@@ -33,7 +33,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.util.ArgumentChecker;
@@ -49,7 +49,7 @@ import org.apache.thrift.transport.TTran
  */
 public class InstanceOperationsImpl implements InstanceOperations {
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   
   /**
    * @param instance
@@ -57,7 +57,7 @@ public class InstanceOperationsImpl impl
    * @param credentials
    *          the username/password for this connection
    */
-  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
+  public InstanceOperationsImpl(Instance instance, Credential credentials) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java Sat Feb  9 07:57:36 2013
@@ -23,6 +23,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.tokens.SecurityToken;
 
 /**
  * Provides a class for managing users and permissions
@@ -42,7 +43,7 @@ public interface SecurityOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to create a user
-   * @deprecated Use {@link #createUser(String, byte[])} instead
+   * @deprecated Use {@link #createUser(String, SecurityToken)} instead
    */
   @Deprecated
   public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException;
@@ -59,7 +60,7 @@ public interface SecurityOperations {
    * @throws AccumuloSecurityException
    *           if the user does not have permission to create a user
    */
-  public void createUser(String principal, byte[] token) throws AccumuloException, AccumuloSecurityException;
+  public void createUser(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException;
   
   /**
    * Delete a user
@@ -85,10 +86,26 @@ public interface SecurityOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to ask
+   * @deprecated see {@link #authenticateUser(String, SecurityToken)}
    */
   public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException;
   
   /**
+   * Verify a username/password combination is valid
+   * 
+   * @param principal
+   *          the name of the user to authenticate
+   * @param token
+   *          the SecurityToken for the user
+   * @return true if the user asking is allowed to know and the specified principal/token is valid, false otherwise
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission to ask
+   */
+  boolean authenticateUser(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException; 
+
+  /**
    * Set the user's password
    * 
    * @param user
@@ -99,6 +116,7 @@ public interface SecurityOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to modify a user
+   * @deprecated see {@link #changeLoginInfo(String, SecurityToken)}          
    */
   public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException;
   
@@ -230,5 +248,20 @@ public interface SecurityOperations {
    * @throws AccumuloSecurityException
    *           if the user does not have permission to query users
    */
-  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException; 
+  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Set the user's password
+   * 
+   * @param principal
+   *          the principal who's password is to be changed
+   * @param token
+   *          the security token with the information about the user to modify
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission to modify a user
+   */
+  void changeLoginInfo(String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException;
+
 }

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java Sat Feb  9 07:57:36 2013
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -30,11 +28,14 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 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.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.thrift.tokens.PasswordToken;
+import org.apache.accumulo.core.security.thrift.tokens.SecurityToken;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.trace.instrument.Tracer;
@@ -42,7 +43,7 @@ import org.apache.accumulo.trace.instrum
 public class SecurityOperationsImpl implements SecurityOperations {
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   
   private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
@@ -86,39 +87,18 @@ public class SecurityOperationsImpl impl
    * @param credentials
    *          the user credentials to use for security operations
    */
-  public SecurityOperationsImpl(Instance instance, Credentials credentials) {
+  public SecurityOperationsImpl(Instance instance, Credential credentials) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;
   }
   
-  /**
-   * Create a user
-   * 
-   * @param user
-   *          the name of the user to create
-   * @param password
-   *          the plaintext password for the user
-   * @param authorizations
-   *          the authorizations that the user has for scanning
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to create a user
-   * @deprecated Use {@link #createUser(String,byte[])} instead
-   */
-  @Override
   @Deprecated
+  @Override
   public void createUser(final String user, final byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(user, password, authorizations);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.createUser(Tracer.traceInfo(), credentials, user, ByteBuffer.wrap(password), ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
-      }
-    });
+    createUser(user, new PasswordToken().setPassword(password), authorizations);
   }
-
+  
   /**
    * Create a user
    * 
@@ -131,10 +111,22 @@ public class SecurityOperationsImpl impl
    * @throws AccumuloSecurityException
    *           if the user does not have permission to create a user
    */
-  public void createUser(final String principal, final byte[] token) throws AccumuloException, AccumuloSecurityException {
+  public void createUser(final String principal, final SecurityToken token) throws AccumuloException, AccumuloSecurityException {
     createUser(principal, token, new Authorizations());
   }
   
+  // Private method because the token/authorization constructor is something which is essentially new and depreciated.
+  private void createUser(String principal, SecurityToken token, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, token, authorizations);
+    final Credential toCreate = CredentialHelper.create(principal, token, instance.getInstanceID());
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.createUser(Tracer.traceInfo(), credentials, toCreate, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
+      }
+    });
+  }
+  
   /**
    * Delete a user
    * 
@@ -168,15 +160,22 @@ public class SecurityOperationsImpl impl
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to ask
+   * @deprecated see {@link #authenticateUser(String, SecurityToken)}
    */
-  @Deprecated
   @Override
   public boolean authenticateUser(final String principal, final byte[] token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(token);
+    return authenticateUser(principal, new PasswordToken().setPassword(token));
+  }
+  
+  @Override
+  public boolean authenticateUser(final String principal, final SecurityToken token) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, token);
+    final Credential toAuth = CredentialHelper.create(principal, token, instance.getInstanceID());
     return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
       public Boolean execute(ClientService.Client client) throws Exception {
-        return client.authenticateUser(Tracer.traceInfo(), credentials, principal, ByteBuffer.wrap(token));
+        return client.authenticateUser(Tracer.traceInfo(), credentials, toAuth);
       }
     });
   }
@@ -192,17 +191,26 @@ public class SecurityOperationsImpl impl
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to modify a user
+   * @deprecated
    */
+  @Deprecated
   public void changeUserPassword(final String principal, final byte[] token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(token);
+    changeLoginInfo(principal, new PasswordToken().setPassword(token));
+  }
+  
+  @Override
+  public void changeLoginInfo(final String principal, SecurityToken token) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, token);
+    final Credential toChange = CredentialHelper.create(principal, token, instance.getInstanceID());
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
-        client.changePassword(Tracer.traceInfo(), credentials, principal, ByteBuffer.wrap(token));
+        client.changePassword(Tracer.traceInfo(), credentials, toChange);
       }
     });
     if (this.credentials.principal.equals(principal)) {
-      this.credentials.token = ByteBuffer.wrap(Arrays.copyOf(token, token.length));
+      this.credentials = toChange;
     }
   }
   

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Sat Feb  9 07:57:36 2013
@@ -74,7 +74,8 @@ import org.apache.accumulo.core.master.s
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.TableOperation;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -105,7 +106,7 @@ import org.apache.thrift.transport.TTran
  */
 public class TableOperationsImpl extends TableOperationsHelper {
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   
   private static final Logger log = Logger.getLogger(TableOperations.class);
   
@@ -115,7 +116,7 @@ public class TableOperationsImpl extends
    * @param credentials
    *          the username/password for this connection
    */
-  public TableOperationsImpl(Instance instance, Credentials credentials) {
+  public TableOperationsImpl(Instance instance, Credential credentials) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;
@@ -1153,7 +1154,7 @@ public class TableOperationsImpl extends
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, auths);
-    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
+    Scanner scanner = instance.getConnector(credentials.getPrincipal(), CredentialHelper.extractToken(credentials)).createScanner(tableName, auths);
     return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
   }
   

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/BatchWriterImpl.java Sat Feb  9 07:57:36 2013
@@ -21,7 +21,7 @@ import org.apache.accumulo.core.client.B
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 
 public class BatchWriterImpl implements BatchWriter {
@@ -29,7 +29,7 @@ public class BatchWriterImpl implements 
   private String table;
   private TabletServerBatchWriter bw;
   
-  public BatchWriterImpl(Instance instance, Credentials credentials, String table, BatchWriterConfig config) {
+  public BatchWriterImpl(Instance instance, Credential credentials, String table, BatchWriterConfig config) {
     ArgumentChecker.notNull(instance, credentials, table, config);
     this.table = table;
     this.bw = new TabletServerBatchWriter(instance, credentials, config);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java Sat Feb  9 07:57:36 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.client.impl;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -40,14 +39,14 @@ import org.apache.accumulo.core.client.a
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.trace.instrument.Tracer;
 
 public class ConnectorImpl extends Connector {
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private SecurityOperations secops = null;
   private TableOperations tableops = null;
   private InstanceOperations instanceops = null;
@@ -65,22 +64,20 @@ public class ConnectorImpl extends Conne
    * @deprecated Not for client use
    */
   @Deprecated
-  public ConnectorImpl(Instance instance, String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(instance, user, password);
+  public ConnectorImpl(Instance instance, Credential cred) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(instance, cred);
     this.instance = instance;
     
-    // copy password so that user can clear it.... in future versions we can clear it...
-    byte[] passCopy = new byte[password.length];
-    System.arraycopy(password, 0, passCopy, 0, password.length);
-    this.credentials = new Credentials(user, ByteBuffer.wrap(password), instance.getInstanceID());
+    
+    this.credentials = cred;
     
     // hardcoded string for SYSTEM user since the definition is
     // in server code
-    if (!user.equals("!SYSTEM")) {
+    if (!cred.getPrincipal().equals("!SYSTEM")) {
       ServerClient.execute(instance, new ClientExec<ClientService.Client>() {
         @Override
         public void execute(ClientService.Client iface) throws Exception {
-          if (!iface.authenticateUser(Tracer.traceInfo(), credentials, credentials.getPrincipal(), credentials.token))
+          if (!iface.authenticateUser(Tracer.traceInfo(), credentials, credentials))
             throw new AccumuloSecurityException("Authentication failed, access denied", SecurityErrorCode.BAD_CREDENTIALS);
         }
       });

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MetadataLocationObtainer.java Sat Feb  9 07:57:36 2013
@@ -41,7 +41,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.MetadataTable;
 import org.apache.accumulo.core.util.OpTimer;
@@ -53,12 +53,12 @@ import org.apache.log4j.Logger;
 
 public class MetadataLocationObtainer implements TabletLocationObtainer {
   private static final Logger log = Logger.getLogger(MetadataLocationObtainer.class);
-  private Credentials credentials;
+  private Credential credentials;
   private SortedSet<Column> locCols;
   private ArrayList<Column> columns;
   private Instance instance;
   
-  MetadataLocationObtainer(Credentials credentials, Instance instance) {
+  MetadataLocationObtainer(Credential credentials, Instance instance) {
     
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java Sat Feb  9 07:57:36 2013
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.log4j.Logger;
 
@@ -72,7 +72,7 @@ public class MultiTableBatchWriterImpl i
   private HashMap<String,BatchWriter> tableWriters;
   private Instance instance;
   
-  public MultiTableBatchWriterImpl(Instance instance, Credentials credentials, BatchWriterConfig config) {
+  public MultiTableBatchWriterImpl(Instance instance, Credential credentials, BatchWriterConfig config) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.bw = new TabletServerBatchWriter(instance, credentials, config);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java Sat Feb  9 07:57:36 2013
@@ -55,7 +55,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
@@ -121,7 +121,7 @@ class OfflineIterator implements Iterato
    * @param authorizations
    * @param table
    */
-  public OfflineIterator(ScannerOptions options, Instance instance, Credentials credentials, Authorizations authorizations, Text table, Range range) {
+  public OfflineIterator(ScannerOptions options, Instance instance, Credential credentials, Authorizations authorizations, Text table, Range range) {
     this.options = new ScannerOptions(options);
     this.instance = instance;
     this.range = range;
@@ -343,11 +343,11 @@ public class OfflineScanner extends Scan
   private Range range;
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private Authorizations authorizations;
   private Text tableId;
   
-  public OfflineScanner(Instance instance, Credentials credentials, String tableId, Authorizations authorizations) {
+  public OfflineScanner(Instance instance, Credential credentials, String tableId, Authorizations authorizations) {
     ArgumentChecker.notNull(instance, credentials, tableId, authorizations);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java Sat Feb  9 07:57:36 2013
@@ -38,7 +38,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.hadoop.io.Text;
 
@@ -50,7 +50,7 @@ public class ScannerImpl extends Scanner
   // and just query for the next highest row from the tablet server
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private Authorizations authorizations;
   private Text table;
   
@@ -59,7 +59,7 @@ public class ScannerImpl extends Scanner
   private Range range;
   private boolean isolated = false;
   
-  public ScannerImpl(Instance instance, Credentials credentials, String table, Authorizations authorizations) {
+  public ScannerImpl(Instance instance, Credential credentials, String table, Authorizations authorizations) {
     ArgumentChecker.notNull(instance, credentials, table, authorizations);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java Sat Feb  9 07:57:36 2013
@@ -39,7 +39,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -55,7 +55,7 @@ public class ScannerIterator implements 
   // scanner state
   private Iterator<KeyValue> iter;
   private ScanState scanState;
-  private Credentials credentials;
+  private Credential credentials;
   private Instance instance;
   
   private ScannerOptions options;
@@ -121,7 +121,7 @@ public class ScannerIterator implements 
     
   }
   
-  ScannerIterator(Instance instance, Credentials credentials, Text table, Authorizations authorizations, Range range, int size, int timeOut,
+  ScannerIterator(Instance instance, Credential credentials, Text table, Authorizations authorizations, Range range, int size, int timeOut,
       ScannerOptions options, boolean isolated) {
     this.instance = instance;
     this.tableName = new Text(table);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java Sat Feb  9 07:57:36 2013
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.client.T
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.hadoop.io.Text;
 
@@ -92,7 +92,7 @@ public abstract class TabletLocator {
   
   private static final Text ROOT_TABLET_MDE = KeyExtent.getMetadataEntry(new Text(Constants.METADATA_TABLE_ID), null);
   
-  public static synchronized TabletLocator getInstance(Instance instance, Credentials credentials, Text tableId) {
+  public static synchronized TabletLocator getInstance(Instance instance, Credential credentials, Text tableId) {
     LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId);
     
     TabletLocator tl = locators.get(key);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchDeleter.java Sat Feb  9 07:57:36 2013
@@ -32,16 +32,16 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.SortedKeyIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 
 public class TabletServerBatchDeleter extends TabletServerBatchReader implements BatchDeleter {
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private String tableId;
   private BatchWriterConfig bwConfig;
   
-  public TabletServerBatchDeleter(Instance instance, Credentials credentials, String tableId, Authorizations authorizations, int numQueryThreads,
+  public TabletServerBatchDeleter(Instance instance, Credential credentials, String tableId, Authorizations authorizations, int numQueryThreads,
       BatchWriterConfig bwConfig) throws TableNotFoundException {
     super(instance, credentials, tableId, authorizations, numQueryThreads);
     this.instance = instance;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Sat Feb  9 07:57:36 2013
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.log4j.Logger;
@@ -44,7 +44,7 @@ public class TabletServerBatchReader ext
   private Instance instance;
   private ArrayList<Range> ranges;
   
-  private Credentials credentials;
+  private Credential credentials;
   private Authorizations authorizations = Constants.NO_AUTHS;
   
   private static int nextBatchReaderInstance = 1;
@@ -55,7 +55,7 @@ public class TabletServerBatchReader ext
   
   private final int batchReaderInstance = getNextBatchReaderInstance();
   
-  public TabletServerBatchReader(Instance instance, Credentials credentials, String table, Authorizations authorizations, int numQueryThreads) {
+  public TabletServerBatchReader(Instance instance, Credential credentials, String table, Authorizations authorizations, int numQueryThreads) {
     ArgumentChecker.notNull(instance, credentials, table, authorizations);
     this.instance = instance;
     this.credentials = credentials;

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java Sat Feb  9 07:57:36 2013
@@ -58,7 +58,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
@@ -79,7 +79,7 @@ public class TabletServerBatchReaderIter
   private static final Logger log = Logger.getLogger(TabletServerBatchReaderIterator.class);
   
   private final Instance instance;
-  private final Credentials credentials;
+  private final Credential credentials;
   private final String table;
   private Authorizations authorizations = Constants.NO_AUTHS;
   private final int numThreads;
@@ -133,7 +133,7 @@ public class TabletServerBatchReaderIter
     
   }
   
-  public TabletServerBatchReaderIterator(Instance instance, Credentials credentials, String table, Authorizations authorizations, ArrayList<Range> ranges,
+  public TabletServerBatchReaderIterator(Instance instance, Credential credentials, String table, Authorizations authorizations, ArrayList<Range> ranges,
       int numThreads, ExecutorService queryThreadPool, ScannerOptions scannerOptions, long timeout) {
     
     this.instance = instance;
@@ -600,13 +600,13 @@ public class TabletServerBatchReaderIter
   }
   
   static void doLookup(String server, Map<KeyExtent,List<Range>> requested, Map<KeyExtent,List<Range>> failures, Map<KeyExtent,List<Range>> unscanned,
-      ResultReceiver receiver, List<Column> columns, Credentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf)
+      ResultReceiver receiver, List<Column> columns, Credential credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf)
       throws IOException, AccumuloSecurityException, AccumuloServerException {
     doLookup(server, requested, failures, unscanned, receiver, columns, credentials, options, authorizations, conf, new TimeoutTracker(Long.MAX_VALUE));
   }
   
   static void doLookup(String server, Map<KeyExtent,List<Range>> requested, Map<KeyExtent,List<Range>> failures, Map<KeyExtent,List<Range>> unscanned,
-      ResultReceiver receiver, List<Column> columns, Credentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
+      ResultReceiver receiver, List<Column> columns, Credential credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf,
       TimeoutTracker timeoutTracker) throws IOException, AccumuloSecurityException, AccumuloServerException {
     
     if (requested.size() == 0) {

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Sat Feb  9 07:57:36 2013
@@ -58,7 +58,7 @@ import org.apache.accumulo.core.data.Mut
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -111,7 +111,7 @@ public class TabletServerBatchWriter {
   private FailedMutations failedMutations;
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   
   private Violations violations;
   private Map<KeyExtent,Set<SecurityErrorCode>> authorizationFailures;
@@ -187,7 +187,7 @@ public class TabletServerBatchWriter {
     }
   }
   
-  public TabletServerBatchWriter(Instance instance, Credentials credentials, BatchWriterConfig config) {
+  public TabletServerBatchWriter(Instance instance, Credential credentials, BatchWriterConfig config) {
     this.instance = instance;
     this.maxMem = config.getMaxMemory();
     this.maxLatency = config.getMaxLatency(TimeUnit.MILLISECONDS) <= 0 ? Long.MAX_VALUE : config.getMaxLatency(TimeUnit.MILLISECONDS);

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java Sat Feb  9 07:57:36 2013
@@ -52,7 +52,7 @@ import org.apache.accumulo.core.data.thr
 import org.apache.accumulo.core.data.thrift.TKeyValue;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -82,7 +82,7 @@ public class ThriftScanner {
     }
   }
   
-  public static boolean getBatchFromServer(Credentials credentials, Text startRow, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  public static boolean getBatchFromServer(Credential credentials, Text startRow, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, boolean skipStartKey, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf)
       throws AccumuloException, AccumuloSecurityException, NotServingTabletException {
     Key startKey;
@@ -105,13 +105,13 @@ public class ThriftScanner {
     return getBatchFromServer(credentials, startKey, (Key) null, extent, server, results, fetchedColumns, size, authorizations, retry, conf);
   }
   
-  static boolean getBatchFromServer(Credentials credentials, Key key, Key endKey, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(Credential credentials, Key key, Key endKey, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf) throws AccumuloException,
       AccumuloSecurityException, NotServingTabletException {
     return getBatchFromServer(credentials, new Range(key, true, endKey, true), extent, server, results, fetchedColumns, size, authorizations, retry, conf);
   }
   
-  static boolean getBatchFromServer(Credentials credentials, Range range, KeyExtent extent, String server, SortedMap<Key,Value> results,
+  static boolean getBatchFromServer(Credential credentials, Range range, KeyExtent extent, String server, SortedMap<Key,Value> results,
       SortedSet<Column> fetchedColumns, int size, Authorizations authorizations, boolean retry, AccumuloConfiguration conf) throws AccumuloException,
       AccumuloSecurityException, NotServingTabletException {
     if (server == null)
@@ -170,7 +170,7 @@ public class ThriftScanner {
     
     int size;
     
-    Credentials credentials;
+    Credential credentials;
     Authorizations authorizations;
     List<Column> columns;
     
@@ -183,7 +183,7 @@ public class ThriftScanner {
     
     Map<String,Map<String,String>> serverSideIteratorOptions;
     
-    public ScanState(Credentials credentials, Text tableName, Authorizations authorizations, Range range, SortedSet<Column> fetchedColumns, int size,
+    public ScanState(Credential credentials, Text tableName, Authorizations authorizations, Range range, SortedSet<Column> fetchedColumns, int size,
         List<IterInfo> serverSideIteratorList, Map<String,Map<String,String>> serverSideIteratorOptions, boolean isolated) {
       this.credentials = credentials;
       this.authorizations = authorizations;
@@ -220,7 +220,7 @@ public class ThriftScanner {
     
   }
   
-  public static List<KeyValue> scan(Instance instance, Credentials credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
+  public static List<KeyValue> scan(Instance instance, Credential credentials, ScanState scanState, int timeOut, AccumuloConfiguration conf)
       throws ScanTimedOutException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TabletLocation loc = null;
     

Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java?rev=1444337&r1=1444336&r2=1444337&view=diff
==============================================================================
--- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java (original)
+++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java Sat Feb  9 07:57:36 2013
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.client.i
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.thrift.Credentials;
+import org.apache.accumulo.core.security.thrift.Credential;
 import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -44,21 +44,21 @@ public class Writer {
   private static final Logger log = Logger.getLogger(Writer.class);
   
   private Instance instance;
-  private Credentials credentials;
+  private Credential credentials;
   private Text table;
   
-  public Writer(Instance instance, Credentials credentials, Text table) {
+  public Writer(Instance instance, Credential credentials, Text table) {
     ArgumentChecker.notNull(instance, credentials, table);
     this.instance = instance;
     this.credentials = credentials;
     this.table = table;
   }
   
-  public Writer(Instance instance, Credentials credentials, String table) {
+  public Writer(Instance instance, Credential credentials, String table) {
     this(instance, credentials, new Text(table));
   }
   
-  private static void updateServer(Mutation m, KeyExtent extent, String server, Credentials ai, AccumuloConfiguration configuration) throws TException,
+  private static void updateServer(Mutation m, KeyExtent extent, String server, Credential ai, AccumuloConfiguration configuration) throws TException,
       NotServingTabletException, ConstraintViolationException, AccumuloSecurityException {
     ArgumentChecker.notNull(m, extent, server, ai);