You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/04/23 19:23:01 UTC

svn commit: r1471056 - in /accumulo/branches/1.4/src: core/src/main/java/org/apache/accumulo/core/util/ proxy/src/main/java/org/apache/accumulo/proxy/ proxy/src/main/java/org/apache/accumulo/proxy/thrift/ proxy/src/main/thrift/ proxy/src/test/java/org/...

Author: kturner
Date: Tue Apr 23 17:23:01 2013
New Revision: 1471056

URL: http://svn.apache.org/r1471056
Log:
ACCUMULO-1237 minimized 1.4 and 1.5 proxy diffs and fixed a few proxy issues

Modified:
    accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
    accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
    accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java
    accumulo/branches/1.4/src/proxy/src/main/thrift/proxy.thrift
    accumulo/branches/1.4/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
    accumulo/branches/1.4/src/proxy/src/test/resources/log4j.properties

Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java (original)
+++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java Tue Apr 23 17:23:01 2013
@@ -28,7 +28,7 @@ public class ByteBufferUtil {
   public static byte[] toBytes(ByteBuffer buffer) {
     if (buffer == null)
       return null;
-    return Arrays.copyOfRange(buffer.array(), buffer.position(), buffer.remaining());
+    return Arrays.copyOfRange(buffer.array(), buffer.position(), buffer.limit());
   }
   
   public static List<ByteBuffer> toByteBuffers(Collection<byte[]> bytesList) {

Modified: accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java (original)
+++ accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java Tue Apr 23 17:23:01 2013
@@ -165,74 +165,55 @@ public class ProxyServer implements Accu
   }
   
   protected Connector getConnector(ByteBuffer login) throws Exception {
-    AuthInfo user = CredentialHelper.fromByteArray(login.array());
+    AuthInfo user = CredentialHelper.fromByteArray(ByteBufferUtil.toBytes(login));
     if (user == null)
       throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException("unknown user");
     Connector connector = instance.getConnector(user.getUser(), user.getPassword());
     return connector;
   }
   
-  private void handleExceptionTNF(Exception ex) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
-    try {
-      throw ex;
-    } catch (AccumuloException e) {
-      if (e.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException) e.getCause();
-        if (ttoe.type == TableOperationExceptionType.NOTFOUND) {
-          throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-        }
-      }
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
-    } catch (AccumuloSecurityException e) {
-      if (e.getErrorCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST))
+  private void handleAccumuloException(AccumuloException e) throws org.apache.accumulo.proxy.thrift.TableNotFoundException,
+      org.apache.accumulo.proxy.thrift.AccumuloException {
+    if (e.getCause() instanceof ThriftTableOperationException) {
+      ThriftTableOperationException ttoe = (ThriftTableOperationException) e.getCause();
+      if (ttoe.type == TableOperationExceptionType.NOTFOUND) {
         throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException(e.toString());
-    } catch (TableNotFoundException e) {
-      throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(ex.toString());
-    } catch (Exception e) {
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
+      }
     }
+    throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
   }
   
-  private void handleExceptionTNFNoSecurity(Exception ex) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
+  private void handleAccumuloSecurityException(AccumuloSecurityException e) throws org.apache.accumulo.proxy.thrift.TableNotFoundException,
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException {
+    if (e.getErrorCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST))
+      throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
+    throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException(e.toString());
+  }
+
+  private void handleExceptionTNF(Exception ex) throws org.apache.accumulo.proxy.thrift.AccumuloException,
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
       throw ex;
     } catch (AccumuloException e) {
-      if (e.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException) e.getCause();
-        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
-          throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      }
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
+      handleAccumuloException(e);
     } catch (AccumuloSecurityException e) {
-      if (e.getErrorCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST))
-        throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
+      handleAccumuloSecurityException(e);
     } catch (TableNotFoundException e) {
       throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(ex.toString());
     } catch (Exception e) {
       throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
     }
   }
-  
+
   private void handleExceptionTEE(Exception ex) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException,
       org.apache.accumulo.proxy.thrift.TableExistsException, TException {
     try {
       throw ex;
     } catch (AccumuloException e) {
-      if (e.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException) e.getCause();
-        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
-          throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      }
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
+      handleAccumuloException(e);
     } catch (AccumuloSecurityException e) {
-      if (e.getErrorCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST))
-        throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException(e.toString());
+      handleAccumuloSecurityException(e);
     } catch (TableNotFoundException e) {
       throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(ex.toString());
     } catch (TableExistsException e) {
@@ -250,16 +231,9 @@ public class ProxyServer implements Accu
     } catch (MutationsRejectedException e) {
       throw new org.apache.accumulo.proxy.thrift.MutationsRejectedException(ex.toString());
     } catch (AccumuloException e) {
-      if (e.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException) e.getCause();
-        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
-          throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      }
-      throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
+      handleAccumuloException(e);
     } catch (AccumuloSecurityException e) {
-      if (e.getErrorCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST))
-        throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(e.toString());
-      throw new org.apache.accumulo.proxy.thrift.AccumuloSecurityException(e.toString());
+      handleAccumuloSecurityException(e);
     } catch (TableNotFoundException e) {
       throw new org.apache.accumulo.proxy.thrift.TableNotFoundException(ex.toString());
     } catch (Exception e) {
@@ -279,7 +253,7 @@ public class ProxyServer implements Accu
       throw new org.apache.accumulo.proxy.thrift.AccumuloException(e.toString());
     }
   }
-  
+
   @Override
   public int addConstraint(ByteBuffer login, String tableName, String constraintClassName) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
