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 2012/11/01 17:29:09 UTC

svn commit: r1404662 [2/3] - in /accumulo/branches/ACCUMULO-259: ./ assemble/ core/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/mapreduce/ core/src/main/java/org/apache/accumulo/core/clie...

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/Master.java Thu Nov  1 16:29:05 2012
@@ -21,6 +21,7 @@ import static java.lang.Math.min;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -214,6 +215,8 @@ public class Master implements LiveTServ
   private MasterState state = MasterState.INITIAL;
   
   private Fate<Master> fate;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   volatile private SortedMap<TServerInstance,TabletServerStatus> tserverStatus = Collections
       .unmodifiableSortedMap(new TreeMap<TServerInstance,TabletServerStatus>());
@@ -290,8 +293,8 @@ public class Master implements LiveTServ
         String[] tablePropsToDelete = new String[] {"table.scan.cache.size", "table.scan.cache.enable"};
 
         for (String id : Tables.getIdToNameMap(instance).keySet()) {
-          zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_FLUSH_ID, "0".getBytes(), NodeExistsPolicy.SKIP);
-          zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_ID, "0".getBytes(), NodeExistsPolicy.SKIP);
+          zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_FLUSH_ID, "0".getBytes(utf8), NodeExistsPolicy.SKIP);
+          zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_ID, "0".getBytes(utf8), NodeExistsPolicy.SKIP);
           
           for (String prop : tablePropsToDelete) {
             String propPath = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + prop;
@@ -538,7 +541,7 @@ public class Master implements LiveTServ
           public byte[] mutate(byte[] currentValue) throws Exception {
             long flushID = Long.parseLong(new String(currentValue));
             flushID++;
-            return ("" + flushID).getBytes();
+            return ("" + flushID).getBytes(utf8);
           }
         });
       } catch (NoNodeException nne) {
@@ -1130,7 +1133,7 @@ public class Master implements LiveTServ
   
   private void setMasterGoalState(MasterGoalState state) {
     try {
-      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
+      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(utf8),
           NodeExistsPolicy.OVERWRITE);
     } catch (Exception ex) {
       log.error("Unable to set master goal state in zookeeper");
@@ -1704,7 +1707,7 @@ public class Master implements LiveTServ
         }
         
         if (maxLogicalTime != null)
-          Constants.METADATA_TIME_COLUMN.put(m, new Value(maxLogicalTime.getBytes()));
+          Constants.METADATA_TIME_COLUMN.put(m, new Value(maxLogicalTime.getBytes(utf8)));
         
         if (!m.getUpdates().isEmpty()) {
           bw.addMutation(m);
@@ -2174,7 +2177,7 @@ public class Master implements LiveTServ
     boolean locked = false;
     while (System.currentTimeMillis() - current < waitTime) {
       masterLock = new ZooLock(zMasterLoc);
-      if (masterLock.tryLock(masterLockWatcher, masterClientAddress.getBytes())) {
+      if (masterLock.tryLock(masterLockWatcher, masterClientAddress.getBytes(utf8))) {
         locked = true;
         break;
       }
@@ -2184,7 +2187,7 @@ public class Master implements LiveTServ
       log.info("Failed to get master lock, even after waiting for session timeout, becoming back-up server");
       while (true) {
         masterLock = new ZooLock(zMasterLoc);
-        if (masterLock.tryLock(masterLockWatcher, masterClientAddress.getBytes())) {
+        if (masterLock.tryLock(masterLockWatcher, masterClientAddress.getBytes(utf8))) {
           break;
         }
         UtilWaitThread.sleep(TIME_TO_WAIT_BETWEEN_LOCK_CHECKS);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/recovery/SubmitFileForRecovery.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/recovery/SubmitFileForRecovery.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/recovery/SubmitFileForRecovery.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/recovery/SubmitFileForRecovery.java Thu Nov  1 16:29:05 2012
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master.recovery;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -38,6 +39,8 @@ public class SubmitFileForRecovery exten
   private static final long serialVersionUID = 1L;
   String server;
   String file;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   SubmitFileForRecovery(String server, String file) {
     this.server = server;
@@ -48,7 +51,7 @@ public class SubmitFileForRecovery exten
   public Repo<Master> call(long tid, final Master master) throws Exception {
     master.updateRecoveryInProgress(file);
     String source = RecoverLease.getSource(master, server, file).toString();
-    new DistributedWorkQueue(ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY).addWork(file, source.getBytes());
+    new DistributedWorkQueue(ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY).addWork(file, source.getBytes(utf8));
     
     ZooReaderWriter zoo = ZooReaderWriter.getInstance();
     final String path = ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY + "/" + file;

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.master.state;
 
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -30,6 +31,8 @@ import org.apache.zookeeper.data.Stat;
 public class DeadServerList {
   private static final Logger log = Logger.getLogger(DeadServerList.class);
   private final String path;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public DeadServerList(String path) {
     this.path = path;
@@ -72,7 +75,7 @@ public class DeadServerList {
   public void post(String server, String cause) {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
-      zoo.putPersistentData(path + "/" + server, cause.getBytes(), NodeExistsPolicy.SKIP);
+      zoo.putPersistentData(path + "/" + server, cause.getBytes(utf8), NodeExistsPolicy.SKIP);
     } catch (Exception ex) {
       log.error(ex, ex);
     }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/SetGoalState.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.server.master.state;
 
+import java.nio.charset.Charset;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
@@ -30,6 +32,8 @@ import org.apache.accumulo.server.zookee
 import org.apache.hadoop.fs.FileSystem;
 
 public class SetGoalState {
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   /**
    * Utility program that will change the goal state for the master from the command line.
@@ -43,7 +47,7 @@ public class SetGoalState {
 
     FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfiguration.getSiteConfiguration());
     Accumulo.waitForZookeeperAndHdfs(fs);
-    ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(),
+    ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(utf8),
         NodeExistsPolicy.OVERWRITE);
   }
   

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java Thu Nov  1 16:29:05 2012
@@ -21,6 +21,7 @@ package org.apache.accumulo.server.maste
 
 import java.io.Serializable;
 import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.Property;
@@ -41,6 +42,8 @@ public class TServerInstance implements 
   private InetSocketAddress location;
   private String session;
   private String cachedStringRepresentation;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public TServerInstance(InetSocketAddress address, String session) {
     this.location = address;
@@ -118,7 +121,7 @@ public class TServerInstance implements 
   }
   
   public Value asMutationValue() {
-    return new Value(org.apache.accumulo.core.util.AddressUtil.toString(getLocation()).getBytes());
+    return new Value(org.apache.accumulo.core.util.AddressUtil.toString(getLocation()).getBytes(utf8));
   }
   
   public InetSocketAddress getLocation() {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java Thu Nov  1 16:29:05 2012
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master.state;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -47,6 +48,8 @@ public class TabletStateChangeIterator e
   private static final String TABLES_OPTION = "tables";
   private static final String MERGES_OPTION = "merges";
   // private static final Logger log = Logger.getLogger(TabletStateChangeIterator.class);
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   Set<TServerInstance> current;
   Set<String> onlineTables;
@@ -93,7 +96,7 @@ public class TabletStateChangeIterator e
     try {
       Map<Text,MergeInfo> result = new HashMap<Text,MergeInfo>();
       DataInputBuffer buffer = new DataInputBuffer();
-      byte[] data = Base64.decodeBase64(merges.getBytes());
+      byte[] data = Base64.decodeBase64(merges.getBytes(utf8));
       buffer.reset(data, data.length);
       while (buffer.available() > 0) {
         MergeInfo mergeInfo = new MergeInfo();

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java Thu Nov  1 16:29:05 2012
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.maste
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
@@ -33,6 +34,8 @@ public class ZooTabletStateStore extends
   
   private static final Logger log = Logger.getLogger(ZooTabletStateStore.class);
   final private DistributedStore store;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public ZooTabletStateStore(DistributedStore store) {
     this.store = store;
@@ -123,7 +126,7 @@ public class ZooTabletStateStore extends
     if (assignment.tablet.compareTo(Constants.ROOT_TABLET_EXTENT) != 0)
       throw new IllegalArgumentException("You can only store the root tablet location");
     String value = AddressUtil.toString(assignment.server.getLocation()) + "|" + assignment.server.getSession();
-    store.put(Constants.ZROOT_TABLET_FUTURE_LOCATION, value.getBytes());
+    store.put(Constants.ZROOT_TABLET_FUTURE_LOCATION, value.getBytes(utf8));
   }
   
   @Override
@@ -134,8 +137,8 @@ public class ZooTabletStateStore extends
     if (assignment.tablet.compareTo(Constants.ROOT_TABLET_EXTENT) != 0)
       throw new IllegalArgumentException("You can only store the root tablet location");
     String value = AddressUtil.toString(assignment.server.getLocation()) + "|" + assignment.server.getSession();
-    store.put(Constants.ZROOT_TABLET_LOCATION, value.getBytes());
-    store.put(Constants.ZROOT_TABLET_LAST_LOCATION, value.getBytes());
+    store.put(Constants.ZROOT_TABLET_LOCATION, value.getBytes(utf8));
+    store.put(Constants.ZROOT_TABLET_LAST_LOCATION, value.getBytes(utf8));
     store.remove(Constants.ZROOT_TABLET_FUTURE_LOCATION);
     log.debug("Put down root tablet location");
   }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.master.state.tables;
 
+import java.nio.charset.Charset;
 import java.security.SecurityPermission;
 import java.util.Collections;
 import java.util.HashMap;
@@ -53,6 +54,8 @@ public class TableManager {
   
   private final Instance instance;
   private ZooCache zooStateCache;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public static void prepareNewTableState(String instanceId, String tableId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
@@ -61,10 +64,10 @@ public class TableManager {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_CONF, new byte[0], existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAME, tableName.getBytes(), existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_STATE, state.name().getBytes(), existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, "0".getBytes(), existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_ID, "0".getBytes(), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAME, tableName.getBytes(utf8), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_STATE, state.name().getBytes(utf8), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, "0".getBytes(utf8), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_ID, "0".getBytes(utf8), existsPolicy);
   }
   
   public synchronized static TableManager getInstance() {
@@ -112,7 +115,7 @@ public class TableManager {
     String statePath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
     
     try {
-      ZooReaderWriter.getRetryingInstance().mutate(statePath, (byte[]) newState.name().getBytes(), ZooUtil.PUBLIC, new Mutator() {
+      ZooReaderWriter.getRetryingInstance().mutate(statePath, (byte[]) newState.name().getBytes(utf8), ZooUtil.PUBLIC, new Mutator() {
         @Override
         public byte[] mutate(byte[] oldData) throws Exception {
           TableState oldState = TableState.UNKNOWN;
@@ -139,7 +142,7 @@ public class TableManager {
           if (!transition)
             throw new IllegalTableTransitionException(oldState, newState);
           log.debug("Transitioning state for table " + tableId + " from " + oldState + " to " + newState);
-          return newState.name().getBytes();
+          return newState.name().getBytes(utf8);
         }
       });
     } catch (Exception e) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/BulkImport.java Thu Nov  1 16:29:05 2012
@@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -333,6 +334,8 @@ class CopyFailed extends MasterRepo {
   private String source;
   private String bulk;
   private String error;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public CopyFailed(String tableId, String source, String bulk, String error) {
     this.tableId = tableId;
@@ -427,7 +430,7 @@ class CopyFailed extends MasterRepo {
         if (fs.exists(dest))
           continue;
         
-        bifCopyQueue.addWork(orig.getName(), (failure + "," + dest).getBytes());
+        bifCopyQueue.addWork(orig.getName(), (failure + "," + dest).getBytes(utf8));
         workIds.add(orig.getName());
         log.debug("tid " + tid + " added to copyq: " + orig + " to " + dest + ": failed");
       }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.master.tableOps;
 
+import java.nio.charset.Charset;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -185,6 +186,8 @@ public class CompactRange extends Master
   private byte[] startRow;
   private byte[] endRow;
   private IteratorConfig iterators;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public CompactRange(String tableId, byte[] startRow, byte[] endRow, List<IteratorSetting> iterators) throws ThriftTableOperationException {
     this.tableId = tableId;
@@ -239,7 +242,7 @@ public class CompactRange extends Master
             encodedIterators.append(new String(hex.encode(IteratorUtil.encodeIteratorSettings(iterators))));
           }
           
-          return ("" + flushID + encodedIterators).getBytes();
+          return ("" + flushID + encodedIterators).getBytes(utf8);
         }
       });
       
@@ -272,7 +275,7 @@ public class CompactRange extends Master
           encodedIterators.append(tokens[i]);
         }
         
-        return ("" + flushID + encodedIterators).getBytes();
+        return ("" + flushID + encodedIterators).getBytes(utf8);
       }
     });
 

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/ImportTable.java Thu Nov  1 16:29:05 2012
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Serializable;
+import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -170,6 +171,8 @@ class PopulateMetadataTable extends Mast
   private static final long serialVersionUID = 1L;
   
   private ImportedTableInfo tableInfo;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   PopulateMetadataTable(ImportedTableInfo ti) {
     this.tableInfo = ti;
@@ -246,14 +249,14 @@ class PopulateMetadataTable extends Mast
             
             if (m == null) {
               m = new Mutation(metadataRow);
-              Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
+              Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes(utf8))));
               currentRow = metadataRow;
             }
             
             if (!currentRow.equals(metadataRow)) {
               mbw.addMutation(m);
               m = new Mutation(metadataRow);
-              Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
+              Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes(utf8))));
             }
             
             m.put(key.getColumnFamily(), cq, val);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/RenameTable.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.server.master.tableOps;
 
+import java.nio.charset.Charset;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -37,6 +39,8 @@ public class RenameTable extends MasterR
   private String tableId;
   private String oldTableName;
   private String newTableName;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
@@ -70,7 +74,7 @@ public class RenameTable extends MasterR
             throw new ThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND,
                 "Name changed while processing");
           }
-          return newTableName.getBytes();
+          return newTableName.getBytes(utf8);
         }
       });
       Tables.clearCache(instance);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/Utils.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/Utils.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/Utils.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tableOps/Utils.java Thu Nov  1 16:29:05 2012
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master.tableOps;
 
 import java.math.BigInteger;
+import java.nio.charset.Charset;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -39,6 +40,8 @@ import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 
 public class Utils {
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   static void checkTableDoesNotExist(Instance instance, String tableName, String tableId, TableOperation operation) throws ThriftTableOperationException {
     
@@ -54,12 +57,12 @@ public class Utils {
     try {
       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
       final String ntp = ZooUtil.getRoot(instance) + Constants.ZTABLES;
-      byte[] nid = zoo.mutate(ntp, "0".getBytes(), ZooUtil.PUBLIC, new Mutator() {
+      byte[] nid = zoo.mutate(ntp, "0".getBytes(utf8), ZooUtil.PUBLIC, new Mutator() {
         @Override
         public byte[] mutate(byte[] currentValue) throws Exception {
           BigInteger nextId = new BigInteger(new String(currentValue), Character.MAX_RADIX);
           nextId = nextId.add(BigInteger.ONE);
-          return nextId.toString(Character.MAX_RADIX).getBytes();
+          return nextId.toString(Character.MAX_RADIX).getBytes(utf8);
         }
       });
       return new String(nid);
@@ -95,7 +98,7 @@ public class Utils {
   public static long reserveHdfsDirectory(String directory, long tid) throws KeeperException, InterruptedException {
     Instance instance = HdfsZooInstance.getInstance();
     
-    String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes()));
+    String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes(utf8)));
     
     IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
     
@@ -107,12 +110,12 @@ public class Utils {
   
   public static void unreserveHdfsDirectory(String directory, long tid) throws KeeperException, InterruptedException {
     Instance instance = HdfsZooInstance.getInstance();
-    String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes()));
+    String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes(utf8)));
     ZooReservation.release(ZooReaderWriter.getRetryingInstance(), resvPath, String.format("%016x", tid));
   }
   
   private static Lock getLock(String tableId, long tid, boolean writeLock) throws Exception {
-    byte[] lockData = String.format("%016x", tid).getBytes();
+    byte[] lockData = String.format("%016x", tid).getBytes(utf8);
     ZooQueueLock qlock = new ZooQueueLock(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZTABLE_LOCKS + "/" + tableId, false);
     Lock lock = DistributedReadWriteLock.recoverLock(qlock, lockData);
     if (lock == null) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tserverOps/ShutdownTServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tserverOps/ShutdownTServer.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tserverOps/ShutdownTServer.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/tserverOps/ShutdownTServer.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.server.master.tserverOps;
 
+import java.nio.charset.Charset;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.util.AddressUtil;
@@ -39,6 +41,8 @@ public class ShutdownTServer extends Mas
   private static final Logger log = Logger.getLogger(ShutdownTServer.class);
   private TServerInstance server;
   private boolean force;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public ShutdownTServer(TServerInstance server, boolean force) {
     this.server = server;
@@ -59,7 +63,7 @@ public class ShutdownTServer extends Mas
       ZooLock.deleteLock(path);
       path = ZooUtil.getRoot(m.getInstance()) + Constants.ZDEADTSERVERS + "/" + tserver;
       IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-      zoo.putPersistentData(path, "forced down".getBytes(), NodeExistsPolicy.OVERWRITE);
+      zoo.putPersistentData(path, "forced down".getBytes(utf8), NodeExistsPolicy.OVERWRITE);
       return null;
     }
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/metanalysis/IndexMeta.java Thu Nov  1 16:29:05 2012
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.metanalysis;
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -52,12 +53,14 @@ import org.apache.log4j.Logger;
 
 public class IndexMeta extends Configured implements Tool {
   
+  private static final Charset utf8 = Charset.forName("UTF8");
+    
   public static class IndexMapper extends Mapper<LogFileKey,LogFileValue,Text,Mutation> {
     private static final Text CREATE_EVENTS_TABLE = new Text("createEvents");
     private static final Text TABLET_EVENTS_TABLE = new Text("tabletEvents");
     private Map<Integer,KeyExtent> tabletIds = new HashMap<Integer,KeyExtent>();
     private String uuid = null;
-    
+
     @Override
     protected void setup(Context context) throws java.io.IOException, java.lang.InterruptedException {
       tabletIds = new HashMap<Integer,KeyExtent>();
@@ -101,14 +104,14 @@ public class IndexMeta extends Configure
       
       if (prevRow != null) {
         Mutation createEvent = new Mutation(new Text(m.getRow()));
-        createEvent.put(prevRow, new Text(String.format("%020d", timestamp)), new Value(metaTablet.toString().getBytes()));
+        createEvent.put(prevRow, new Text(String.format("%020d", timestamp)), new Value(metaTablet.toString().getBytes(utf8)));
         context.write(CREATE_EVENTS_TABLE, createEvent);
       }
       
       Mutation tabletEvent = new Mutation(new Text(m.getRow()));
       tabletEvent.put(new Text(String.format("%020d", timestamp)), new Text("mut"), new Value(serMut));
-      tabletEvent.put(new Text(String.format("%020d", timestamp)), new Text("mtab"), new Value(metaTablet.toString().getBytes()));
-      tabletEvent.put(new Text(String.format("%020d", timestamp)), new Text("log"), new Value(logFile.getBytes()));
+      tabletEvent.put(new Text(String.format("%020d", timestamp)), new Text("mtab"), new Value(metaTablet.toString().getBytes(utf8)));
+      tabletEvent.put(new Text(String.format("%020d", timestamp)), new Text("log"), new Value(logFile.getBytes(utf8)));
       context.write(TABLET_EVENTS_TABLE, tabletEvent);
     }
   }
@@ -146,7 +149,7 @@ public class IndexMeta extends Configure
     
     job.setOutputFormatClass(AccumuloOutputFormat.class);
     AccumuloOutputFormat.setZooKeeperInstance(job.getConfiguration(), instance, zookeepers);
-    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), user, pass.getBytes(), false, null);
+    AccumuloOutputFormat.setOutputInfo(job.getConfiguration(), user, pass.getBytes(utf8), false, null);
     
     job.setMapperClass(IndexMapper.class);
 

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/ZooKeeperStatus.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/ZooKeeperStatus.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/ZooKeeperStatus.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/ZooKeeperStatus.java Thu Nov  1 16:29:05 2012
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.monitor;
 
 import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -35,6 +36,8 @@ public class ZooKeeperStatus implements 
   private static final Logger log = Logger.getLogger(ZooKeeperStatus.class);
   
   private volatile boolean stop = false;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public static class ZooKeeperState {
     public final String keeper;
@@ -80,7 +83,7 @@ public class ZooKeeperStatus implements 
             addr = new InetSocketAddress(parts[0], 2181);
           
           transport = TTimeoutTransport.create(addr, 10 * 1000l);
-          transport.write("stat\n".getBytes(), 0, 5);
+          transport.write("stat\n".getBytes(utf8), 0, 5);
           StringBuilder response = new StringBuilder();
           try {
             transport.flush();

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java Thu Nov  1 16:29:05 2012
@@ -22,6 +22,7 @@ import java.io.FilePermission;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
 import java.security.AccessControlContext;
 import java.security.AccessController;
 import java.security.PermissionCollection;
@@ -64,6 +65,8 @@ import org.apache.hadoop.mapred.JobTrack
 public class DefaultServlet extends BasicServlet {
   
   private static final long serialVersionUID = 1L;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   @Override
   protected String getTitle(HttpServletRequest req) {
@@ -90,7 +93,7 @@ public class DefaultServlet extends Basi
     		  while ((n = data.read(buffer)) > 0)
     			  out.write(buffer, 0, n);
     	  } else {
-    		  out.write(("could not get resource " + path + "").getBytes());
+    		  out.write(("could not get resource " + path + "").getBytes(utf8));
     	  }
       } finally {
     	  data.close();

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/trace/Basic.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/trace/Basic.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/trace/Basic.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/monitor/servlets/trace/Basic.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.monitor.servlets.trace;
 
+import java.nio.charset.Charset;
 import java.util.Date;
 
 import javax.servlet.http.HttpServletRequest;
@@ -36,6 +37,8 @@ import org.apache.accumulo.server.monito
 abstract class Basic extends BasicServlet {
   
   private static final long serialVersionUID = 1L;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public static String getStringParameter(HttpServletRequest req, String name, String defaultValue) {
     String result = req.getParameter(name);
@@ -65,7 +68,7 @@ abstract class Basic extends BasicServle
   protected Scanner getScanner(StringBuilder sb) throws AccumuloException {
     AccumuloConfiguration conf = Monitor.getSystemConfiguration();
     String user = conf.get(Property.TRACE_USER);
-    byte[] passwd = conf.get(Property.TRACE_PASSWORD).getBytes();
+    byte[] passwd = conf.get(Property.TRACE_PASSWORD).getBytes(utf8);
     String table = conf.get(Property.TRACE_TABLE);
     try {
       Connector conn = HdfsZooInstance.getInstance().getConnector(user, passwd);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java Thu Nov  1 16:29:05 2012
@@ -22,6 +22,7 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecurityPermission;
@@ -39,6 +40,8 @@ import org.apache.commons.codec.binary.B
 public class SecurityConstants {
   private static SecurityPermission SYSTEM_CREDENTIALS_PERMISSION = new SecurityPermission("systemCredentialsPermission");
   
+  private static final Charset utf8 = Charset.forName("UTF8");
+  
   public static final String SYSTEM_USERNAME = "!SYSTEM";
   private static final byte[] SYSTEM_PASSWORD = makeSystemPassword();
   private static final AuthInfo systemCredentials = new AuthInfo(SYSTEM_USERNAME, ByteBuffer.wrap(SYSTEM_PASSWORD), HdfsZooInstance.getInstance()
@@ -54,8 +57,8 @@ public class SecurityConstants {
   }
   
   private static byte[] makeSystemPassword() {
-    byte[] version = Constants.VERSION.getBytes();
-    byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes();
+    byte[] version = Constants.VERSION.getBytes(utf8);
+    byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes(utf8);
     try {
       confChecksum = getSystemConfigChecksum();
     } catch (NoSuchAlgorithmException e) {
@@ -97,10 +100,10 @@ public class SecurityConstants {
     try {
       byte[] buff = new byte[in.readInt()];
       in.readFully(buff);
-      versionFails = !Arrays.equals(buff, Constants.VERSION.getBytes());
+      versionFails = !Arrays.equals(buff, Constants.VERSION.getBytes(utf8));
       buff = new byte[in.readInt()];
       in.readFully(buff);
-      instanceFails = !Arrays.equals(buff, HdfsZooInstance.getInstance().getInstanceID().getBytes());
+      instanceFails = !Arrays.equals(buff, HdfsZooInstance.getInstance().getInstanceID().getBytes(utf8));
       buff = new byte[in.readInt()];
       in.readFully(buff);
       confFails = !Arrays.equals(buff, getSystemConfigChecksum());
@@ -131,14 +134,14 @@ public class SecurityConstants {
       
       // seed the config with the version and instance id, so at least
       // it's not empty
-      md.update(Constants.VERSION.getBytes());
-      md.update(HdfsZooInstance.getInstance().getInstanceID().getBytes());
+      md.update(Constants.VERSION.getBytes(utf8));
+      md.update(HdfsZooInstance.getInstance().getInstanceID().getBytes(utf8));
       
       for (Entry<String,String> entry : ServerConfiguration.getSiteConfiguration()) {
         // only include instance properties
         if (entry.getKey().startsWith(Property.INSTANCE_PREFIX.toString())) {
-          md.update(entry.getKey().getBytes());
-          md.update(entry.getValue().getBytes());
+          md.update(entry.getKey().getBytes(utf8));
+          md.update(entry.getValue().getBytes(utf8));
         }
       }
       

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java Thu Nov  1 16:29:05 2012
@@ -143,7 +143,7 @@ public class SecurityOperation {
   
   private void authenticate(String user, ByteBuffer password, String instance) throws ThriftSecurityException {
     if (!instance.equals(HdfsZooInstance.getInstance().getInstanceID()))
-      throw new ThriftSecurityException(user, SecurityErrorCode.INVALID_INSTANCEID);
+      throw new ThriftSecurityException(user + '_' + instance + '_' + HdfsZooInstance.getInstance().getInstanceID(), SecurityErrorCode.INVALID_INSTANCEID);
     
     if (user.equals(SecurityConstants.SYSTEM_USERNAME)) {
       if (Arrays.equals(SecurityConstants.getSystemCredentials().password.array(), password.array())
@@ -153,8 +153,10 @@ public class SecurityOperation {
         throw new ThriftSecurityException(user, SecurityErrorCode.BAD_CREDENTIALS);
     }
     
-    if (!authenticator.authenticateUser(user, password, instance))
+    if (!authenticator.authenticateUser(user, password, instance)) {
+      log.debug("It appears that " + user + " password isn't " + new String(password.array()));
       throw new ThriftSecurityException(user, SecurityErrorCode.BAD_CREDENTIALS);
+    }
   }
   
   private void authenticate(AuthInfo credentials) throws ThriftSecurityException {
@@ -817,7 +819,7 @@ public class SecurityOperation {
       throw new ThriftSecurityException(credentials.user, SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
   }
-
+  
   public boolean canExport(AuthInfo credentials, String tableId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials.user, tableId, TablePermission.READ, false);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Thu Nov  1 16:29:05 2012
@@ -24,6 +24,7 @@ package org.apache.accumulo.server.table
  */
 
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -145,6 +146,8 @@ import org.apache.zookeeper.KeeperExcept
 
 public class Tablet {
   
+  private Charset utf8 = Charset.forName("UTF8");
+    
   enum MajorCompactionReason {
     // do not change the order, the order of this enum determines the order
     // in which queued major compactions are executed
@@ -2174,7 +2177,7 @@ public class Tablet {
     private DataFileValue stats;
     private String mergeFile;
     private long flushId;
-    
+
     MinorCompactionTask(String mergeFile, CommitSession commitSession, long flushId) {
       queued = System.currentTimeMillis();
       minorCompactionWaitingToStart = true;
@@ -2405,7 +2408,7 @@ public class Tablet {
       List<IteratorSetting> allIters = new ArrayList<IteratorSetting>();
       for (int i = 1; i < tokens.length; i++) {
         Hex hex = new Hex();
-        List<IteratorSetting> iters = IteratorUtil.decodeIteratorSettings(hex.decode(tokens[i].split("=")[1].getBytes()));
+        List<IteratorSetting> iters = IteratorUtil.decodeIteratorSettings(hex.decode(tokens[i].split("=")[1].getBytes(utf8)));
         allIters.addAll(iters);
       }
       

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Thu Nov  1 16:29:05 2012
@@ -28,6 +28,7 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -232,6 +233,8 @@ public class TabletServer extends Abstra
   
   private ServerConfiguration serverConfig;
   private LogSorter logSorter = null;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   public TabletServer(ServerConfiguration conf, FileSystem fs) {
     super();
@@ -2667,7 +2670,7 @@ public class TabletServer extends Abstra
         }
       };
       
-      byte[] lockContent = new ServerServices(getClientAddressString(), Service.TSERV_CLIENT).toString().getBytes();
+      byte[] lockContent = new ServerServices(getClientAddressString(), Service.TSERV_CLIENT).toString().getBytes(utf8);
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP);
         

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/tabletserver/UniqueNameAllocator.java Thu Nov  1 16:29:05 2012
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.tabletserver;
 
+import java.nio.charset.Charset;
 import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
@@ -36,6 +37,8 @@ public class UniqueNameAllocator {
   private long maxAllocated = 0;
   private String nextNamePath;
   private Random rand;
+
+  private static final Charset utf8 = Charset.forName("UTF8");
   
   private UniqueNameAllocator() {
     nextNamePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZNEXT_FILE;
@@ -52,7 +55,7 @@ public class UniqueNameAllocator {
           public byte[] mutate(byte[] currentValue) throws Exception {
             long l = Long.parseLong(new String(currentValue), Character.MAX_RADIX);
             l += allocate;
-            return Long.toString(l, Character.MAX_RADIX).getBytes();
+            return Long.toString(l, Character.MAX_RADIX).getBytes(utf8);
           }
         });
         

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java Thu Nov  1 16:29:05 2012
@@ -23,6 +23,8 @@ import java.util.HashMap;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -79,15 +81,15 @@ public class State {
     return (String) stateMap.get(key);
   }
   
-  public Integer getInteger(String key) {
-    return (Integer) stateMap.get(key);
+  public Long getLong(String key) {
+    return (Long) stateMap.get(key);
   }
   
   public String getProperty(String key) {
     return props.getProperty(key);
   }
   
-  public Connector getConnector() throws Exception {
+  public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
     if (connector == null) {
       String instance = props.getProperty("INSTANCE");
       String zookeepers = props.getProperty("ZOOKEEPERS");
@@ -101,7 +103,7 @@ public class State {
     String username = props.getProperty("USERNAME");
     String password = props.getProperty("PASSWORD");
     String instance = props.getProperty("INSTANCE");
-    return new AuthInfo(username, ByteBuffer.wrap(password.getBytes()), instance);
+    return new AuthInfo(username, ByteBuffer.wrap(password.getBytes()), new ZooKeeperInstance(instance, props.getProperty("ZOOKEEPERS")).getInstanceID());
   }
 
   public Instance getInstance() {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java Thu Nov  1 16:29:05 2012
@@ -27,7 +27,7 @@ public class Commit extends Test {
   public void visit(State state, Properties props) throws Exception {
     state.getMultiTableBatchWriter().flush();
     
-    log.debug("Committed " + state.getInteger("numWrites") + " writes.  Total writes: " + state.getInteger("totalWrites"));
+    log.debug("Committed " + state.getLong("numWrites") + " writes.  Total writes: " + state.getLong("totalWrites"));
     state.set("numWrites", new Integer(0));
   }
   

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java Thu Nov  1 16:29:05 2012
@@ -113,6 +113,6 @@ public class ImageFixture extends Fixtur
     conn.tableOperations().delete(imageTableName);
     conn.tableOperations().delete(indexTableName);
     
-    log.debug("Final total of writes: " + state.getInteger("totalWrites"));
+    log.debug("Final total of writes: " + state.getLong("totalWrites"));
   }
 }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java Thu Nov  1 16:29:05 2012
@@ -69,8 +69,8 @@ public class Write extends Test {
     m.put(META_COLUMN_FAMILY, SHA1_COLUMN_QUALIFIER, new Value(hash));
     
     // update write counts
-    state.set("numWrites", state.getInteger("numWrites") + 1);
-    Integer totalWrites = state.getInteger("totalWrites") + 1;
+    state.set("numWrites", state.getLong("numWrites") + 1);
+    Long totalWrites = state.getLong("totalWrites") + 1;
     state.set("totalWrites", totalWrites);
     
     // set count

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java Thu Nov  1 16:29:05 2012
@@ -27,8 +27,8 @@ public class Commit extends Test {
   public void visit(State state, Properties props) throws Exception {
     state.getMultiTableBatchWriter().flush();
     
-    Integer numWrites = state.getInteger("numWrites");
-    Integer totalWrites = state.getInteger("totalWrites") + numWrites;
+    Long numWrites = state.getLong("numWrites");
+    Long totalWrites = state.getLong("totalWrites") + numWrites;
     
     log.debug("Committed " + numWrites + " writes.  Total writes: " + totalWrites);
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Write.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Write.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Write.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Write.java Thu Nov  1 16:29:05 2012
@@ -76,7 +76,7 @@ public class Write extends Test {
     // add mutation
     bw.addMutation(m);
     
-    state.set("numWrites", state.getInteger("numWrites") + 1);
+    state.set("numWrites", state.getLong("numWrites") + 1);
   }
   
 }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTable.java Thu Nov  1 16:29:05 2012
@@ -32,7 +32,7 @@ public class AlterTable extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getSystemConnector();
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
     
     String tableName = WalkingSecurity.get(state).getTableName();
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/AlterTablePerm.java Thu Nov  1 16:29:05 2012
@@ -35,9 +35,7 @@ public class AlterTablePerm extends Test
     alter(state, props);
   }
   
-  public static void alter(State state, Properties props) throws Exception {
-    Connector conn;
-    
+  public static void alter(State state, Properties props) throws Exception {    
     String action = props.getProperty("task", "toggle");
     String perm = props.getProperty("perm", "random");
     String sourceUser = props.getProperty("source", "system");
@@ -65,15 +63,13 @@ public class AlterTablePerm extends Test
     boolean canGive;
     AuthInfo source;
     if ("system".equals(sourceUser)) {
-      conn = WalkingSecurity.get(state).getSystemConnector();
       source = WalkingSecurity.get(state).getSysAuthInfo();
     } else if ("table".equals(sourceUser)) {
-      conn = WalkingSecurity.get(state).getTableConnector();
       source = WalkingSecurity.get(state).getTabAuthInfo();
     } else {
-      conn = state.getConnector();
       source = state.getAuthInfo();
     }
+    Connector conn = state.getInstance().getConnector(source);
     
     canGive = WalkingSecurity.get(state).canGrantTable(source, target, WalkingSecurity.get(state).getTableName());
 

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Authenticate.java Thu Nov  1 16:29:05 2012
@@ -30,24 +30,21 @@ public class Authenticate extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getSystemConnector();
-    
-    authenticate(conn, state, props);
+    authenticate(WalkingSecurity.get(state).getSysAuthInfo(), state, props);
   }
   
-  public static void authenticate(Connector conn, State state, Properties props) throws Exception {
+  public static void authenticate(AuthInfo auth, State state, Properties props) throws Exception {
     String targetProp = props.getProperty("target");
     boolean success = Boolean.parseBoolean(props.getProperty("valid"));
     
+    Connector conn = state.getInstance().getConnector(auth);
+    
     String target;
     
-    AuthInfo auth;
     if (targetProp.equals("table")) {
       target = WalkingSecurity.get(state).getTabUserName();
-      auth = WalkingSecurity.get(state).getTabAuthInfo();
     } else {
       target = WalkingSecurity.get(state).getSysUserName();
-      auth = WalkingSecurity.get(state).getSysAuthInfo();
     }
     boolean exists = WalkingSecurity.get(state).userExists(target);
     // Copy so if failed it doesn't mess with the password stored in state

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/ChangePass.java Thu Nov  1 16:29:05 2012
@@ -31,20 +31,17 @@ public class ChangePass extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn;
-    
     String target = props.getProperty("target");
     String source = props.getProperty("source");
     
     AuthInfo auth;
     if (source.equals("system")) {
-      conn = WalkingSecurity.get(state).getSystemConnector();
       auth = WalkingSecurity.get(state).getSysAuthInfo();
     } else {
-      conn = WalkingSecurity.get(state).getTableConnector();
       auth = WalkingSecurity.get(state).getTabAuthInfo();
     }
-    
+    Connector conn = state.getInstance().getConnector(auth);
+        
     boolean hasPerm;
     boolean targetExists;
     if (target.equals("table")) {
@@ -64,6 +61,7 @@ public class ChangePass extends Test {
     newPass = bi.toString(36).getBytes();
     
     try {
+      log.debug("Changing password for user " + target + " to " + new String(newPass));
       conn.securityOperations().changeUserPassword(target, newPass);
     } catch (AccumuloSecurityException ae) {
       switch (ae.getErrorCode()) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateTable.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateTable.java Thu Nov  1 16:29:05 2012
@@ -31,7 +31,7 @@ public class CreateTable extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getSystemConnector();
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
     
     String tableName = WalkingSecurity.get(state).getTableName();
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateUser.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateUser.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateUser.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/CreateUser.java Thu Nov  1 16:29:05 2012
@@ -29,7 +29,7 @@ public class CreateUser extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getSystemConnector();
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
     
     String tableUserName = WalkingSecurity.get(state).getTabUserName();
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropTable.java Thu Nov  1 16:29:05 2012
@@ -37,15 +37,13 @@ public class DropTable extends Test {
   
   public static void dropTable(State state, Properties props) throws Exception {
     String sourceUser = props.getProperty("source", "system");
-    Connector conn;
     AuthInfo auth;
     if (sourceUser.equals("table")) {
       auth = WalkingSecurity.get(state).getTabAuthInfo();
-      conn = WalkingSecurity.get(state).getTableConnector();
     } else {
       auth = WalkingSecurity.get(state).getSysAuthInfo();
-      conn = WalkingSecurity.get(state).getSystemConnector();
     }
+    Connector conn = state.getInstance().getConnector(auth);
     
     String tableName = WalkingSecurity.get(state).getTableName();
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropUser.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropUser.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropUser.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/DropUser.java Thu Nov  1 16:29:05 2012
@@ -28,7 +28,7 @@ public class DropUser extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getSystemConnector();
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getSysAuthInfo());
     
     String tableUserName = WalkingSecurity.get(state).getTabUserName();
     

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityFixture.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityFixture.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityFixture.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityFixture.java Thu Nov  1 16:29:05 2012
@@ -19,7 +19,6 @@ package org.apache.accumulo.server.test.
 import java.net.InetAddress;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -31,10 +30,7 @@ public class SecurityFixture extends Fix
   @Override
   public void setUp(State state) throws Exception {
     String secTableName, systemUserName, tableUserName;
-    Connector sysConn;
-    
     Connector conn = state.getConnector();
-    Instance instance = state.getInstance();
     
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
     
@@ -44,12 +40,13 @@ public class SecurityFixture extends Fix
     
     byte[] sysUserPass = "sysUser".getBytes();
     conn.securityOperations().createUser(systemUserName, sysUserPass, new Authorizations());
-    sysConn = instance.getConnector(systemUserName, sysUserPass);
     
+    state.set("rootUserPass", state.getAuthInfo().password.array());
+    
+    WalkingSecurity.get(state).setSysUserName(systemUserName);
     WalkingSecurity.get(state).createUser(systemUserName, sysUserPass);
     
     WalkingSecurity.get(state).changePassword(tableUserName, new byte[0]);
-    WalkingSecurity.get(state).setSystemConnector(sysConn);
     
     WalkingSecurity.get(state).setTableName(secTableName);
     WalkingSecurity.get(state).setTabUserName(tableUserName);

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SecurityHelper.java Thu Nov  1 16:29:05 2012
@@ -86,7 +86,7 @@ public class SecurityHelper {
   }
   
   public static boolean sysUserPassTransient(State state) {
-    return System.currentTimeMillis() - state.getInteger(masterPass + "time") < 1000;
+    return System.currentTimeMillis() - state.getLong(masterPass + "time") < 1000;
   }
 
   public static byte[] getTabUserPass(State state) {
@@ -100,7 +100,7 @@ public class SecurityHelper {
   }
   
   public static boolean tabUserPassTransient(State state) {
-    return System.currentTimeMillis() - state.getInteger(tUserPass + "time") < 1000;
+    return System.currentTimeMillis() - state.getLong(tUserPass + "time") < 1000;
   }
 
   public static boolean getTabUserExists(State state) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SetAuths.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SetAuths.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SetAuths.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/SetAuths.java Thu Nov  1 16:29:05 2012
@@ -23,6 +23,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.AuthInfo;
 import org.apache.accumulo.server.test.randomwalk.State;
 import org.apache.accumulo.server.test.randomwalk.Test;
 
@@ -30,7 +31,7 @@ public class SetAuths extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn;
+    AuthInfo auth;
     
     String authsString = props.getProperty("auths", "_random");
     
@@ -38,15 +39,16 @@ public class SetAuths extends Test {
     String target;
     if ("table".equals(targetUser)) {
       target = WalkingSecurity.get(state).getTabUserName();
-      conn = WalkingSecurity.get(state).getSystemConnector();
+      auth = WalkingSecurity.get(state).getSysAuthInfo();
     } else {
       target = WalkingSecurity.get(state).getSysUserName();
-      conn = state.getConnector();
+      auth = state.getAuthInfo();
     }
-
+    Connector conn = state.getInstance().getConnector(auth);
+    
     boolean exists = WalkingSecurity.get(state).userExists(target);
-    boolean hasPermission = WalkingSecurity.get(state).canChangeAuthorizations(WalkingSecurity.get(state).getSysAuthInfo(), target);
-
+    boolean hasPermission = WalkingSecurity.get(state).canChangeAuthorizations(auth, target);
+    
     Authorizations auths;
     if (authsString.equals("_random")) {
       String[] possibleAuths = WalkingSecurity.get(state).getAuthsArray();

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/TableOp.java Thu Nov  1 16:29:05 2012
@@ -53,7 +53,7 @@ public class TableOp extends Test {
   
   @Override
   public void visit(State state, Properties props) throws Exception {
-    Connector conn = WalkingSecurity.get(state).getTableConnector();
+    Connector conn = state.getInstance().getConnector(WalkingSecurity.get(state).getTabAuthInfo());
 
     String action = props.getProperty("action", "_random");
     TablePermission tp;
@@ -145,7 +145,7 @@ public class TableOp extends Test {
           if (ambiguousZone) {
             Thread.sleep(1000);
             try {
-              writer = conn.createBatchWriter(tableName, 9000l, 0l, 1);
+              writer = conn.createBatchWriter(tableName, new BatchWriterConfig().setMaxWriteThreads(1));
               writer.addMutation(m);
               writer.close();
             } catch (MutationsRejectedException mre2) {

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java?rev=1404662&r1=1404661&r2=1404662&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/test/randomwalk/security/Validate.java Thu Nov  1 16:29:05 2012
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.security
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.security.thrift.ThriftSecurityException;
 import org.apache.accumulo.server.test.randomwalk.State;
 import org.apache.accumulo.server.test.randomwalk.Test;
 import org.apache.log4j.Logger;
@@ -51,9 +52,9 @@ public class Validate extends Test {
     
     Properties props = new Properties();
     props.setProperty("target", "system");
-    Authenticate.authenticate(conn, state, props);
+    Authenticate.authenticate(state.getAuthInfo(), state, props);
     props.setProperty("target", "table");
-    Authenticate.authenticate(conn, state, props);
+    Authenticate.authenticate(state.getAuthInfo(), state, props);
     
     for (String user : new String[] {WalkingSecurity.get(state).getSysUserName(), WalkingSecurity.get(state).getTabUserName()}) {
       for (SystemPermission sp : SystemPermission.values()) {
@@ -101,12 +102,13 @@ public class Validate extends Test {
       
     }
     
-    Authorizations auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
     Authorizations accuAuths;
+    Authorizations auths;
     try {
+      auths = WalkingSecurity.get(state).getUserAuthorizations(WalkingSecurity.get(state).getTabAuthInfo());
       accuAuths = conn.securityOperations().getUserAuthorizations(WalkingSecurity.get(state).getTabUserName());
-    } catch (AccumuloSecurityException ae) {
-      if (ae.getErrorCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {
+    } catch (ThriftSecurityException ae) {
+      if (ae.getCode().equals(SecurityErrorCode.USER_DOESNT_EXIST)) {
         if (tableUserExists)
           throw new AccumuloException("Table user didn't exist when they should.", ae);
         else