@@ -349,6 +323,8 @@ public class ProxyServer implements Accu
       throws org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException,
       org.apache.accumulo.proxy.thrift.AccumuloException, TException {
     try {
+      if (iterators != null && iterators.size() > 0)
+        throw new UnsupportedOperationException("compactTable does not support passing iterators until Accumulo 1.5.0");
       getConnector(login).tableOperations().compact(tableName, ByteBufferUtil.toText(startRow), ByteBufferUtil.toText(endRow), flush, wait);
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -405,7 +381,6 @@ public class ProxyServer implements Accu
       throws org.apache.accumulo.proxy.thrift.AccumuloException, org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
       org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      checkTableExists(getConnector(login), tableName);
       getConnector(login).tableOperations().flush(tableName, ByteBufferUtil.toText(startRow), ByteBufferUtil.toText(endRow), wait);
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -414,7 +389,7 @@ public class ProxyServer implements Accu
   
   @Override
   public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
       checkTableExists(getConnector(login), tableName);
       Map<String,Set<Text>> groups = getConnector(login).tableOperations().getLocalityGroups(tableName);
@@ -427,7 +402,7 @@ public class ProxyServer implements Accu
       }
       return ret;
     } catch (Exception e) {
-      handleExceptionTNFNoSecurity(e);
+      handleExceptionTNF(e);
       return null;
     }
   }
@@ -456,7 +431,7 @@ public class ProxyServer implements Accu
   
   @Override
   public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
       checkTableExists(getConnector(login), tableName);
       Map<String,String> ret = new HashMap<String,String>();
@@ -466,7 +441,7 @@ public class ProxyServer implements Accu
       }
       return ret;
     } catch (Exception e) {
-      handleExceptionTNFNoSecurity(e);
+      handleExceptionTNF(e);
       return null;
     }
   }
@@ -498,7 +473,7 @@ public class ProxyServer implements Accu
   
   @Override
   public Map<String,Integer> listConstraints(ByteBuffer login, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
       checkTableExists(getConnector(login), tableName);
       Connector connector = getConnector(login);
@@ -518,7 +493,7 @@ public class ProxyServer implements Accu
       }
       return constraints;
     } catch (Exception e) {
-      handleExceptionTNFNoSecurity(e);
+      handleExceptionTNF(e);
       return null;
     }
   }
@@ -559,7 +534,6 @@ public class ProxyServer implements Accu
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     
     try {
-      checkTableExists(getConnector(login), tableName);
       getConnector(login).tableOperations().removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + constraint);
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -570,7 +544,6 @@ public class ProxyServer implements Accu
   public void removeTableProperty(ByteBuffer login, String tableName, String property) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      checkTableExists(getConnector(login), tableName);
       getConnector(login).tableOperations().removeProperty(tableName, property);
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -610,7 +583,6 @@ public class ProxyServer implements Accu
   public void setTableProperty(ByteBuffer login, String tableName, String property, String value) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      checkTableExists(getConnector(login), tableName);
       getConnector(login).tableOperations().setProperty(tableName, property, value);
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -766,7 +738,7 @@ public class ProxyServer implements Accu
   public void changeLocalUserPassword(ByteBuffer login, String user, ByteBuffer password) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, TException {
     try {
-      getConnector(login).securityOperations().changeUserPassword(user, password.array());
+      getConnector(login).securityOperations().changeUserPassword(user, ByteBufferUtil.toBytes(password));
     } catch (Exception e) {
       handleException(e);
     }
@@ -776,7 +748,7 @@ public class ProxyServer implements Accu
   public void createLocalUser(ByteBuffer login, String user, ByteBuffer password) throws org.apache.accumulo.proxy.thrift.AccumuloException,
       org.apache.accumulo.proxy.thrift.AccumuloSecurityException, TException {
     try {
-      getConnector(login).securityOperations().createUser(user, password.array(), new Authorizations());
+      getConnector(login).securityOperations().createUser(user, ByteBufferUtil.toBytes(password), new Authorizations());
     } catch (Exception e) {
       handleException(e);
     }
@@ -818,9 +790,6 @@ public class ProxyServer implements Accu
       throws org.apache.accumulo.proxy.thrift.AccumuloException, org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
       org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      
-      checkTableExists(getConnector(login), table);
-      
       getConnector(login).securityOperations().grantTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -843,7 +812,6 @@ public class ProxyServer implements Accu
       throws org.apache.accumulo.proxy.thrift.AccumuloException, org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
       org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      checkTableExists(getConnector(login), table);
       return getConnector(login).securityOperations().hasTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -877,7 +845,6 @@ public class ProxyServer implements Accu
       throws org.apache.accumulo.proxy.thrift.AccumuloException, org.apache.accumulo.proxy.thrift.AccumuloSecurityException,
       org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
-      checkTableExists(getConnector(login), table);
       getConnector(login).securityOperations().revokeTablePermission(user, table, TablePermission.getPermissionById((byte) perm.getValue()));
     } catch (Exception e) {
       handleExceptionTNF(e);
@@ -1242,11 +1209,11 @@ public class ProxyServer implements Accu
   @Override
   public void checkIteratorConflicts(ByteBuffer login, String tableName, org.apache.accumulo.proxy.thrift.IteratorSetting setting,
       Set<org.apache.accumulo.proxy.thrift.IteratorScope> scopes) throws org.apache.accumulo.proxy.thrift.AccumuloException,
-      org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
+      org.apache.accumulo.proxy.thrift.AccumuloSecurityException, org.apache.accumulo.proxy.thrift.TableNotFoundException, TException {
     try {
       getConnector(login).tableOperations().checkIteratorConflicts(tableName, getIteratorSetting(setting), getIteratorScopes(scopes));
     } catch (Exception e) {
-      handleExceptionTNFNoSecurity(e);
+      handleExceptionTNF(e);
     }
   }
   

Modified: accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java (original)
+++ accumulo/branches/1.4/src/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java Tue Apr 23 17:23:01 2013
@@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory;
 
     public void attachIterator(ByteBuffer login, String tableName, IteratorSetting setting, Set<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException, org.apache.thrift.TException;
 
-    public void checkIteratorConflicts(ByteBuffer login, String tableName, IteratorSetting setting, Set<IteratorScope> scopes) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException;
+    public void checkIteratorConflicts(ByteBuffer login, String tableName, IteratorSetting setting, Set<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException, org.apache.thrift.TException;
 
     public void clearLocatorCache(ByteBuffer login, String tableName) throws TableNotFoundException, org.apache.thrift.TException;
 
@@ -64,13 +64,13 @@ import org.slf4j.LoggerFactory;
 
     public void flushTable(ByteBuffer login, String tableName, ByteBuffer startRow, ByteBuffer endRow, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
-    public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException;
+    public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
     public IteratorSetting getIteratorSetting(ByteBuffer login, String tableName, String iteratorName, IteratorScope scope) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
     public ByteBuffer getMaxRow(ByteBuffer login, String tableName, Set<ByteBuffer> auths, ByteBuffer startRow, boolean startInclusive, ByteBuffer endRow, boolean endInclusive) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
-    public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException;
+    public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
     public void importDirectory(ByteBuffer login, String tableName, String importDir, String failureDir, boolean setTime) throws TableNotFoundException, AccumuloException, AccumuloSecurityException, org.apache.thrift.TException;
 
@@ -80,7 +80,7 @@ import org.slf4j.LoggerFactory;
 
     public Map<String,Set<IteratorScope>> listIterators(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
-    public Map<String,Integer> listConstraints(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException;
+    public Map<String,Integer> listConstraints(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
     public void mergeTablets(ByteBuffer login, String tableName, ByteBuffer startRow, ByteBuffer endRow) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException;
 
@@ -523,7 +523,7 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void checkIteratorConflicts(ByteBuffer login, String tableName, IteratorSetting setting, Set<IteratorScope> scopes) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public void checkIteratorConflicts(ByteBuffer login, String tableName, IteratorSetting setting, Set<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException, org.apache.thrift.TException
     {
       send_checkIteratorConflicts(login, tableName, setting, scopes);
       recv_checkIteratorConflicts();
@@ -542,7 +542,7 @@ import org.slf4j.LoggerFactory;
       oprot_.getTransport().flush();
     }
 
-    public void recv_checkIteratorConflicts() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public void recv_checkIteratorConflicts() throws AccumuloSecurityException, AccumuloException, TableNotFoundException, org.apache.thrift.TException
     {
       org.apache.thrift.protocol.TMessage msg = iprot_.readMessageBegin();
       if (msg.type == org.apache.thrift.protocol.TMessageType.EXCEPTION) {
@@ -562,6 +562,9 @@ import org.slf4j.LoggerFactory;
       if (result.ouch2 != null) {
         throw result.ouch2;
       }
+      if (result.ouch3 != null) {
+        throw result.ouch3;
+      }
       return;
     }
 
@@ -879,7 +882,7 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,Set<String>> getLocalityGroups(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       send_getLocalityGroups(login, tableName);
       return recv_getLocalityGroups();
@@ -896,7 +899,7 @@ import org.slf4j.LoggerFactory;
       oprot_.getTransport().flush();
     }
 
-    public Map<String,Set<String>> recv_getLocalityGroups() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,Set<String>> recv_getLocalityGroups() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       org.apache.thrift.protocol.TMessage msg = iprot_.readMessageBegin();
       if (msg.type == org.apache.thrift.protocol.TMessageType.EXCEPTION) {
@@ -919,6 +922,9 @@ import org.slf4j.LoggerFactory;
       if (result.ouch2 != null) {
         throw result.ouch2;
       }
+      if (result.ouch3 != null) {
+        throw result.ouch3;
+      }
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLocalityGroups failed: unknown result");
     }
 
@@ -1021,7 +1027,7 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getMaxRow failed: unknown result");
     }
 
-    public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,String> getTableProperties(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       send_getTableProperties(login, tableName);
       return recv_getTableProperties();
@@ -1038,7 +1044,7 @@ import org.slf4j.LoggerFactory;
       oprot_.getTransport().flush();
     }
 
-    public Map<String,String> recv_getTableProperties() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,String> recv_getTableProperties() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       org.apache.thrift.protocol.TMessage msg = iprot_.readMessageBegin();
       if (msg.type == org.apache.thrift.protocol.TMessageType.EXCEPTION) {
@@ -1061,6 +1067,9 @@ import org.slf4j.LoggerFactory;
       if (result.ouch2 != null) {
         throw result.ouch2;
       }
+      if (result.ouch3 != null) {
+        throw result.ouch3;
+      }
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableProperties failed: unknown result");
     }
 
@@ -1239,7 +1248,7 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listIterators failed: unknown result");
     }
 
-    public Map<String,Integer> listConstraints(ByteBuffer login, String tableName) throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,Integer> listConstraints(ByteBuffer login, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       send_listConstraints(login, tableName);
       return recv_listConstraints();
@@ -1256,7 +1265,7 @@ import org.slf4j.LoggerFactory;
       oprot_.getTransport().flush();
     }
 
-    public Map<String,Integer> recv_listConstraints() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException
+    public Map<String,Integer> recv_listConstraints() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException
     {
       org.apache.thrift.protocol.TMessage msg = iprot_.readMessageBegin();
       if (msg.type == org.apache.thrift.protocol.TMessageType.EXCEPTION) {
@@ -1279,6 +1288,9 @@ import org.slf4j.LoggerFactory;
       if (result.ouch2 != null) {
         throw result.ouch2;
       }
+      if (result.ouch3 != null) {
+        throw result.ouch3;
+      }
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listConstraints failed: unknown result");
     }
 
@@ -3372,7 +3384,7 @@ import org.slf4j.LoggerFactory;
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException {
+      public void getResult() throws AccumuloSecurityException, AccumuloException, TableNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -3700,7 +3712,7 @@ import org.slf4j.LoggerFactory;
         prot.writeMessageEnd();
       }
 
-      public Map<String,Set<String>> getResult() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException {
+      public Map<String,Set<String>> getResult() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -3826,7 +3838,7 @@ import org.slf4j.LoggerFactory;
         prot.writeMessageEnd();
       }
 
-      public Map<String,String> getResult() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException {
+      public Map<String,String> getResult() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -4010,7 +4022,7 @@ import org.slf4j.LoggerFactory;
         prot.writeMessageEnd();
       }
 
-      public Map<String,Integer> getResult() throws AccumuloException, TableNotFoundException, org.apache.thrift.TException {
+      public Map<String,Integer> getResult() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -5938,10 +5950,12 @@ import org.slf4j.LoggerFactory;
         checkIteratorConflicts_result result = new checkIteratorConflicts_result();
         try {
           iface_.checkIteratorConflicts(args.login, args.tableName, args.setting, args.scopes);
-        } catch (AccumuloException ouch1) {
+        } catch (AccumuloSecurityException ouch1) {
           result.ouch1 = ouch1;
-        } catch (TableNotFoundException ouch2) {
+        } catch (AccumuloException ouch2) {
           result.ouch2 = ouch2;
+        } catch (TableNotFoundException ouch3) {
+          result.ouch3 = ouch3;
         } catch (Throwable th) {
           LOGGER.error("Internal error processing checkIteratorConflicts", th);
           org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, "Internal error processing checkIteratorConflicts");
@@ -6272,8 +6286,10 @@ import org.slf4j.LoggerFactory;
           result.success = iface_.getLocalityGroups(args.login, args.tableName);
         } catch (AccumuloException ouch1) {
           result.ouch1 = ouch1;
-        } catch (TableNotFoundException ouch2) {
+        } catch (AccumuloSecurityException ouch2) {
           result.ouch2 = ouch2;
+        } catch (TableNotFoundException ouch3) {
+          result.ouch3 = ouch3;
         } catch (Throwable th) {
           LOGGER.error("Internal error processing getLocalityGroups", th);
           org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, "Internal error processing getLocalityGroups");
@@ -6396,8 +6412,10 @@ import org.slf4j.LoggerFactory;
           result.success = iface_.getTableProperties(args.login, args.tableName);
         } catch (AccumuloException ouch1) {
           result.ouch1 = ouch1;
-        } catch (TableNotFoundException ouch2) {
+        } catch (AccumuloSecurityException ouch2) {
           result.ouch2 = ouch2;
+        } catch (TableNotFoundException ouch3) {
+          result.ouch3 = ouch3;
         } catch (Throwable th) {
           LOGGER.error("Internal error processing getTableProperties", th);
           org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, "Internal error processing getTableProperties");
@@ -6588,8 +6606,10 @@ import org.slf4j.LoggerFactory;
           result.success = iface_.listConstraints(args.login, args.tableName);
         } catch (AccumuloException ouch1) {
           result.ouch1 = ouch1;
-        } catch (TableNotFoundException ouch2) {
+        } catch (AccumuloSecurityException ouch2) {
           result.ouch2 = ouch2;
+        } catch (TableNotFoundException ouch3) {
+          result.ouch3 = ouch3;
         } catch (Throwable th) {
           LOGGER.error("Internal error processing listConstraints", th);
           org.apache.thrift.TApplicationException x = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, "Internal error processing listConstraints");
@@ -12948,14 +12968,17 @@ import org.slf4j.LoggerFactory;
 
     private static final org.apache.thrift.protocol.TField OUCH1_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField OUCH2_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField OUCH3_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch3", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
-    public AccumuloException ouch1;
-    public TableNotFoundException ouch2;
+    public AccumuloSecurityException ouch1;
+    public AccumuloException ouch2;
+    public TableNotFoundException ouch3;
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       OUCH1((short)1, "ouch1"),
-      OUCH2((short)2, "ouch2");
+      OUCH2((short)2, "ouch2"),
+      OUCH3((short)3, "ouch3");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -12974,6 +12997,8 @@ import org.slf4j.LoggerFactory;
             return OUCH1;
           case 2: // OUCH2
             return OUCH2;
+          case 3: // OUCH3
+            return OUCH3;
           default:
             return null;
         }
@@ -13022,6 +13047,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.OUCH2, new org.apache.thrift.meta_data.FieldMetaData("ouch2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.OUCH3, new org.apache.thrift.meta_data.FieldMetaData("ouch3", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkIteratorConflicts_result.class, metaDataMap);
     }
@@ -13030,12 +13057,14 @@ import org.slf4j.LoggerFactory;
     }
 
     public checkIteratorConflicts_result(
-      AccumuloException ouch1,
-      TableNotFoundException ouch2)
+      AccumuloSecurityException ouch1,
+      AccumuloException ouch2,
+      TableNotFoundException ouch3)
     {
       this();
       this.ouch1 = ouch1;
       this.ouch2 = ouch2;
+      this.ouch3 = ouch3;
     }
 
     /**
@@ -13043,10 +13072,13 @@ import org.slf4j.LoggerFactory;
      */
     public checkIteratorConflicts_result(checkIteratorConflicts_result other) {
       if (other.isSetOuch1()) {
-        this.ouch1 = new AccumuloException(other.ouch1);
+        this.ouch1 = new AccumuloSecurityException(other.ouch1);
       }
       if (other.isSetOuch2()) {
-        this.ouch2 = new TableNotFoundException(other.ouch2);
+        this.ouch2 = new AccumuloException(other.ouch2);
+      }
+      if (other.isSetOuch3()) {
+        this.ouch3 = new TableNotFoundException(other.ouch3);
       }
     }
 
@@ -13058,13 +13090,14 @@ import org.slf4j.LoggerFactory;
     public void clear() {
       this.ouch1 = null;
       this.ouch2 = null;
+      this.ouch3 = null;
     }
 
-    public AccumuloException getOuch1() {
+    public AccumuloSecurityException getOuch1() {
       return this.ouch1;
     }
 
-    public checkIteratorConflicts_result setOuch1(AccumuloException ouch1) {
+    public checkIteratorConflicts_result setOuch1(AccumuloSecurityException ouch1) {
       this.ouch1 = ouch1;
       return this;
     }
@@ -13084,11 +13117,11 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public TableNotFoundException getOuch2() {
+    public AccumuloException getOuch2() {
       return this.ouch2;
     }
 
-    public checkIteratorConflicts_result setOuch2(TableNotFoundException ouch2) {
+    public checkIteratorConflicts_result setOuch2(AccumuloException ouch2) {
       this.ouch2 = ouch2;
       return this;
     }
@@ -13108,13 +13141,37 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public TableNotFoundException getOuch3() {
+      return this.ouch3;
+    }
+
+    public checkIteratorConflicts_result setOuch3(TableNotFoundException ouch3) {
+      this.ouch3 = ouch3;
+      return this;
+    }
+
+    public void unsetOuch3() {
+      this.ouch3 = null;
+    }
+
+    /** Returns true if field ouch3 is set (has been assigned a value) and false otherwise */
+    public boolean isSetOuch3() {
+      return this.ouch3 != null;
+    }
+
+    public void setOuch3IsSet(boolean value) {
+      if (!value) {
+        this.ouch3 = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case OUCH1:
         if (value == null) {
           unsetOuch1();
         } else {
-          setOuch1((AccumuloException)value);
+          setOuch1((AccumuloSecurityException)value);
         }
         break;
 
@@ -13122,7 +13179,15 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetOuch2();
         } else {
-          setOuch2((TableNotFoundException)value);
+          setOuch2((AccumuloException)value);
+        }
+        break;
+
+      case OUCH3:
+        if (value == null) {
+          unsetOuch3();
+        } else {
+          setOuch3((TableNotFoundException)value);
         }
         break;
 
@@ -13137,6 +13202,9 @@ import org.slf4j.LoggerFactory;
       case OUCH2:
         return getOuch2();
 
+      case OUCH3:
+        return getOuch3();
+
       }
       throw new IllegalStateException();
     }
@@ -13152,6 +13220,8 @@ import org.slf4j.LoggerFactory;
         return isSetOuch1();
       case OUCH2:
         return isSetOuch2();
+      case OUCH3:
+        return isSetOuch3();
       }
       throw new IllegalStateException();
     }
@@ -13187,6 +13257,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_ouch3 = true && this.isSetOuch3();
+      boolean that_present_ouch3 = true && that.isSetOuch3();
+      if (this_present_ouch3 || that_present_ouch3) {
+        if (!(this_present_ouch3 && that_present_ouch3))
+          return false;
+        if (!this.ouch3.equals(that.ouch3))
+          return false;
+      }
+
       return true;
     }
 
@@ -13223,6 +13302,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetOuch3()).compareTo(typedOther.isSetOuch3());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOuch3()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch3, typedOther.ouch3);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -13242,7 +13331,7 @@ import org.slf4j.LoggerFactory;
         switch (field.id) {
           case 1: // OUCH1
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.ouch1 = new AccumuloException();
+              this.ouch1 = new AccumuloSecurityException();
               this.ouch1.read(iprot);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
@@ -13250,12 +13339,20 @@ import org.slf4j.LoggerFactory;
             break;
           case 2: // OUCH2
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.ouch2 = new TableNotFoundException();
+              this.ouch2 = new AccumuloException();
               this.ouch2.read(iprot);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // OUCH3
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.ouch3 = new TableNotFoundException();
+              this.ouch3.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -13278,6 +13375,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OUCH2_FIELD_DESC);
         this.ouch2.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.isSetOuch3()) {
+        oprot.writeFieldBegin(OUCH3_FIELD_DESC);
+        this.ouch3.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -13303,6 +13404,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.ouch2);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("ouch3:");
+      if (this.ouch3 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ouch3);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -21368,16 +21477,19 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
     private static final org.apache.thrift.protocol.TField OUCH1_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField OUCH2_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField OUCH3_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch3", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     public Map<String,Set<String>> success;
     public AccumuloException ouch1;
-    public TableNotFoundException ouch2;
+    public AccumuloSecurityException ouch2;
+    public TableNotFoundException ouch3;
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       OUCH1((short)1, "ouch1"),
-      OUCH2((short)2, "ouch2");
+      OUCH2((short)2, "ouch2"),
+      OUCH3((short)3, "ouch3");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -21398,6 +21510,8 @@ import org.slf4j.LoggerFactory;
             return OUCH1;
           case 2: // OUCH2
             return OUCH2;
+          case 3: // OUCH3
+            return OUCH3;
           default:
             return null;
         }
@@ -21451,6 +21565,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.OUCH2, new org.apache.thrift.meta_data.FieldMetaData("ouch2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.OUCH3, new org.apache.thrift.meta_data.FieldMetaData("ouch3", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLocalityGroups_result.class, metaDataMap);
     }
@@ -21461,12 +21577,14 @@ import org.slf4j.LoggerFactory;
     public getLocalityGroups_result(
       Map<String,Set<String>> success,
       AccumuloException ouch1,
-      TableNotFoundException ouch2)
+      AccumuloSecurityException ouch2,
+      TableNotFoundException ouch3)
     {
       this();
       this.success = success;
       this.ouch1 = ouch1;
       this.ouch2 = ouch2;
+      this.ouch3 = ouch3;
     }
 
     /**
@@ -21495,7 +21613,10 @@ import org.slf4j.LoggerFactory;
         this.ouch1 = new AccumuloException(other.ouch1);
       }
       if (other.isSetOuch2()) {
-        this.ouch2 = new TableNotFoundException(other.ouch2);
+        this.ouch2 = new AccumuloSecurityException(other.ouch2);
+      }
+      if (other.isSetOuch3()) {
+        this.ouch3 = new TableNotFoundException(other.ouch3);
       }
     }
 
@@ -21508,6 +21629,7 @@ import org.slf4j.LoggerFactory;
       this.success = null;
       this.ouch1 = null;
       this.ouch2 = null;
+      this.ouch3 = null;
     }
 
     public int getSuccessSize() {
@@ -21569,11 +21691,11 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public TableNotFoundException getOuch2() {
+    public AccumuloSecurityException getOuch2() {
       return this.ouch2;
     }
 
-    public getLocalityGroups_result setOuch2(TableNotFoundException ouch2) {
+    public getLocalityGroups_result setOuch2(AccumuloSecurityException ouch2) {
       this.ouch2 = ouch2;
       return this;
     }
@@ -21593,6 +21715,30 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public TableNotFoundException getOuch3() {
+      return this.ouch3;
+    }
+
+    public getLocalityGroups_result setOuch3(TableNotFoundException ouch3) {
+      this.ouch3 = ouch3;
+      return this;
+    }
+
+    public void unsetOuch3() {
+      this.ouch3 = null;
+    }
+
+    /** Returns true if field ouch3 is set (has been assigned a value) and false otherwise */
+    public boolean isSetOuch3() {
+      return this.ouch3 != null;
+    }
+
+    public void setOuch3IsSet(boolean value) {
+      if (!value) {
+        this.ouch3 = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case SUCCESS:
@@ -21615,7 +21761,15 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetOuch2();
         } else {
-          setOuch2((TableNotFoundException)value);
+          setOuch2((AccumuloSecurityException)value);
+        }
+        break;
+
+      case OUCH3:
+        if (value == null) {
+          unsetOuch3();
+        } else {
+          setOuch3((TableNotFoundException)value);
         }
         break;
 
@@ -21633,6 +21787,9 @@ import org.slf4j.LoggerFactory;
       case OUCH2:
         return getOuch2();
 
+      case OUCH3:
+        return getOuch3();
+
       }
       throw new IllegalStateException();
     }
@@ -21650,6 +21807,8 @@ import org.slf4j.LoggerFactory;
         return isSetOuch1();
       case OUCH2:
         return isSetOuch2();
+      case OUCH3:
+        return isSetOuch3();
       }
       throw new IllegalStateException();
     }
@@ -21694,6 +21853,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_ouch3 = true && this.isSetOuch3();
+      boolean that_present_ouch3 = true && that.isSetOuch3();
+      if (this_present_ouch3 || that_present_ouch3) {
+        if (!(this_present_ouch3 && that_present_ouch3))
+          return false;
+        if (!this.ouch3.equals(that.ouch3))
+          return false;
+      }
+
       return true;
     }
 
@@ -21740,6 +21908,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetOuch3()).compareTo(typedOther.isSetOuch3());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOuch3()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch3, typedOther.ouch3);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -21796,12 +21974,20 @@ import org.slf4j.LoggerFactory;
             break;
           case 2: // OUCH2
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.ouch2 = new TableNotFoundException();
+              this.ouch2 = new AccumuloSecurityException();
               this.ouch2.read(iprot);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // OUCH3
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.ouch3 = new TableNotFoundException();
+              this.ouch3.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -21843,6 +22029,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OUCH2_FIELD_DESC);
         this.ouch2.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.isSetOuch3()) {
+        oprot.writeFieldBegin(OUCH3_FIELD_DESC);
+        this.ouch3.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -21876,6 +22066,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.ouch2);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("ouch3:");
+      if (this.ouch3 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ouch3);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -24932,16 +25130,19 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
     private static final org.apache.thrift.protocol.TField OUCH1_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField OUCH2_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField OUCH3_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch3", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     public Map<String,String> success;
     public AccumuloException ouch1;
-    public TableNotFoundException ouch2;
+    public AccumuloSecurityException ouch2;
+    public TableNotFoundException ouch3;
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       OUCH1((short)1, "ouch1"),
-      OUCH2((short)2, "ouch2");
+      OUCH2((short)2, "ouch2"),
+      OUCH3((short)3, "ouch3");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -24962,6 +25163,8 @@ import org.slf4j.LoggerFactory;
             return OUCH1;
           case 2: // OUCH2
             return OUCH2;
+          case 3: // OUCH3
+            return OUCH3;
           default:
             return null;
         }
@@ -25014,6 +25217,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.OUCH2, new org.apache.thrift.meta_data.FieldMetaData("ouch2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.OUCH3, new org.apache.thrift.meta_data.FieldMetaData("ouch3", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTableProperties_result.class, metaDataMap);
     }
@@ -25024,12 +25229,14 @@ import org.slf4j.LoggerFactory;
     public getTableProperties_result(
       Map<String,String> success,
       AccumuloException ouch1,
-      TableNotFoundException ouch2)
+      AccumuloSecurityException ouch2,
+      TableNotFoundException ouch3)
     {
       this();
       this.success = success;
       this.ouch1 = ouch1;
       this.ouch2 = ouch2;
+      this.ouch3 = ouch3;
     }
 
     /**
@@ -25055,7 +25262,10 @@ import org.slf4j.LoggerFactory;
         this.ouch1 = new AccumuloException(other.ouch1);
       }
       if (other.isSetOuch2()) {
-        this.ouch2 = new TableNotFoundException(other.ouch2);
+        this.ouch2 = new AccumuloSecurityException(other.ouch2);
+      }
+      if (other.isSetOuch3()) {
+        this.ouch3 = new TableNotFoundException(other.ouch3);
       }
     }
 
@@ -25068,6 +25278,7 @@ import org.slf4j.LoggerFactory;
       this.success = null;
       this.ouch1 = null;
       this.ouch2 = null;
+      this.ouch3 = null;
     }
 
     public int getSuccessSize() {
@@ -25129,11 +25340,11 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public TableNotFoundException getOuch2() {
+    public AccumuloSecurityException getOuch2() {
       return this.ouch2;
     }
 
-    public getTableProperties_result setOuch2(TableNotFoundException ouch2) {
+    public getTableProperties_result setOuch2(AccumuloSecurityException ouch2) {
       this.ouch2 = ouch2;
       return this;
     }
@@ -25153,6 +25364,30 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public TableNotFoundException getOuch3() {
+      return this.ouch3;
+    }
+
+    public getTableProperties_result setOuch3(TableNotFoundException ouch3) {
+      this.ouch3 = ouch3;
+      return this;
+    }
+
+    public void unsetOuch3() {
+      this.ouch3 = null;
+    }
+
+    /** Returns true if field ouch3 is set (has been assigned a value) and false otherwise */
+    public boolean isSetOuch3() {
+      return this.ouch3 != null;
+    }
+
+    public void setOuch3IsSet(boolean value) {
+      if (!value) {
+        this.ouch3 = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case SUCCESS:
@@ -25175,7 +25410,15 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetOuch2();
         } else {
-          setOuch2((TableNotFoundException)value);
+          setOuch2((AccumuloSecurityException)value);
+        }
+        break;
+
+      case OUCH3:
+        if (value == null) {
+          unsetOuch3();
+        } else {
+          setOuch3((TableNotFoundException)value);
         }
         break;
 
@@ -25193,6 +25436,9 @@ import org.slf4j.LoggerFactory;
       case OUCH2:
         return getOuch2();
 
+      case OUCH3:
+        return getOuch3();
+
       }
       throw new IllegalStateException();
     }
@@ -25210,6 +25456,8 @@ import org.slf4j.LoggerFactory;
         return isSetOuch1();
       case OUCH2:
         return isSetOuch2();
+      case OUCH3:
+        return isSetOuch3();
       }
       throw new IllegalStateException();
     }
@@ -25254,6 +25502,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_ouch3 = true && this.isSetOuch3();
+      boolean that_present_ouch3 = true && that.isSetOuch3();
+      if (this_present_ouch3 || that_present_ouch3) {
+        if (!(this_present_ouch3 && that_present_ouch3))
+          return false;
+        if (!this.ouch3.equals(that.ouch3))
+          return false;
+      }
+
       return true;
     }
 
@@ -25300,6 +25557,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetOuch3()).compareTo(typedOther.isSetOuch3());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOuch3()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch3, typedOther.ouch3);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -25346,12 +25613,20 @@ import org.slf4j.LoggerFactory;
             break;
           case 2: // OUCH2
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.ouch2 = new TableNotFoundException();
+              this.ouch2 = new AccumuloSecurityException();
               this.ouch2.read(iprot);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // OUCH3
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.ouch3 = new TableNotFoundException();
+              this.ouch3.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -25386,6 +25661,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OUCH2_FIELD_DESC);
         this.ouch2.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.isSetOuch3()) {
+        oprot.writeFieldBegin(OUCH3_FIELD_DESC);
+        this.ouch3.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -25419,6 +25698,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.ouch2);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("ouch3:");
+      if (this.ouch3 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ouch3);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -29742,16 +30029,19 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0);
     private static final org.apache.thrift.protocol.TField OUCH1_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField OUCH2_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField OUCH3_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch3", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
     public Map<String,Integer> success;
     public AccumuloException ouch1;
-    public TableNotFoundException ouch2;
+    public AccumuloSecurityException ouch2;
+    public TableNotFoundException ouch3;
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       OUCH1((short)1, "ouch1"),
-      OUCH2((short)2, "ouch2");
+      OUCH2((short)2, "ouch2"),
+      OUCH3((short)3, "ouch3");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -29772,6 +30062,8 @@ import org.slf4j.LoggerFactory;
             return OUCH1;
           case 2: // OUCH2
             return OUCH2;
+          case 3: // OUCH3
+            return OUCH3;
           default:
             return null;
         }
@@ -29824,6 +30116,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.OUCH2, new org.apache.thrift.meta_data.FieldMetaData("ouch2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.OUCH3, new org.apache.thrift.meta_data.FieldMetaData("ouch3", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(listConstraints_result.class, metaDataMap);
     }
@@ -29834,12 +30128,14 @@ import org.slf4j.LoggerFactory;
     public listConstraints_result(
       Map<String,Integer> success,
       AccumuloException ouch1,
-      TableNotFoundException ouch2)
+      AccumuloSecurityException ouch2,
+      TableNotFoundException ouch3)
     {
       this();
       this.success = success;
       this.ouch1 = ouch1;
       this.ouch2 = ouch2;
+      this.ouch3 = ouch3;
     }
 
     /**
@@ -29865,7 +30161,10 @@ import org.slf4j.LoggerFactory;
         this.ouch1 = new AccumuloException(other.ouch1);
       }
       if (other.isSetOuch2()) {
-        this.ouch2 = new TableNotFoundException(other.ouch2);
+        this.ouch2 = new AccumuloSecurityException(other.ouch2);
+      }
+      if (other.isSetOuch3()) {
+        this.ouch3 = new TableNotFoundException(other.ouch3);
       }
     }
 
@@ -29878,6 +30177,7 @@ import org.slf4j.LoggerFactory;
       this.success = null;
       this.ouch1 = null;
       this.ouch2 = null;
+      this.ouch3 = null;
     }
 
     public int getSuccessSize() {
@@ -29939,11 +30239,11 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public TableNotFoundException getOuch2() {
+    public AccumuloSecurityException getOuch2() {
       return this.ouch2;
     }
 
-    public listConstraints_result setOuch2(TableNotFoundException ouch2) {
+    public listConstraints_result setOuch2(AccumuloSecurityException ouch2) {
       this.ouch2 = ouch2;
       return this;
     }
@@ -29963,6 +30263,30 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public TableNotFoundException getOuch3() {
+      return this.ouch3;
+    }
+
+    public listConstraints_result setOuch3(TableNotFoundException ouch3) {
+      this.ouch3 = ouch3;
+      return this;
+    }
+
+    public void unsetOuch3() {
+      this.ouch3 = null;
+    }
+
+    /** Returns true if field ouch3 is set (has been assigned a value) and false otherwise */
+    public boolean isSetOuch3() {
+      return this.ouch3 != null;
+    }
+
+    public void setOuch3IsSet(boolean value) {
+      if (!value) {
+        this.ouch3 = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case SUCCESS:
@@ -29985,7 +30309,15 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetOuch2();
         } else {
-          setOuch2((TableNotFoundException)value);
+          setOuch2((AccumuloSecurityException)value);
+        }
+        break;
+
+      case OUCH3:
+        if (value == null) {
+          unsetOuch3();
+        } else {
+          setOuch3((TableNotFoundException)value);
         }
         break;
 
@@ -30003,6 +30335,9 @@ import org.slf4j.LoggerFactory;
       case OUCH2:
         return getOuch2();
 
+      case OUCH3:
+        return getOuch3();
+
       }
       throw new IllegalStateException();
     }
@@ -30020,6 +30355,8 @@ import org.slf4j.LoggerFactory;
         return isSetOuch1();
       case OUCH2:
         return isSetOuch2();
+      case OUCH3:
+        return isSetOuch3();
       }
       throw new IllegalStateException();
     }
@@ -30064,6 +30401,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_ouch3 = true && this.isSetOuch3();
+      boolean that_present_ouch3 = true && that.isSetOuch3();
+      if (this_present_ouch3 || that_present_ouch3) {
+        if (!(this_present_ouch3 && that_present_ouch3))
+          return false;
+        if (!this.ouch3.equals(that.ouch3))
+          return false;
+      }
+
       return true;
     }
 
@@ -30110,6 +30456,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetOuch3()).compareTo(typedOther.isSetOuch3());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetOuch3()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch3, typedOther.ouch3);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -30156,12 +30512,20 @@ import org.slf4j.LoggerFactory;
             break;
           case 2: // OUCH2
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
-              this.ouch2 = new TableNotFoundException();
+              this.ouch2 = new AccumuloSecurityException();
               this.ouch2.read(iprot);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // OUCH3
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.ouch3 = new TableNotFoundException();
+              this.ouch3.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -30196,6 +30560,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OUCH2_FIELD_DESC);
         this.ouch2.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.isSetOuch3()) {
+        oprot.writeFieldBegin(OUCH3_FIELD_DESC);
+        this.ouch3.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -30229,6 +30597,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.ouch2);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("ouch3:");
+      if (this.ouch3 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ouch3);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }

Modified: accumulo/branches/1.4/src/proxy/src/main/thrift/proxy.thrift
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/proxy/src/main/thrift/proxy.thrift?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/proxy/src/main/thrift/proxy.thrift (original)
+++ accumulo/branches/1.4/src/proxy/src/main/thrift/proxy.thrift Tue Apr 23 17:23:01 2013
@@ -214,7 +214,7 @@ service AccumuloProxy
                                                                                                        throws (1:AccumuloSecurityException ouch1, 2:AccumuloException ouch2, 3:TableNotFoundException ouch3);
   void checkIteratorConflicts (1:binary login, 2:string tableName, 3:IteratorSetting setting, 
                                4:set<IteratorScope> scopes) 
-                                                                                                       throws (1:AccumuloException ouch1, 2:TableNotFoundException ouch2);
+                                                                                                       throws (1:AccumuloSecurityException ouch1, 2:AccumuloException ouch2, 3:TableNotFoundException ouch3);
   void clearLocatorCache (1:binary login, 2:string tableName)                                          throws (1:TableNotFoundException ouch1);
   void cloneTable (1:binary login, 2:string tableName, 3:string newTableName, 4:bool flush, 
                    5:map<string,string> propertiesToSet, 6:set<string> propertiesToExclude) 
@@ -228,21 +228,21 @@ service AccumuloProxy
   void flushTable (1:binary login, 2:string tableName, 3:binary startRow, 4:binary endRow,
                    5:bool wait)
                                                                                                        throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
-  map<string,set<string>> getLocalityGroups (1:binary login, 2:string tableName)                       throws (1:AccumuloException ouch1, 2:TableNotFoundException ouch2);
+  map<string,set<string>> getLocalityGroups (1:binary login, 2:string tableName)                       throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   IteratorSetting getIteratorSetting (1:binary login, 2:string tableName, 
                                       3:string iteratorName, 4:IteratorScope scope) 
                                                                                                        throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   binary getMaxRow (1:binary login, 2:string tableName, 3:set<binary> auths, 4:binary startRow, 
                     5:bool startInclusive, 6:binary endRow, 7:bool endInclusive) 
                                                                                                        throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
-  map<string,string> getTableProperties (1:binary login, 2:string tableName)                           throws (1:AccumuloException ouch1, 2:TableNotFoundException ouch2);
+  map<string,string> getTableProperties (1:binary login, 2:string tableName)                           throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   void importDirectory (1:binary login, 2:string tableName, 3:string importDir, 
                         4:string failureDir, 5:bool setTime) 
                                                                                                        throws (1:TableNotFoundException ouch1, 2:AccumuloException ouch3, 3:AccumuloSecurityException ouch4);
   list<binary> listSplits (1:binary login, 2:string tableName, 3:i32 maxSplits)                        throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   set<string> listTables (1:binary login);
   map<string,set<IteratorScope>> listIterators (1:binary login, 2:string tableName)                    throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
-  map<string,i32> listConstraints (1:binary login, 2:string tableName)                                 throws (1:AccumuloException ouch1, 2:TableNotFoundException ouch2);
+  map<string,i32> listConstraints (1:binary login, 2:string tableName)                                 throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   void mergeTablets (1:binary login, 2:string tableName, 3:binary startRow, 4:binary endRow)           throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   void offlineTable (1:binary login, 2:string tableName)                                               throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);
   void onlineTable (1:binary login, 2:string tableName)                                                throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3);

Modified: accumulo/branches/1.4/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java (original)
+++ accumulo/branches/1.4/src/proxy/src/test/java/org/apache/accumulo/proxy/SimpleTest.java Tue Apr 23 17:23:01 2013
@@ -113,7 +113,7 @@ public class SimpleTest {
   static Class<? extends TProtocolFactory> getRandomProtocol() {
     List<Class<? extends TProtocolFactory>> protocolFactories = new ArrayList<Class<? extends TProtocolFactory>>();
     protocolFactories.add(org.apache.thrift.protocol.TJSONProtocol.Factory.class);
-    // protocolFactories.add(org.apache.thrift.protocol.TBinaryProtocol.Factory.class); // This consistently fails for some reason
+    protocolFactories.add(org.apache.thrift.protocol.TBinaryProtocol.Factory.class);
     protocolFactories.add(org.apache.thrift.protocol.TCompactProtocol.Factory.class);
     
     Random rand = new Random();

Modified: accumulo/branches/1.4/src/proxy/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/proxy/src/test/resources/log4j.properties?rev=1471056&r1=1471055&r2=1471056&view=diff
==============================================================================
--- accumulo/branches/1.4/src/proxy/src/test/resources/log4j.properties (original)
+++ accumulo/branches/1.4/src/proxy/src/test/resources/log4j.properties Tue Apr 23 17:23:01 2013
@@ -16,7 +16,7 @@
 log4j.rootLogger=INFO, CA
 log4j.appender.CA=org.apache.log4j.ConsoleAppender
 log4j.appender.CA.layout=org.apache.log4j.PatternLayout
-log4j.appender.CA.layout.ConversionPattern=[%t} %-5p %c %x - %m%n
+log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 
 log4j.logger.org.apache.accumulo.core.zookeeper.ZooCache=ERROR
 log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR