You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2014/09/05 23:17:17 UTC

[01/18] ACCUMULO-1957 implement per-session Durability

Repository: accumulo
Updated Branches:
  refs/heads/master 17f625034 -> 6e02b3c7b


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
index edd12aa..f9c443a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.tserver;
 import java.util.List;
 
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.tserver.tablet.Durability;
+import org.apache.accumulo.core.client.Durability;
 
 public class TabletMutations {
   private final int tid; 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 63bf4a3..57de347 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -56,6 +56,7 @@ import javax.management.StandardMBean;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.CompressedIterators;
 import org.apache.accumulo.core.client.impl.CompressedIterators.IterConfig;
@@ -115,6 +116,7 @@ import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -215,7 +217,6 @@ import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.accumulo.tserver.tablet.CompactionInfo;
 import org.apache.accumulo.tserver.tablet.CompactionWatcher;
 import org.apache.accumulo.tserver.tablet.Compactor;
-import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.accumulo.tserver.tablet.KVEntry;
 import org.apache.accumulo.tserver.tablet.ScanBatch;
 import org.apache.accumulo.tserver.tablet.Scanner;
@@ -652,14 +653,14 @@ public class TabletServer implements Runnable {
     }
 
     @Override
-    public long startUpdate(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
+    public long startUpdate(TInfo tinfo, TCredentials credentials, TDurability tdurabilty) throws ThriftSecurityException {
       // Make sure user is real
-
+      Durability durability = Durability.fromThrift(tdurabilty);
       security.authenticateUser(credentials, credentials);
       if (updateMetrics.isEnabled())
         updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
 
-      UpdateSession us = new UpdateSession(new TservConstraintEnv(security, credentials), credentials);
+      UpdateSession us = new UpdateSession(new TservConstraintEnv(security, credentials), credentials, durability);
       long sid = sessionManager.createSession(us, false);
       return sid;
     }
@@ -738,11 +739,11 @@ public class TabletServer implements Runnable {
         sessionManager.unreserveSession(us);
       }
     }
-
+    
     private void flush(UpdateSession us) {
 
       int mutationCount = 0;
-      Map<CommitSession,List<Mutation>> sendables = new HashMap<CommitSession,List<Mutation>>();
+      Map<CommitSession,Mutations> sendables = new HashMap<CommitSession,Mutations>();
       Throwable error = null;
 
       long pt1 = System.currentTimeMillis();
@@ -760,6 +761,7 @@ public class TabletServer implements Runnable {
         for (Entry<Tablet,? extends List<Mutation>> entry : us.queuedMutations.entrySet()) {
 
           Tablet tablet = entry.getKey();
+          Durability tabletDurability = tablet.getDurability();
           List<Mutation> mutations = entry.getValue();
           if (mutations.size() > 0) {
             try {
@@ -773,7 +775,8 @@ public class TabletServer implements Runnable {
                 }
                 us.failures.put(tablet.getExtent(), us.successfulCommits.get(tablet));
               } else {
-                sendables.put(commitSession, mutations);
+                log.debug("Durablity for " + tablet.getExtent() + " durability " + us.durability + " table durability " + tabletDurability + " using " + us.durability.resolveDurability(tabletDurability));
+                sendables.put(commitSession, new Mutations(us.durability.resolveDurability(tabletDurability), mutations));
                 mutationCount += mutations.size();
               }
 
@@ -788,7 +791,7 @@ public class TabletServer implements Runnable {
                 // violate constraints... this is what
                 // prepareMutationsForCommit()
                 // expects
-                sendables.put(e.getCommitSession(), e.getNonViolators());
+                sendables.put(e.getCommitSession(), new Mutations(us.durability.resolveDurability(tabletDurability), e.getNonViolators()));
               }
 
               mutationCount += mutations.size();
@@ -813,8 +816,8 @@ public class TabletServer implements Runnable {
       updateAvgPrepTime(pt2 - pt1, us.queuedMutations.size());
 
       if (error != null) {
-        for (Entry<CommitSession,List<Mutation>> e : sendables.entrySet()) {
-          e.getKey().abortCommit(e.getValue());
+        for (Entry<CommitSession,Mutations> e : sendables.entrySet()) {
+          e.getKey().abortCommit(e.getValue().getMutations());
         }
         throw new RuntimeException(error);
       }
@@ -847,9 +850,9 @@ public class TabletServer implements Runnable {
         Span commit = Trace.start("commit");
         try {
           long t1 = System.currentTimeMillis();
-          for (Entry<CommitSession,? extends List<Mutation>> entry : sendables.entrySet()) {
+          for (Entry<CommitSession,Mutations> entry : sendables.entrySet()) {
             CommitSession commitSession = entry.getKey();
-            List<Mutation> mutations = entry.getValue();
+            List<Mutation> mutations = entry.getValue().getMutations();
 
             commitSession.commit(mutations);
 
@@ -937,7 +940,7 @@ public class TabletServer implements Runnable {
     }
 
     @Override
-    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
+    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, TMutation tmutation, TDurability tdurability) throws NotServingTabletException,
         ConstraintViolationException, ThriftSecurityException {
 
       final String tableId = new String(tkeyExtent.getTable(), StandardCharsets.UTF_8);
@@ -948,6 +951,7 @@ public class TabletServer implements Runnable {
       if (tablet == null) {
         throw new NotServingTabletException(tkeyExtent);
       }
+      Durability tabletDurability = tablet.getDurability();
 
       if (!keyExtent.isMeta())
         TabletServer.this.resourceManager.waitUntilCommitsAreEnabled();
@@ -973,7 +977,7 @@ public class TabletServer implements Runnable {
           try {
             final Span wal = Trace.start("wal");
             try {
-              logger.log(cs, cs.getWALogSeq(), mutation);
+              logger.log(cs, cs.getWALogSeq(), mutation, Durability.fromThrift(tdurability).resolveDurability(tabletDurability));
             } finally {
               wal.stop();
             }
@@ -1076,7 +1080,7 @@ public class TabletServer implements Runnable {
     private void writeConditionalMutations(Map<KeyExtent,List<ServerConditionalMutation>> updates, ArrayList<TCMResult> results, ConditionalSession sess) {
       Set<Entry<KeyExtent,List<ServerConditionalMutation>>> es = updates.entrySet();
 
-      Map<CommitSession,List<Mutation>> sendables = new HashMap<CommitSession,List<Mutation>>();
+      Map<CommitSession,Mutations> sendables = new HashMap<CommitSession,Mutations>();
 
       boolean sessionCanceled = sess.interruptFlag.get();
 
@@ -1085,6 +1089,7 @@ public class TabletServer implements Runnable {
         long t1 = System.currentTimeMillis();
         for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : es) {
           Tablet tablet = onlineTablets.get(entry.getKey());
+          Durability tabletDurability = tablet.getDurability();
           if (tablet == null || tablet.isClosed() || sessionCanceled) {
             for (ServerConditionalMutation scm : entry.getValue())
               results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
@@ -1096,19 +1101,19 @@ public class TabletServer implements Runnable {
               if (mutations.size() > 0) {
 
                 CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, sess.credentials), mutations);
-
+                
                 if (cs == null) {
                   for (ServerConditionalMutation scm : entry.getValue())
                     results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
                 } else {
                   for (ServerConditionalMutation scm : entry.getValue())
                     results.add(new TCMResult(scm.getID(), TCMStatus.ACCEPTED));
-                  sendables.put(cs, mutations);
+                  sendables.put(cs, new Mutations(sess.durability.resolveDurability(tabletDurability), mutations));
                 }
               }
             } catch (TConstraintViolationException e) {
               if (e.getNonViolators().size() > 0) {
-                sendables.put(e.getCommitSession(), e.getNonViolators());
+                sendables.put(e.getCommitSession(), new Mutations(sess.durability.resolveDurability(tabletDurability), e.getNonViolators()));
                 for (Mutation m : e.getNonViolators())
                   results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.ACCEPTED));
               }
@@ -1150,9 +1155,9 @@ public class TabletServer implements Runnable {
       Span commitSpan = Trace.start("commit");
       try {
         long t1 = System.currentTimeMillis();
-        for (Entry<CommitSession,? extends List<Mutation>> entry : sendables.entrySet()) {
+        for (Entry<CommitSession,Mutations> entry : sendables.entrySet()) {
           CommitSession commitSession = entry.getKey();
-          List<Mutation> mutations = entry.getValue();
+          List<Mutation> mutations = entry.getValue().getMutations();
 
           commitSession.commit(mutations);
         }
@@ -1197,7 +1202,7 @@ public class TabletServer implements Runnable {
     }
 
     @Override
-    public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableId)
+    public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableId, TDurability tdurabilty)
         throws ThriftSecurityException, TException {
 
       Authorizations userauths = null;
@@ -1209,7 +1214,7 @@ public class TabletServer implements Runnable {
         if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
 
-      ConditionalSession cs = new ConditionalSession(credentials, new Authorizations(authorizations), tableId);
+      ConditionalSession cs = new ConditionalSession(credentials, new Authorizations(authorizations), tableId, Durability.fromThrift(tdurabilty));
 
       long sid = sessionManager.createSession(cs, false);
       return new TConditionalSession(sid, lockID, sessionManager.getMaxIdleTime());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 443ba2e..f6829ea 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -26,7 +26,6 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.channels.ClosedChannelException;
 import java.nio.charset.StandardCharsets;
@@ -41,6 +40,7 @@ import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -59,7 +59,6 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.logger.LogFileKey;
 import org.apache.accumulo.tserver.logger.LogFileValue;
-import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -147,30 +146,39 @@ public class DfsLogger {
         }
         workQueue.drainTo(work);
 
+        String durability = null;
         Method durabilityMethod = null;
         loop:
         for (LogWork logWork : work) {
           switch (logWork.durability) {
+            case DEFAULT:
             case NONE:
               // shouldn't make it to the work queue
+              log.warn("unexpected durability " + logWork.durability, new Throwable());
               break;
             case LOG:
               // do nothing
               break;
             case SYNC:
               durabilityMethod = sync;
+              durability = logWork.durability.toString();
               break loop;
             case FLUSH:
               if (durabilityMethod == null) {
                 durabilityMethod = flush;
+                durability = logWork.durability.toString();
               }
               break;
           }
         }
 
         try {
-          if (durabilityMethod != null)
+          if (durabilityMethod != null) {
+            log.debug("durability method " + durability);
             durabilityMethod.invoke(logFile);
+          } else {
+            log.debug("skipping flush/sync");
+          }
         } catch (Exception ex) {
           log.warn("Exception syncing " + ex);
           for (DfsLogger.LogWork logWork : work) {
@@ -493,25 +501,9 @@ public class DfsLogger {
     key.tablet = tablet;
     try {
       write(key, EMPTY);
-      sync.invoke(logFile);
     } catch (IllegalArgumentException e) {
       log.error("Signature of sync method changed. Accumulo is likely incompatible with this version of Hadoop.");
       throw new RuntimeException(e);
-    } catch (IllegalAccessException e) {
-      log.error("Could not invoke sync method due to permission error.");
-      throw new RuntimeException(e);
-    } catch (InvocationTargetException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof IOException) {
-        throw (IOException) cause;
-      } else if (cause instanceof RuntimeException) {
-        throw (RuntimeException) cause;
-      } else if (cause instanceof Error) {
-        throw (Error) cause;
-      } else {
-        // Cause is null, or some other checked exception that was added later.
-        throw new RuntimeException(e);
-      }
     }
   }
 
@@ -563,6 +555,7 @@ public class DfsLogger {
       LogFileValue value = new LogFileValue();
       value.mutations = tabletMutations.getMutations();
       data.add(new Pair<LogFileKey,LogFileValue>(key, value));
+      log.debug("Durability for " + tabletMutations.getDurability() + " (ordinal) " + tabletMutations.getDurability().ordinal() + " durability " + durability + " (ordinal) " + durability.ordinal());
       if (tabletMutations.getDurability().ordinal() > durability.ordinal())
         durability = tabletMutations.getDurability();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 56998d4..cb476c9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -31,7 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
@@ -43,11 +43,11 @@ import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
+import org.apache.accumulo.tserver.Mutations;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
 import org.apache.accumulo.tserver.tablet.CommitSession;
-import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
@@ -351,8 +351,6 @@ public class TabletServerLogger {
 
   public int defineTablet(final CommitSession commitSession) throws IOException {
     // scribble this into the metadata tablet, too.
-    if (!enabled(commitSession))
-      return -1;
     return write(commitSession, false, new Writer() {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
@@ -362,29 +360,29 @@ public class TabletServerLogger {
     });
   }
 
-  private boolean enabled(CommitSession commitSession) {
-    return commitSession.getDurabilty() != Durability.NONE;
-  }
-
-  public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m) throws IOException {
-    if (!enabled(commitSession))
+  public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m, final Durability durability) throws IOException {
+    if (durability == Durability.NONE)
       return -1;
+    if (durability == Durability.DEFAULT)
+      log.warn("Unexpected durability " + durability, new Throwable());
     int seq = write(commitSession, false, new Writer() {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
-        return logger.log(tabletSeq, commitSession.getLogId(), m, commitSession.getDurabilty());
+        return logger.log(tabletSeq, commitSession.getLogId(), m, durability);
       }
     });
     logSizeEstimate.addAndGet(m.numBytes());
     return seq;
   }
 
-  public int logManyTablets(Map<CommitSession,List<Mutation>> mutations) throws IOException {
+  public int logManyTablets(Map<CommitSession,Mutations> mutations) throws IOException {
 
-    final Map<CommitSession,List<Mutation>> loggables = new HashMap<CommitSession,List<Mutation>>(mutations);
-    for (CommitSession t : mutations.keySet()) {
-      if (!enabled(t))
-        loggables.remove(t);
+    final Map<CommitSession,Mutations> loggables = new HashMap<CommitSession,Mutations>(mutations);
+    for (Entry<CommitSession,Mutations> entry : mutations.entrySet()) {
+      Durability durability = entry.getValue().getDurability();
+      if (durability == Durability.NONE) {
+        loggables.remove(entry.getKey());
+      }
     }
     if (loggables.size() == 0)
       return -1;
@@ -393,17 +391,18 @@ public class TabletServerLogger {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
         List<TabletMutations> copy = new ArrayList<TabletMutations>(loggables.size());
-        for (Entry<CommitSession,List<Mutation>> entry : loggables.entrySet()) {
+        for (Entry<CommitSession,Mutations> entry : loggables.entrySet()) {
           CommitSession cs = entry.getKey();
-          copy.add(new TabletMutations(cs.getLogId(), cs.getWALogSeq(), entry.getValue(), cs.getDurabilty()));
+          Durability durability = entry.getValue().getDurability();
+          copy.add(new TabletMutations(cs.getLogId(), cs.getWALogSeq(), entry.getValue().getMutations(), durability));
         }
         return logger.logManyTablets(copy);
       }
     });
-    for (List<Mutation> entry : loggables.values()) {
-      if (entry.size() < 1)
+    for (Mutations entry : loggables.values()) {
+      if (entry.getMutations().size() < 1)
         throw new IllegalArgumentException("logManyTablets: logging empty mutation list");
-      for (Mutation m : entry) {
+      for (Mutation m : entry.getMutations()) {
         logSizeEstimate.addAndGet(m.numBytes());
       }
     }
@@ -412,9 +411,6 @@ public class TabletServerLogger {
 
   public void minorCompactionFinished(final CommitSession commitSession, final String fullyQualifiedFileName, final int walogSeq) throws IOException {
 
-    if (!enabled(commitSession))
-      return;
-
     long t1 = System.currentTimeMillis();
 
     int seq = write(commitSession, true, new Writer() {
@@ -431,8 +427,6 @@ public class TabletServerLogger {
   }
 
   public int minorCompactionStarted(final CommitSession commitSession, final int seq, final String fullyQualifiedFileName) throws IOException {
-    if (!enabled(commitSession))
-      return -1;
     write(commitSession, false, new Writer() {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
@@ -445,8 +439,6 @@ public class TabletServerLogger {
 
   public void recover(VolumeManager fs, KeyExtent extent, TableConfiguration tconf, List<Path> logs, Set<String> tabletFiles, MutationReceiver mr)
       throws IOException {
-    if (Durability.fromString(tconf.get(Property.TABLE_DURABILITY)) == Durability.NONE)
-      return;
     try {
       SortedLogRecovery recovery = new SortedLogRecovery(fs);
       recovery.recover(extent, logs, tabletFiles, mr);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
index 26668f6..d2515e6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
@@ -20,18 +20,21 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.client.Durability;
 
 public class ConditionalSession extends Session {
   public final TCredentials credentials;
   public final Authorizations auths;
   public final String tableId;
   public final AtomicBoolean interruptFlag = new AtomicBoolean();
+  public final Durability durability;
   
-  public ConditionalSession(TCredentials credentials, Authorizations authorizations, String tableId) {
+  public ConditionalSession(TCredentials credentials, Authorizations authorizations, String tableId, Durability durability) {
     super(credentials);
     this.credentials = credentials;
     this.auths = authorizations;
     this.tableId = tableId;
+    this.durability = durability;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
index bc04a85..65430ce 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.tserver.TservConstraintEnv;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.tserver.tablet.Tablet;
 
 public class UpdateSession extends Session {
@@ -46,11 +47,12 @@ public class UpdateSession extends Session {
   public long totalUpdates = 0;
   public long flushTime = 0;
   public long queuedMutationSize = 0;
+  public final Durability durability;
   
-  public UpdateSession(TservConstraintEnv env, TCredentials credentials) {
+  public UpdateSession(TservConstraintEnv env, TCredentials credentials, Durability durability) {
     super(credentials);
     this.cenv = env;
     this.violations = new Violations();
+    this.durability = durability;
   }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index b2d89c9..db4100f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -111,10 +111,6 @@ public class CommitSession {
     return maxCommittedTime;
   }
 
-  public Durability getDurabilty() {
-    return committer.getDurability();
-  }
-
   public void mutate(List<Mutation> mutations) {
     memTable.mutate(mutations);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
deleted file mode 100644
index a88e377..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.tserver.tablet;
-
-public enum Durability {
-  NONE,
-  LOG,
-  FLUSH,
-  SYNC;
-
-  static public Durability fromString(String value) {
-    try {
-      return Durability.valueOf(value.toUpperCase());
-    } catch (IllegalArgumentException ex) {
-      return Durability.SYNC;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index fdf072a..965324a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index b6bb458..3042267 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.tserver.tablet;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.server.conf.TableConfiguration;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
index aeba2e0..51880b5 100644
--- a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.test;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
@@ -46,7 +47,7 @@ public class WrongTabletTest {
       Mutation mutation = new Mutation(new Text("row_0003750001"));
       mutation.putDelete(new Text("colf"), new Text("colq"));
       client.update(Tracer.traceInfo(), new Credentials(opts.principal, opts.getToken()).toThrift(opts.getInstance()), new KeyExtent(new Text("!!"), null,
-          new Text("row_0003750000")).toThrift(), mutation.toThrift());
+          new Text("row_0003750000")).toThrift(), mutation.toThrift(), TDurability.DEFAULT);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index 6c34172..ee8a80d 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -93,7 +94,7 @@ public class NullTserver {
     }
     
     @Override
-    public long startUpdate(TInfo tinfo, TCredentials credentials) {
+    public long startUpdate(TInfo tinfo, TCredentials credentials, TDurability durability) {
       return updateSession++;
     }
     
@@ -144,7 +145,7 @@ public class NullTserver {
     }
     
     @Override
-    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent keyExtent, TMutation mutation) {
+    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent keyExtent, TMutation mutation, TDurability durability) {
       
     }
     
@@ -207,7 +208,7 @@ public class NullTserver {
     }
     
     @Override
-    public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID)
+    public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability)
         throws ThriftSecurityException, TException {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 590c7c4..05e3bef 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -34,49 +34,58 @@ import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.minicluster.impl.ProcessReference;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.junit.Test;
 
 public class DurabilityIT extends ConfigurableMacIT {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.useMiniDFS(true);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setNumTservers(1);
   }
 
   static final long N = 100000;
 
-  String tableNames[] = null;
-
-  void init() throws Exception {
-    synchronized (this) {
-      if (tableNames == null) {
-        tableNames = getUniqueNames(4);
-        Connector c = getConnector();
-        TableOperations tableOps = c.tableOperations();
-        tableOps.create(tableNames[0]);
-        tableOps.create(tableNames[1]);
-        tableOps.create(tableNames[2]);
-        tableOps.create(tableNames[3]);
-        // default is sync
-        tableOps.setProperty(tableNames[1], Property.TABLE_DURABILITY.getKey(), "flush");
-        tableOps.setProperty(tableNames[2], Property.TABLE_DURABILITY.getKey(), "log");
-        tableOps.setProperty(tableNames[3], Property.TABLE_DURABILITY.getKey(), "none");
-        // zookeeper propagation
-        UtilWaitThread.sleep(2 * 1000);
-      }
+  private String[] init() throws Exception {
+    String[] tableNames = getUniqueNames(4);
+    Connector c = getConnector();
+    TableOperations tableOps = c.tableOperations();
+    createTable(tableNames[0]);
+    createTable(tableNames[1]);
+    createTable(tableNames[2]);
+    createTable(tableNames[3]);
+    // default is sync
+    tableOps.setProperty(tableNames[1], Property.TABLE_DURABILITY.getKey(), "flush");
+    tableOps.setProperty(tableNames[2], Property.TABLE_DURABILITY.getKey(), "log");
+    tableOps.setProperty(tableNames[3], Property.TABLE_DURABILITY.getKey(), "none");
+    UtilWaitThread.sleep(1000);
+    return tableNames;
+  }
+  
+  private void cleanup(String[] tableNames) throws Exception {
+    Connector c = getConnector();
+    for (String tableName : tableNames) {
+      c.tableOperations().delete(tableName);
     }
   }
+  
+  private void createTable(String tableName) throws Exception {
+    TableOperations tableOps = getConnector().tableOperations();
+    tableOps.create(tableName);
+  }
 
   @Test(timeout = 2 * 60 * 1000)
   public void testWriteSpeed() throws Exception {
-    init();
-    // write some gunk
-    long t0 = writeSome(tableNames[0], N); flush(tableNames[0]);
-    long t1 = writeSome(tableNames[1], N); flush(tableNames[1]);
-    long t2 = writeSome(tableNames[2], N); flush(tableNames[2]);
-    long t3 = writeSome(tableNames[3], N); flush(tableNames[3]);
-    System.out.println(String.format("t0 %d t1 %d t2 %d t3 %d", t0, t1, t2, t3));
+    TableOperations tableOps = getConnector().tableOperations();
+    String tableNames[] = init();
+    // write some gunk, delete the table to keep that table from messing with the performance numbers of successive calls
+    long t0 = writeSome(tableNames[0], N); tableOps.delete(tableNames[0]);
+    long t1 = writeSome(tableNames[1], N); tableOps.delete(tableNames[1]);
+    long t2 = writeSome(tableNames[2], N); tableOps.delete(tableNames[2]);
+    long t3 = writeSome(tableNames[3], N); tableOps.delete(tableNames[3]);
+    System.out.println(String.format("sync %d flush %d log %d none %d", t0, t1, t2, t3));
     assertTrue(t0 > t1);
     assertTrue(t1 > t2);
     assertTrue(t2 > t3);
@@ -84,42 +93,42 @@ public class DurabilityIT extends ConfigurableMacIT {
 
   @Test(timeout = 4 * 60 * 1000)
   public void testSync() throws Exception {
-    init();
+    String tableNames[] = init();
     // sync table should lose nothing
-    getConnector().tableOperations().deleteRows(tableNames[0], null, null);
     writeSome(tableNames[0], N);
     restartTServer();
     assertEquals(N, readSome(tableNames[0], N));
+    cleanup(tableNames);
   }
 
   @Test(timeout = 4 * 60 * 1000)
   public void testFlush() throws Exception {
-    init();
+    String tableNames[] = init();
     // flush table won't lose anything since we're not losing power/dfs
-    getConnector().tableOperations().deleteRows(tableNames[1], null, null);
     writeSome(tableNames[1], N);
     restartTServer();
     assertEquals(N, readSome(tableNames[1], N));
+    cleanup(tableNames);
   }
 
   @Test(timeout = 4 * 60 * 1000)
   public void testLog() throws Exception {
-    init();
+    String tableNames[] = init();
     // we're probably going to lose something the the log setting
-    getConnector().tableOperations().deleteRows(tableNames[2], null, null);
     writeSome(tableNames[2], N);
     restartTServer();
-    assertTrue(N > readSome(tableNames[2], N));
+    assertTrue(N >= readSome(tableNames[2], N));
+    cleanup(tableNames);
   }
 
   @Test(timeout = 4 * 60 * 1000)
   public void testNone() throws Exception {
-    init();
+    String tableNames[] = init();
     // probably won't get any data back without logging
-    getConnector().tableOperations().deleteRows(tableNames[3], null, null);
     writeSome(tableNames[3], N);
     restartTServer();
     assertTrue(N > readSome(tableNames[3], N));
+    cleanup(tableNames);
   }
 
   private long readSome(String table, long n) throws Exception {
@@ -137,26 +146,21 @@ public class DurabilityIT extends ConfigurableMacIT {
     cluster.start();
   }
 
-  private void flush(String table) throws Exception {
-    getConnector().tableOperations().flush(table, null, null, true);
-  }
-
   private long writeSome(String table, long count) throws Exception {
     long now = System.currentTimeMillis();
     Connector c = getConnector();
     BatchWriter bw = c.createBatchWriter(table, null);
     for (int i = 1; i < count + 1; i++) {
-      String data = "" + i;
       Mutation m = new Mutation("" + i);
-      m.put(data, data, data);
+      m.put("", "", "");
       bw.addMutation(m);
-      if (i % (count/100) == 0) {
+      if (i % (Math.max(1, count/100)) == 0) {
         bw.flush();
       }
     }
     bw.close();
     long result = System.currentTimeMillis() - now;
-    c.tableOperations().flush(table, null, null, true);
+    //c.tableOperations().flush(table, null, null, true);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
new file mode 100644
index 0000000..1f84327
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -0,0 +1,75 @@
+package org.apache.accumulo.test.functional;
+
+import static org.junit.Assert.*;
+
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Durability;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.junit.Test;
+
+public class SessionDurabilityIT extends ConfigurableMacIT {
+  
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+  }
+  
+  @Test
+  public void nondurableTableHasDurableWrites() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "none");
+    BatchWriterConfig cfg = new BatchWriterConfig();
+    cfg.setDurability(Durability.SYNC);
+    write(tableName, 10, cfg);
+    assertEquals(10, count(tableName));
+    restartTServer();
+    assertEquals(10, count(tableName));
+  }
+  
+  private int count(String tableName) throws Exception {
+    Connector c = getConnector();
+    Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
+    int result = 0;
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry :scanner) {
+      result++;
+    }
+    return result;
+  }
+
+  private void write(String tableName, int n, BatchWriterConfig cfg) throws Exception {
+    Connector c = getConnector();
+    BatchWriter bw = c.createBatchWriter(tableName, cfg);
+    for (int i = 0; i < 10; i++) {
+      Mutation m = new Mutation(i + "");
+      m.put("", "", "");
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+  
+  private void restartTServer() throws Exception {
+    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+      cluster.killProcess(ServerType.TABLET_SERVER, proc);
+    }
+    cluster.start();
+  }
+
+}


[05/18] git commit: ACCUMULO-1957 per-table durability settings

Posted by ec...@apache.org.
ACCUMULO-1957 per-table durability settings


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/e3aa7eac
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/e3aa7eac
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/e3aa7eac

Branch: refs/heads/master
Commit: e3aa7eac9ee5a8ac79b481cae5d8a47d62a104b5
Parents: 17f6250
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Aug 27 15:29:32 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:58 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/core/conf/Property.java |  13 +-
 .../apache/accumulo/core/conf/PropertyType.java |   2 +
 .../apache/accumulo/server/init/Initialize.java |   2 +-
 .../accumulo/tserver/TabletMutations.java       |  10 +-
 .../apache/accumulo/tserver/TabletServer.java   |   3 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |  65 +++++---
 .../tserver/log/TabletServerLogger.java         |  19 +--
 .../accumulo/tserver/tablet/CommitSession.java  |   4 +-
 .../accumulo/tserver/tablet/Durability.java     |  33 ++++
 .../apache/accumulo/tserver/tablet/Tablet.java  |   4 +-
 .../tserver/tablet/TabletCommitter.java         |   2 +-
 .../accumulo/test/functional/DurabilityIT.java  | 163 +++++++++++++++++++
 12 files changed, 275 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 9837867..72d9aa1 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -276,10 +276,8 @@ public enum Property {
       "The number of threads for the distributed work queue. These threads are used for copying failed bulk files."),
   TSERV_WAL_SYNC("tserver.wal.sync", "true", PropertyType.BOOLEAN,
       "Use the SYNC_BLOCK create flag to sync WAL writes to disk. Prevents problems recovering from sudden system resets."),
-  TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING, "The method to invoke when sync'ing WALs. HSync will provide " +
-      "resiliency in the face of unexpected power outages, at the cost of speed. If method is not available, the legacy 'sync' method " +
-      "will be used to ensure backwards compatibility with older Hadoop versions. A value of 'hflush' is the alternative to the default value " +
-      "of 'hsync' which will result in faster writes, but with less durability"),
+  @Deprecated
+  TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING, "This property is deprecated. Use table.durability instead."),
   TSERV_REPLICATION_REPLAYERS("tserver.replication.replayer.", null, PropertyType.PREFIX, "Allows configuration of implementation used to apply replicated data"),
   TSERV_REPLICATION_DEFAULT_HANDLER("tserver.replication.default.replayer", "org.apache.accumulo.tserver.replication.BatchWriterReplicationReplayer",
       PropertyType.CLASSNAME, "Default AccumuloReplicationReplayer implementation"),
@@ -379,7 +377,8 @@ public enum Property {
       "Determines the max # of files each tablet in a table can have. When adjusting this property you may want to consider adjusting"
           + " table.compaction.major.ratio also. Setting this property to 0 will make it default to tserver.scan.files.open.max-1, this will prevent a"
           + " tablet from having more files than can be opened. Setting this property low may throttle ingest and increase query performance."),
-  TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN, "Use the write-ahead log to prevent the loss of data."),
+  @Deprecated
+  TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN, "This setting is deprecated.  Use table.durability=none instead."),
   TABLE_BLOOM_ENABLED("table.bloom.enabled", "false", PropertyType.BOOLEAN, "Use bloom filters on this table."),
   TABLE_BLOOM_LOAD_THRESHOLD("table.bloom.load.threshold", "1", PropertyType.COUNT,
       "This number of seeks that would actually use a bloom filter must occur before a file's bloom filter is loaded."
@@ -391,6 +390,10 @@ public enum Property {
           + ",org.apache.accumulo.core.file.keyfunctor.ColumnFamilyFunctor, and org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor are"
           + " allowable values. One can extend any of the above mentioned classes to perform specialized parsing of the key. "),
   TABLE_BLOOM_HASHTYPE("table.bloom.hash.type", "murmur", PropertyType.STRING, "The bloom filter hash type"),
+  TABLE_DURABILITY("table.durability", "sync", PropertyType.DURABILITY, "The durability used to write to the write-ahead log." + 
+      " Legal values are: none, which skips the write-ahead log; " + 
+      "flush, which pushes data to the file system; and " + 
+      "sync, which ensures the data is written to disk."),
   TABLE_FAILURES_IGNORE("table.failures.ignore", "false", PropertyType.BOOLEAN,
       "If you want queries for your table to hang or fail when data is missing from the system, "
           + "then set this to false. When this set to true missing data will be reported but queries "

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index f39a8bd..5d5dd5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -67,6 +67,8 @@ public enum PropertyType {
 
   CLASSNAME("java class", "[\\w$.]*", "A fully qualified java class name representing a class on the classpath.\n"
       + "An example is 'java.lang.String', rather than 'String'"),
+      
+  DURABILITY("durability", "(?:none|log|flush|sync)", "One of 'none', 'flush' or 'sync'."),
 
   STRING("string", ".*",
       "An arbitrary string of characters whose format is unspecified and interpreted based on the context of the property to which it applies."), BOOLEAN(

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 9b952ba..5f1e287 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -130,7 +130,7 @@ public class Initialize {
   static {
     initialMetadataConf.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "32K");
     initialMetadataConf.put(Property.TABLE_FILE_REPLICATION.getKey(), "5");
-    initialMetadataConf.put(Property.TABLE_WALOG_ENABLED.getKey(), "true");
+    initialMetadataConf.put(Property.TABLE_DURABILITY.getKey(), "sync");
     initialMetadataConf.put(Property.TABLE_MAJC_RATIO.getKey(), "1");
     initialMetadataConf.put(Property.TABLE_SPLIT_THRESHOLD.getKey(), "64M");
     initialMetadataConf.put(Property.TABLE_CONSTRAINT_PREFIX.getKey() + "1", MetadataConstraints.class.getName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
index e814f0e..a30fa02 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
@@ -19,16 +19,19 @@ package org.apache.accumulo.tserver;
 import java.util.List;
 
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.tserver.tablet.Durability;
 
 public class TabletMutations {
   private final int tid; 
   private final int seq; 
   private final List<Mutation> mutations;
+  private final Durability durability;
 
-  public TabletMutations(int tid, int seq, List<Mutation> mutations) {
+  public TabletMutations(int tid, int seq, List<Mutation> mutations, Durability durability) {
     this.tid = tid;
     this.seq = seq;
     this.mutations = mutations;
+    this.durability = durability;
   }
 
   public List<Mutation> getMutations() {
@@ -38,10 +41,13 @@ public class TabletMutations {
   public int getTid() {
     return tid;
   }
+  
   public int getSeq() {
     return seq;
   }
   
-  
+  public Durability getDurability() {
+    return durability;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 57e3dee..63bf4a3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -215,6 +215,7 @@ import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.accumulo.tserver.tablet.CompactionInfo;
 import org.apache.accumulo.tserver.tablet.CompactionWatcher;
 import org.apache.accumulo.tserver.tablet.Compactor;
+import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.accumulo.tserver.tablet.KVEntry;
 import org.apache.accumulo.tserver.tablet.ScanBatch;
 import org.apache.accumulo.tserver.tablet.Scanner;
@@ -2899,7 +2900,7 @@ public class TabletServer implements Runnable {
 
   public int createLogId(KeyExtent tablet) {
     AccumuloConfiguration acuTableConf = getTableConfiguration(tablet);
-    if (acuTableConf.getBoolean(Property.TABLE_WALOG_ENABLED)) {
+    if (Durability.fromString(acuTableConf.get(Property.TABLE_DURABILITY)) != Durability.NONE) {
       return logIdGenerator.incrementAndGet();
     }
     return -1;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index c01e54a..d907ee7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -59,6 +59,7 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.logger.LogFileKey;
 import org.apache.accumulo.tserver.logger.LogFileValue;
+import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -124,7 +125,7 @@ public class DfsLogger {
 
   private final Object closeLock = new Object();
 
-  private static final DfsLogger.LogWork CLOSED_MARKER = new DfsLogger.LogWork(null);
+  private static final DfsLogger.LogWork CLOSED_MARKER = new DfsLogger.LogWork(null, Durability.FLUSH);
 
   private static final LogFileValue EMPTY = new LogFileValue();
 
@@ -145,9 +146,31 @@ public class DfsLogger {
           continue;
         }
         workQueue.drainTo(work);
+        
+        Method durabilityMethod = null;
+        loop:
+        for (LogWork logWork : work) {
+          switch (logWork.durability) {
+            case NONE:
+              // shouldn't make it to the work queue
+              break;
+            case LOG:
+              // do nothing
+              break;
+            case SYNC:
+              durabilityMethod = sync;
+              break loop;
+            case FLUSH:
+              if (durabilityMethod == null) {
+                durabilityMethod = flush;
+              }
+              break;
+          }
+        }
 
         try {
-          sync.invoke(logFile);
+          if (durabilityMethod != null)
+            durabilityMethod.invoke(logFile);
         } catch (Exception ex) {
           log.warn("Exception syncing " + ex);
           for (DfsLogger.LogWork logWork : work) {
@@ -165,11 +188,13 @@ public class DfsLogger {
   }
 
   static class LogWork {
-    CountDownLatch latch;
+    final CountDownLatch latch;
+    final Durability durability;
     volatile Exception exception;
 
-    public LogWork(CountDownLatch latch) {
+    public LogWork(CountDownLatch latch, Durability durability) {
       this.latch = latch;
+      this.durability = durability;
     }
   }
 
@@ -213,11 +238,12 @@ public class DfsLogger {
     // filename is unique
     return getFileName().hashCode();
   }
-
+  
   private final ServerResources conf;
   private FSDataOutputStream logFile;
   private DataOutputStream encryptingLogFile = null;
   private Method sync;
+  private Method flush;
   private String logPath;
   private Daemon syncThread;
 
@@ -337,16 +363,13 @@ public class DfsLogger {
       else
         logFile = fs.create(new Path(logPath), true, 0, replication, blockSize);
 
-      String syncMethod = conf.getConfiguration().get(Property.TSERV_WAL_SYNC_METHOD);
       try {
-        // hsync: send data to datanodes and sync the data to disk
-        sync = logFile.getClass().getMethod(syncMethod);
+        sync = logFile.getClass().getMethod("hsync");
+        flush = logFile.getClass().getMethod("hflush");
       } catch (Exception ex) {
-        log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
-
         try {
-          // sync: send data to datanodes
-          sync = logFile.getClass().getMethod("sync");
+          // fall back to sync: send data to datanodes
+          flush = sync = logFile.getClass().getMethod("sync");
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
@@ -387,7 +410,6 @@ public class DfsLogger {
       key.tserverSession = filename;
       key.filename = filename;
       write(key, EMPTY);
-      sync.invoke(logFile);
       log.debug("Got new write-ahead log: " + this);
     } catch (Exception ex) {
       if (logFile != null)
@@ -499,12 +521,12 @@ public class DfsLogger {
     encryptingLogFile.flush();
   }
 
-  public LoggerOperation log(int seq, int tid, Mutation mutation) throws IOException {
-    return logManyTablets(Collections.singletonList(new TabletMutations(tid, seq, Collections.singletonList(mutation))));
+  public LoggerOperation log(int seq, int tid, Mutation mutation, Durability durability) throws IOException {
+    return logManyTablets(Collections.singletonList(new TabletMutations(tid, seq, Collections.singletonList(mutation), durability)));
   }
 
-  private LoggerOperation logFileData(List<Pair<LogFileKey,LogFileValue>> keys) throws IOException {
-    DfsLogger.LogWork work = new DfsLogger.LogWork(new CountDownLatch(1));
+  private LoggerOperation logFileData(List<Pair<LogFileKey,LogFileValue>> keys, Durability durability) throws IOException {
+    DfsLogger.LogWork work = new DfsLogger.LogWork(new CountDownLatch(1), durability);
     synchronized (DfsLogger.this) {
       try {
         for (Pair<LogFileKey,LogFileValue> pair : keys) {
@@ -531,6 +553,7 @@ public class DfsLogger {
   }
 
   public LoggerOperation logManyTablets(List<TabletMutations> mutations) throws IOException {
+    Durability durability = Durability.NONE;
     List<Pair<LogFileKey,LogFileValue>> data = new ArrayList<Pair<LogFileKey,LogFileValue>>();
     for (TabletMutations tabletMutations : mutations) {
       LogFileKey key = new LogFileKey();
@@ -540,8 +563,10 @@ public class DfsLogger {
       LogFileValue value = new LogFileValue();
       value.mutations = tabletMutations.getMutations();
       data.add(new Pair<LogFileKey,LogFileValue>(key, value));
+      if (tabletMutations.getDurability().ordinal() > durability.ordinal())
+        durability = tabletMutations.getDurability();
     }
-    return logFileData(data);
+    return logFileData(data, durability);
   }
 
   public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn) throws IOException {
@@ -549,7 +574,7 @@ public class DfsLogger {
     key.event = COMPACTION_FINISH;
     key.seq = seq;
     key.tid = tid;
-    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
+    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)), Durability.SYNC);
   }
 
   public LoggerOperation minorCompactionStarted(int seq, int tid, String fqfn) throws IOException {
@@ -558,7 +583,7 @@ public class DfsLogger {
     key.seq = seq;
     key.tid = tid;
     key.filename = fqfn;
-    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
+    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)), Durability.SYNC);
   }
 
   public String getLogger() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 26e6891..56998d4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -47,6 +47,7 @@ import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
 import org.apache.accumulo.tserver.tablet.CommitSession;
+import org.apache.accumulo.tserver.tablet.Durability;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
@@ -83,14 +84,6 @@ public class TabletServerLogger {
 
   private final AtomicInteger seqGen = new AtomicInteger();
 
-  private static boolean enabled(TableConfiguration tconf) {
-    return tconf.getBoolean(Property.TABLE_WALOG_ENABLED);
-  }
-
-  private static boolean enabled(CommitSession commitSession) {
-    return commitSession.getUseWAL();
-  }
-
   static private abstract class TestCallWithWriteLock {
     abstract boolean test();
 
@@ -369,13 +362,17 @@ public class TabletServerLogger {
     });
   }
 
+  private boolean enabled(CommitSession commitSession) {
+    return commitSession.getDurabilty() != Durability.NONE;
+  }
+
   public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m) throws IOException {
     if (!enabled(commitSession))
       return -1;
     int seq = write(commitSession, false, new Writer() {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
-        return logger.log(tabletSeq, commitSession.getLogId(), m);
+        return logger.log(tabletSeq, commitSession.getLogId(), m, commitSession.getDurabilty());
       }
     });
     logSizeEstimate.addAndGet(m.numBytes());
@@ -398,7 +395,7 @@ public class TabletServerLogger {
         List<TabletMutations> copy = new ArrayList<TabletMutations>(loggables.size());
         for (Entry<CommitSession,List<Mutation>> entry : loggables.entrySet()) {
           CommitSession cs = entry.getKey();
-          copy.add(new TabletMutations(cs.getLogId(), cs.getWALogSeq(), entry.getValue()));
+          copy.add(new TabletMutations(cs.getLogId(), cs.getWALogSeq(), entry.getValue(), cs.getDurabilty()));
         }
         return logger.logManyTablets(copy);
       }
@@ -448,7 +445,7 @@ public class TabletServerLogger {
 
   public void recover(VolumeManager fs, KeyExtent extent, TableConfiguration tconf, List<Path> logs, Set<String> tabletFiles, MutationReceiver mr)
       throws IOException {
-    if (!enabled(tconf))
+    if (Durability.fromString(tconf.get(Property.TABLE_DURABILITY)) == Durability.NONE)
       return;
     try {
       SortedLogRecovery recovery = new SortedLogRecovery(fs);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index 6402797..b2d89c9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -111,8 +111,8 @@ public class CommitSession {
     return maxCommittedTime;
   }
 
-  public boolean getUseWAL() {
-    return committer.getUseWAL();
+  public Durability getDurabilty() {
+    return committer.getDurability();
   }
 
   public void mutate(List<Mutation> mutations) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
new file mode 100644
index 0000000..675b196
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.tserver.tablet;
+
+public enum Durability {
+  NONE,
+  LOG,
+  FLUSH,
+  SYNC;
+  
+  static public Durability fromString(String value) {
+    try {
+      return Durability.valueOf(value.toUpperCase());
+    } catch (IllegalArgumentException ex) {
+      return Durability.SYNC;
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 37950fc..fdf072a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -2510,8 +2510,8 @@ public class Tablet implements TabletCommitter {
   }
 
   @Override
-  public boolean getUseWAL() {
-    return getTableConfiguration().getBoolean(Property.TABLE_WALOG_ENABLED);
+  public Durability getDurability() {
+    return Durability.fromString(getTableConfiguration().get(Property.TABLE_DURABILITY));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index a5d197c..b6bb458 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -44,7 +44,7 @@ public interface TabletCommitter {
 
   int getLogId();
 
-  boolean getUseWAL();
+  Durability getDurability();
 
   void updateMemoryUsageStats(long estimatedSizeInBytes, long estimatedSizeInBytes2);
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e3aa7eac/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
new file mode 100644
index 0000000..b4d9c83
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class DurabilityIT extends ConfigurableMacIT {
+  
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.useMiniDFS(true);
+    cfg.setNumTservers(1);
+  }
+  
+  static final long N = 100000;
+  
+  String tableNames[] = null;
+  
+  void init() throws Exception {
+    synchronized (this) {
+      if (tableNames == null) {
+        tableNames = getUniqueNames(4);
+        Connector c = getConnector();
+        TableOperations tableOps = c.tableOperations();
+        tableOps.create(tableNames[0]); 
+        tableOps.create(tableNames[1]);
+        tableOps.create(tableNames[2]);
+        tableOps.create(tableNames[3]);
+        // default is sync
+        tableOps.setProperty(tableNames[1], Property.TABLE_DURABILITY.getKey(), "flush");
+        tableOps.setProperty(tableNames[2], Property.TABLE_DURABILITY.getKey(), "log");
+        tableOps.setProperty(tableNames[3], Property.TABLE_DURABILITY.getKey(), "none");
+        // zookeeper propagation
+        UtilWaitThread.sleep(2 * 1000);
+      }
+    }
+  }
+
+  @Test(timeout = 2 * 60 * 1000)
+  public void testWriteSpeed() throws Exception {
+    init();
+    // write some gunk
+    long t0 = writeSome(tableNames[0], N); flush(tableNames[0]);
+    long t1 = writeSome(tableNames[1], N); flush(tableNames[1]);
+    long t2 = writeSome(tableNames[2], N); flush(tableNames[2]);
+    long t3 = writeSome(tableNames[3], N); flush(tableNames[3]);
+    System.out.println(String.format("t0 %d t1 %d t2 %d t3 %d", t0, t1, t2, t3));
+    assertTrue(t0 > t1);
+    assertTrue(t1 > t2);
+    assertTrue(t2 > t3);
+  }
+  
+  @Test(timeout = 4 * 60 * 1000)
+  public void testSync() throws Exception {
+    init();
+    // sync table should lose nothing
+    getConnector().tableOperations().deleteRows(tableNames[0], null, null);
+    writeSome(tableNames[0], N);
+    restartTServer();
+    assertEquals(N, readSome(tableNames[0], N));
+  }
+
+  @Test(timeout = 4 * 60 * 1000)
+  public void testFlush() throws Exception {
+    init();
+    // flush table won't lose anything since we're not losing power/dfs
+    getConnector().tableOperations().deleteRows(tableNames[1], null, null); 
+    writeSome(tableNames[1], N);
+    restartTServer();
+    assertEquals(N, readSome(tableNames[1], N));
+  }
+
+  @Test(timeout = 4 * 60 * 1000)
+  public void testLog() throws Exception {
+    init();
+    // we're probably going to lose something the the log setting
+    getConnector().tableOperations().deleteRows(tableNames[2], null, null); 
+    writeSome(tableNames[2], N);
+    restartTServer();
+    assertTrue(N > readSome(tableNames[2], N));
+  }
+  
+  @Test(timeout = 4 * 60 * 1000)
+  public void testNone() throws Exception {
+    init();
+    // probably won't get any data back without logging
+    getConnector().tableOperations().deleteRows(tableNames[3], null, null);
+    writeSome(tableNames[3], N);
+    restartTServer();
+    assertTrue(N > readSome(tableNames[3], N));
+  }
+
+  private long readSome(String table, long n) throws Exception {
+    long count = 0;
+    for (@SuppressWarnings("unused") Entry<Key,Value> entry : getConnector().createScanner(table, Authorizations.EMPTY)) {
+      count++;
+    }
+    return count;
+  }
+
+  private void restartTServer() throws Exception {
+    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+      cluster.killProcess(ServerType.TABLET_SERVER, proc);
+    }
+    cluster.start();
+  }
+
+  private void flush(String table) throws Exception {
+    getConnector().tableOperations().flush(table, null, null, true);
+  }
+
+  private long writeSome(String table, long count) throws Exception {
+    long now = System.currentTimeMillis();
+    Connector c = getConnector();
+    BatchWriter bw = c.createBatchWriter(table, null);
+    for (int i = 1; i < count + 1; i++) {
+      String data = "" + i;
+      Mutation m = new Mutation("" + i);
+      m.put(data, data, data);
+      bw.addMutation(m);
+      if (i % (count/100) == 0) {
+        bw.flush();
+      }
+    }
+    bw.close();
+    long result = System.currentTimeMillis() - now;
+    c.tableOperations().flush(table, null, null, true);
+    return result;
+  }
+  
+}


[14/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/cpp/proxy_types.cpp
----------------------------------------------------------------------
diff --git a/proxy/src/main/cpp/proxy_types.cpp b/proxy/src/main/cpp/proxy_types.cpp
index d472402..edb0978 100644
--- a/proxy/src/main/cpp/proxy_types.cpp
+++ b/proxy/src/main/cpp/proxy_types.cpp
@@ -122,6 +122,22 @@ const char* _kConditionalStatusNames[] = {
 };
 const std::map<int, const char*> _ConditionalStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(5, _kConditionalStatusValues, _kConditionalStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
+int _kDurabilityValues[] = {
+  Durability::DEFAULT,
+  Durability::NONE,
+  Durability::LOG,
+  Durability::FLUSH,
+  Durability::SYNC
+};
+const char* _kDurabilityNames[] = {
+  "DEFAULT",
+  "NONE",
+  "LOG",
+  "FLUSH",
+  "SYNC"
+};
+const std::map<int, const char*> _Durability_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(5, _kDurabilityValues, _kDurabilityNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
 int _kCompactionTypeValues[] = {
   CompactionType::MINOR,
   CompactionType::MERGE,
@@ -1857,8 +1873,8 @@ void swap(ConditionalUpdates &a, ConditionalUpdates &b) {
   swap(a.__isset, b.__isset);
 }
 
-const char* ConditionalWriterOptions::ascii_fingerprint = "2A7184C7CE319A61E12C337D8EAB3FB9";
-const uint8_t ConditionalWriterOptions::binary_fingerprint[16] = {0x2A,0x71,0x84,0xC7,0xCE,0x31,0x9A,0x61,0xE1,0x2C,0x33,0x7D,0x8E,0xAB,0x3F,0xB9};
+const char* ConditionalWriterOptions::ascii_fingerprint = "C345C04E84A351638B6EACB741BD600E";
+const uint8_t ConditionalWriterOptions::binary_fingerprint[16] = {0xC3,0x45,0xC0,0x4E,0x84,0xA3,0x51,0x63,0x8B,0x6E,0xAC,0xB7,0x41,0xBD,0x60,0x0E};
 
 uint32_t ConditionalWriterOptions::read(::apache::thrift::protocol::TProtocol* iprot) {
 
@@ -1925,6 +1941,16 @@ uint32_t ConditionalWriterOptions::read(::apache::thrift::protocol::TProtocol* i
           xfer += iprot->skip(ftype);
         }
         break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast88;
+          xfer += iprot->readI32(ecast88);
+          this->durability = (Durability::type)ecast88;
+          this->__isset.durability = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1960,15 +1986,20 @@ uint32_t ConditionalWriterOptions::write(::apache::thrift::protocol::TProtocol*
     xfer += oprot->writeFieldBegin("authorizations", ::apache::thrift::protocol::T_SET, 4);
     {
       xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->authorizations.size()));
-      std::set<std::string> ::const_iterator _iter88;
-      for (_iter88 = this->authorizations.begin(); _iter88 != this->authorizations.end(); ++_iter88)
+      std::set<std::string> ::const_iterator _iter89;
+      for (_iter89 = this->authorizations.begin(); _iter89 != this->authorizations.end(); ++_iter89)
       {
-        xfer += oprot->writeBinary((*_iter88));
+        xfer += oprot->writeBinary((*_iter89));
       }
       xfer += oprot->writeSetEnd();
     }
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.durability) {
+    xfer += oprot->writeFieldBegin("durability", ::apache::thrift::protocol::T_I32, 5);
+    xfer += oprot->writeI32((int32_t)this->durability);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -1980,6 +2011,7 @@ void swap(ConditionalWriterOptions &a, ConditionalWriterOptions &b) {
   swap(a.timeoutMs, b.timeoutMs);
   swap(a.threads, b.threads);
   swap(a.authorizations, b.authorizations);
+  swap(a.durability, b.durability);
   swap(a.__isset, b.__isset);
 }
 
@@ -2048,9 +2080,9 @@ uint32_t ActiveScan::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast89;
-          xfer += iprot->readI32(ecast89);
-          this->type = (ScanType::type)ecast89;
+          int32_t ecast90;
+          xfer += iprot->readI32(ecast90);
+          this->type = (ScanType::type)ecast90;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2058,9 +2090,9 @@ uint32_t ActiveScan::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast90;
-          xfer += iprot->readI32(ecast90);
-          this->state = (ScanState::type)ecast90;
+          int32_t ecast91;
+          xfer += iprot->readI32(ecast91);
+          this->state = (ScanState::type)ecast91;
           this->__isset.state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2078,14 +2110,14 @@ uint32_t ActiveScan::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->columns.clear();
-            uint32_t _size91;
-            ::apache::thrift::protocol::TType _etype94;
-            xfer += iprot->readListBegin(_etype94, _size91);
-            this->columns.resize(_size91);
-            uint32_t _i95;
-            for (_i95 = 0; _i95 < _size91; ++_i95)
+            uint32_t _size92;
+            ::apache::thrift::protocol::TType _etype95;
+            xfer += iprot->readListBegin(_etype95, _size92);
+            this->columns.resize(_size92);
+            uint32_t _i96;
+            for (_i96 = 0; _i96 < _size92; ++_i96)
             {
-              xfer += this->columns[_i95].read(iprot);
+              xfer += this->columns[_i96].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2098,14 +2130,14 @@ uint32_t ActiveScan::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->iterators.clear();
-            uint32_t _size96;
-            ::apache::thrift::protocol::TType _etype99;
-            xfer += iprot->readListBegin(_etype99, _size96);
-            this->iterators.resize(_size96);
-            uint32_t _i100;
-            for (_i100 = 0; _i100 < _size96; ++_i100)
+            uint32_t _size97;
+            ::apache::thrift::protocol::TType _etype100;
+            xfer += iprot->readListBegin(_etype100, _size97);
+            this->iterators.resize(_size97);
+            uint32_t _i101;
+            for (_i101 = 0; _i101 < _size97; ++_i101)
             {
-              xfer += this->iterators[_i100].read(iprot);
+              xfer += this->iterators[_i101].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2118,14 +2150,14 @@ uint32_t ActiveScan::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->authorizations.clear();
-            uint32_t _size101;
-            ::apache::thrift::protocol::TType _etype104;
-            xfer += iprot->readListBegin(_etype104, _size101);
-            this->authorizations.resize(_size101);
-            uint32_t _i105;
-            for (_i105 = 0; _i105 < _size101; ++_i105)
+            uint32_t _size102;
+            ::apache::thrift::protocol::TType _etype105;
+            xfer += iprot->readListBegin(_etype105, _size102);
+            this->authorizations.resize(_size102);
+            uint32_t _i106;
+            for (_i106 = 0; _i106 < _size102; ++_i106)
             {
-              xfer += iprot->readBinary(this->authorizations[_i105]);
+              xfer += iprot->readBinary(this->authorizations[_i106]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2185,10 +2217,10 @@ uint32_t ActiveScan::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("columns", ::apache::thrift::protocol::T_LIST, 9);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->columns.size()));
-    std::vector<Column> ::const_iterator _iter106;
-    for (_iter106 = this->columns.begin(); _iter106 != this->columns.end(); ++_iter106)
+    std::vector<Column> ::const_iterator _iter107;
+    for (_iter107 = this->columns.begin(); _iter107 != this->columns.end(); ++_iter107)
     {
-      xfer += (*_iter106).write(oprot);
+      xfer += (*_iter107).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2197,10 +2229,10 @@ uint32_t ActiveScan::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("iterators", ::apache::thrift::protocol::T_LIST, 10);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->iterators.size()));
-    std::vector<IteratorSetting> ::const_iterator _iter107;
-    for (_iter107 = this->iterators.begin(); _iter107 != this->iterators.end(); ++_iter107)
+    std::vector<IteratorSetting> ::const_iterator _iter108;
+    for (_iter108 = this->iterators.begin(); _iter108 != this->iterators.end(); ++_iter108)
     {
-      xfer += (*_iter107).write(oprot);
+      xfer += (*_iter108).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2209,10 +2241,10 @@ uint32_t ActiveScan::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("authorizations", ::apache::thrift::protocol::T_LIST, 11);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->authorizations.size()));
-    std::vector<std::string> ::const_iterator _iter108;
-    for (_iter108 = this->authorizations.begin(); _iter108 != this->authorizations.end(); ++_iter108)
+    std::vector<std::string> ::const_iterator _iter109;
+    for (_iter109 = this->authorizations.begin(); _iter109 != this->authorizations.end(); ++_iter109)
     {
-      xfer += oprot->writeBinary((*_iter108));
+      xfer += oprot->writeBinary((*_iter109));
     }
     xfer += oprot->writeListEnd();
   }
@@ -2282,14 +2314,14 @@ uint32_t ActiveCompaction::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->inputFiles.clear();
-            uint32_t _size109;
-            ::apache::thrift::protocol::TType _etype112;
-            xfer += iprot->readListBegin(_etype112, _size109);
-            this->inputFiles.resize(_size109);
-            uint32_t _i113;
-            for (_i113 = 0; _i113 < _size109; ++_i113)
+            uint32_t _size110;
+            ::apache::thrift::protocol::TType _etype113;
+            xfer += iprot->readListBegin(_etype113, _size110);
+            this->inputFiles.resize(_size110);
+            uint32_t _i114;
+            for (_i114 = 0; _i114 < _size110; ++_i114)
             {
-              xfer += iprot->readString(this->inputFiles[_i113]);
+              xfer += iprot->readString(this->inputFiles[_i114]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2308,9 +2340,9 @@ uint32_t ActiveCompaction::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast114;
-          xfer += iprot->readI32(ecast114);
-          this->type = (CompactionType::type)ecast114;
+          int32_t ecast115;
+          xfer += iprot->readI32(ecast115);
+          this->type = (CompactionType::type)ecast115;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2318,9 +2350,9 @@ uint32_t ActiveCompaction::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast115;
-          xfer += iprot->readI32(ecast115);
-          this->reason = (CompactionReason::type)ecast115;
+          int32_t ecast116;
+          xfer += iprot->readI32(ecast116);
+          this->reason = (CompactionReason::type)ecast116;
           this->__isset.reason = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2354,14 +2386,14 @@ uint32_t ActiveCompaction::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->iterators.clear();
-            uint32_t _size116;
-            ::apache::thrift::protocol::TType _etype119;
-            xfer += iprot->readListBegin(_etype119, _size116);
-            this->iterators.resize(_size116);
-            uint32_t _i120;
-            for (_i120 = 0; _i120 < _size116; ++_i120)
+            uint32_t _size117;
+            ::apache::thrift::protocol::TType _etype120;
+            xfer += iprot->readListBegin(_etype120, _size117);
+            this->iterators.resize(_size117);
+            uint32_t _i121;
+            for (_i121 = 0; _i121 < _size117; ++_i121)
             {
-              xfer += this->iterators[_i120].read(iprot);
+              xfer += this->iterators[_i121].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2397,10 +2429,10 @@ uint32_t ActiveCompaction::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("inputFiles", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->inputFiles.size()));
-    std::vector<std::string> ::const_iterator _iter121;
-    for (_iter121 = this->inputFiles.begin(); _iter121 != this->inputFiles.end(); ++_iter121)
+    std::vector<std::string> ::const_iterator _iter122;
+    for (_iter122 = this->inputFiles.begin(); _iter122 != this->inputFiles.end(); ++_iter122)
     {
-      xfer += oprot->writeString((*_iter121));
+      xfer += oprot->writeString((*_iter122));
     }
     xfer += oprot->writeListEnd();
   }
@@ -2433,10 +2465,10 @@ uint32_t ActiveCompaction::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("iterators", ::apache::thrift::protocol::T_LIST, 10);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->iterators.size()));
-    std::vector<IteratorSetting> ::const_iterator _iter122;
-    for (_iter122 = this->iterators.begin(); _iter122 != this->iterators.end(); ++_iter122)
+    std::vector<IteratorSetting> ::const_iterator _iter123;
+    for (_iter123 = this->iterators.begin(); _iter123 != this->iterators.end(); ++_iter123)
     {
-      xfer += (*_iter122).write(oprot);
+      xfer += (*_iter123).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2462,8 +2494,8 @@ void swap(ActiveCompaction &a, ActiveCompaction &b) {
   swap(a.__isset, b.__isset);
 }
 
-const char* WriterOptions::ascii_fingerprint = "D6FB71C9973666B9F6B5C20D5B7B19EF";
-const uint8_t WriterOptions::binary_fingerprint[16] = {0xD6,0xFB,0x71,0xC9,0x97,0x36,0x66,0xB9,0xF6,0xB5,0xC2,0x0D,0x5B,0x7B,0x19,0xEF};
+const char* WriterOptions::ascii_fingerprint = "6640C55D2C0D4C8C2E7589456EA0C61A";
+const uint8_t WriterOptions::binary_fingerprint[16] = {0x66,0x40,0xC5,0x5D,0x2C,0x0D,0x4C,0x8C,0x2E,0x75,0x89,0x45,0x6E,0xA0,0xC6,0x1A};
 
 uint32_t WriterOptions::read(::apache::thrift::protocol::TProtocol* iprot) {
 
@@ -2517,6 +2549,16 @@ uint32_t WriterOptions::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast124;
+          xfer += iprot->readI32(ecast124);
+          this->durability = (Durability::type)ecast124;
+          this->__isset.durability = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2549,6 +2591,11 @@ uint32_t WriterOptions::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeI32(this->threads);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.durability) {
+    xfer += oprot->writeFieldBegin("durability", ::apache::thrift::protocol::T_I32, 5);
+    xfer += oprot->writeI32((int32_t)this->durability);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -2560,6 +2607,7 @@ void swap(WriterOptions &a, WriterOptions &b) {
   swap(a.latencyMs, b.latencyMs);
   swap(a.timeoutMs, b.timeoutMs);
   swap(a.threads, b.threads);
+  swap(a.durability, b.durability);
   swap(a.__isset, b.__isset);
 }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/cpp/proxy_types.h
----------------------------------------------------------------------
diff --git a/proxy/src/main/cpp/proxy_types.h b/proxy/src/main/cpp/proxy_types.h
index c320d39..625586c 100644
--- a/proxy/src/main/cpp/proxy_types.h
+++ b/proxy/src/main/cpp/proxy_types.h
@@ -105,6 +105,18 @@ struct ConditionalStatus {
 
 extern const std::map<int, const char*> _ConditionalStatus_VALUES_TO_NAMES;
 
+struct Durability {
+  enum type {
+    DEFAULT = 0,
+    NONE = 1,
+    LOG = 2,
+    FLUSH = 3,
+    SYNC = 4
+  };
+};
+
+extern const std::map<int, const char*> _Durability_VALUES_TO_NAMES;
+
 struct CompactionType {
   enum type {
     MINOR = 0,
@@ -1138,20 +1150,21 @@ class ConditionalUpdates {
 void swap(ConditionalUpdates &a, ConditionalUpdates &b);
 
 typedef struct _ConditionalWriterOptions__isset {
-  _ConditionalWriterOptions__isset() : maxMemory(false), timeoutMs(false), threads(false), authorizations(false) {}
+  _ConditionalWriterOptions__isset() : maxMemory(false), timeoutMs(false), threads(false), authorizations(false), durability(false) {}
   bool maxMemory;
   bool timeoutMs;
   bool threads;
   bool authorizations;
+  bool durability;
 } _ConditionalWriterOptions__isset;
 
 class ConditionalWriterOptions {
  public:
 
-  static const char* ascii_fingerprint; // = "2A7184C7CE319A61E12C337D8EAB3FB9";
-  static const uint8_t binary_fingerprint[16]; // = {0x2A,0x71,0x84,0xC7,0xCE,0x31,0x9A,0x61,0xE1,0x2C,0x33,0x7D,0x8E,0xAB,0x3F,0xB9};
+  static const char* ascii_fingerprint; // = "C345C04E84A351638B6EACB741BD600E";
+  static const uint8_t binary_fingerprint[16]; // = {0xC3,0x45,0xC0,0x4E,0x84,0xA3,0x51,0x63,0x8B,0x6E,0xAC,0xB7,0x41,0xBD,0x60,0x0E};
 
-  ConditionalWriterOptions() : maxMemory(0), timeoutMs(0), threads(0) {
+  ConditionalWriterOptions() : maxMemory(0), timeoutMs(0), threads(0), durability((Durability::type)0) {
   }
 
   virtual ~ConditionalWriterOptions() throw() {}
@@ -1160,6 +1173,7 @@ class ConditionalWriterOptions {
   int64_t timeoutMs;
   int32_t threads;
   std::set<std::string>  authorizations;
+  Durability::type durability;
 
   _ConditionalWriterOptions__isset __isset;
 
@@ -1183,6 +1197,11 @@ class ConditionalWriterOptions {
     __isset.authorizations = true;
   }
 
+  void __set_durability(const Durability::type val) {
+    durability = val;
+    __isset.durability = true;
+  }
+
   bool operator == (const ConditionalWriterOptions & rhs) const
   {
     if (__isset.maxMemory != rhs.__isset.maxMemory)
@@ -1201,6 +1220,10 @@ class ConditionalWriterOptions {
       return false;
     else if (__isset.authorizations && !(authorizations == rhs.authorizations))
       return false;
+    if (__isset.durability != rhs.__isset.durability)
+      return false;
+    else if (__isset.durability && !(durability == rhs.durability))
+      return false;
     return true;
   }
   bool operator != (const ConditionalWriterOptions &rhs) const {
@@ -1455,20 +1478,21 @@ class ActiveCompaction {
 void swap(ActiveCompaction &a, ActiveCompaction &b);
 
 typedef struct _WriterOptions__isset {
-  _WriterOptions__isset() : maxMemory(false), latencyMs(false), timeoutMs(false), threads(false) {}
+  _WriterOptions__isset() : maxMemory(false), latencyMs(false), timeoutMs(false), threads(false), durability(false) {}
   bool maxMemory;
   bool latencyMs;
   bool timeoutMs;
   bool threads;
+  bool durability;
 } _WriterOptions__isset;
 
 class WriterOptions {
  public:
 
-  static const char* ascii_fingerprint; // = "D6FB71C9973666B9F6B5C20D5B7B19EF";
-  static const uint8_t binary_fingerprint[16]; // = {0xD6,0xFB,0x71,0xC9,0x97,0x36,0x66,0xB9,0xF6,0xB5,0xC2,0x0D,0x5B,0x7B,0x19,0xEF};
+  static const char* ascii_fingerprint; // = "6640C55D2C0D4C8C2E7589456EA0C61A";
+  static const uint8_t binary_fingerprint[16]; // = {0x66,0x40,0xC5,0x5D,0x2C,0x0D,0x4C,0x8C,0x2E,0x75,0x89,0x45,0x6E,0xA0,0xC6,0x1A};
 
-  WriterOptions() : maxMemory(0), latencyMs(0), timeoutMs(0), threads(0) {
+  WriterOptions() : maxMemory(0), latencyMs(0), timeoutMs(0), threads(0), durability((Durability::type)0) {
   }
 
   virtual ~WriterOptions() throw() {}
@@ -1477,6 +1501,7 @@ class WriterOptions {
   int64_t latencyMs;
   int64_t timeoutMs;
   int32_t threads;
+  Durability::type durability;
 
   _WriterOptions__isset __isset;
 
@@ -1496,6 +1521,11 @@ class WriterOptions {
     threads = val;
   }
 
+  void __set_durability(const Durability::type val) {
+    durability = val;
+    __isset.durability = true;
+  }
+
   bool operator == (const WriterOptions & rhs) const
   {
     if (!(maxMemory == rhs.maxMemory))
@@ -1506,6 +1536,10 @@ class WriterOptions {
       return false;
     if (!(threads == rhs.threads))
       return false;
+    if (__isset.durability != rhs.__isset.durability)
+      return false;
+    else if (__isset.durability && !(durability == rhs.durability))
+      return false;
     return true;
   }
   bool operator != (const WriterOptions &rhs) const {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index e65b956..1dc1dd3 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -88,6 +88,7 @@ import org.apache.accumulo.proxy.thrift.ConditionalStatus;
 import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
 import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
 import org.apache.accumulo.proxy.thrift.DiskUsage;
+import org.apache.accumulo.proxy.thrift.Durability;
 import org.apache.accumulo.proxy.thrift.KeyValue;
 import org.apache.accumulo.proxy.thrift.KeyValueAndPeek;
 import org.apache.accumulo.proxy.thrift.NoMoreEntriesException;
@@ -1285,12 +1286,31 @@ public class ProxyServer implements AccumuloProxy.Iface {
         cfg.setTimeout(opts.timeoutMs, TimeUnit.MILLISECONDS);
       if (opts.latencyMs != 0)
         cfg.setMaxLatency(opts.latencyMs, TimeUnit.MILLISECONDS);
+      if (opts.isSetDurability()) {
+        cfg.setDurability(getDurability(opts.getDurability()));
+      }
     }
     BatchWriterPlusException result = new BatchWriterPlusException();
     result.writer = getConnector(login).createBatchWriter(tableName, cfg);
     return result;
   }
   
+  private org.apache.accumulo.core.client.Durability getDurability(Durability durability) {
+    switch (durability) {
+      case DEFAULT:
+        return org.apache.accumulo.core.client.Durability.DEFAULT;
+      case FLUSH:
+        return org.apache.accumulo.core.client.Durability.FLUSH;
+      case LOG:
+        return org.apache.accumulo.core.client.Durability.LOG;
+      case NONE:
+        return org.apache.accumulo.core.client.Durability.NONE;
+      case SYNC:
+        return org.apache.accumulo.core.client.Durability.SYNC;
+    }
+    return org.apache.accumulo.core.client.Durability.DEFAULT;
+  }
+
   private IteratorSetting getIteratorSetting(org.apache.accumulo.proxy.thrift.IteratorSetting setting) {
     return new IteratorSetting(setting.priority, setting.name, setting.iteratorClass, setting.getProperties());
   }
@@ -1541,6 +1561,8 @@ public class ProxyServer implements AccumuloProxy.Iface {
         cwc.setTimeout(options.getTimeoutMs(), TimeUnit.MILLISECONDS);
       if (options.isSetAuthorizations() && options.getAuthorizations() != null)
         cwc.setAuthorizations(getAuthorizations(options.getAuthorizations()));
+      if (options.isSetDurability() && options.getDurability() != null)
+        cwc.setDurability(getDurability(options.getDurability()));
       
       ConditionalWriter cw = getConnector(login).createConditionalWriter(tableName, cwc);
       

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ConditionalWriterOptions.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ConditionalWriterOptions.java b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ConditionalWriterOptions.java
index bb4ad51..42b8ca5 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ConditionalWriterOptions.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/ConditionalWriterOptions.java
@@ -55,6 +55,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TIMEOUT_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("timeoutMs", org.apache.thrift.protocol.TType.I64, (short)2);
   private static final org.apache.thrift.protocol.TField THREADS_FIELD_DESC = new org.apache.thrift.protocol.TField("threads", org.apache.thrift.protocol.TType.I32, (short)3);
   private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.SET, (short)4);
+  private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -66,13 +67,23 @@ import org.slf4j.LoggerFactory;
   public long timeoutMs; // optional
   public int threads; // optional
   public Set<ByteBuffer> authorizations; // optional
+  /**
+   * 
+   * @see Durability
+   */
+  public Durability durability; // optional
 
   /** 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 {
     MAX_MEMORY((short)1, "maxMemory"),
     TIMEOUT_MS((short)2, "timeoutMs"),
     THREADS((short)3, "threads"),
-    AUTHORIZATIONS((short)4, "authorizations");
+    AUTHORIZATIONS((short)4, "authorizations"),
+    /**
+     * 
+     * @see Durability
+     */
+    DURABILITY((short)5, "durability");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -95,6 +106,8 @@ import org.slf4j.LoggerFactory;
           return THREADS;
         case 4: // AUTHORIZATIONS
           return AUTHORIZATIONS;
+        case 5: // DURABILITY
+          return DURABILITY;
         default:
           return null;
       }
@@ -139,7 +152,7 @@ import org.slf4j.LoggerFactory;
   private static final int __TIMEOUTMS_ISSET_ID = 1;
   private static final int __THREADS_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.MAX_MEMORY,_Fields.TIMEOUT_MS,_Fields.THREADS,_Fields.AUTHORIZATIONS};
+  private _Fields optionals[] = {_Fields.MAX_MEMORY,_Fields.TIMEOUT_MS,_Fields.THREADS,_Fields.AUTHORIZATIONS,_Fields.DURABILITY};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -152,6 +165,8 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Durability.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ConditionalWriterOptions.class, metaDataMap);
   }
@@ -171,6 +186,9 @@ import org.slf4j.LoggerFactory;
       Set<ByteBuffer> __this__authorizations = new HashSet<ByteBuffer>(other.authorizations);
       this.authorizations = __this__authorizations;
     }
+    if (other.isSetDurability()) {
+      this.durability = other.durability;
+    }
   }
 
   public ConditionalWriterOptions deepCopy() {
@@ -186,6 +204,7 @@ import org.slf4j.LoggerFactory;
     setThreadsIsSet(false);
     this.threads = 0;
     this.authorizations = null;
+    this.durability = null;
   }
 
   public long getMaxMemory() {
@@ -296,6 +315,38 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  /**
+   * 
+   * @see Durability
+   */
+  public Durability getDurability() {
+    return this.durability;
+  }
+
+  /**
+   * 
+   * @see Durability
+   */
+  public ConditionalWriterOptions setDurability(Durability durability) {
+    this.durability = durability;
+    return this;
+  }
+
+  public void unsetDurability() {
+    this.durability = null;
+  }
+
+  /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+  public boolean isSetDurability() {
+    return this.durability != null;
+  }
+
+  public void setDurabilityIsSet(boolean value) {
+    if (!value) {
+      this.durability = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case MAX_MEMORY:
@@ -330,6 +381,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case DURABILITY:
+      if (value == null) {
+        unsetDurability();
+      } else {
+        setDurability((Durability)value);
+      }
+      break;
+
     }
   }
 
@@ -347,6 +406,9 @@ import org.slf4j.LoggerFactory;
     case AUTHORIZATIONS:
       return getAuthorizations();
 
+    case DURABILITY:
+      return getDurability();
+
     }
     throw new IllegalStateException();
   }
@@ -366,6 +428,8 @@ import org.slf4j.LoggerFactory;
       return isSetThreads();
     case AUTHORIZATIONS:
       return isSetAuthorizations();
+    case DURABILITY:
+      return isSetDurability();
     }
     throw new IllegalStateException();
   }
@@ -419,6 +483,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_durability = true && this.isSetDurability();
+    boolean that_present_durability = true && that.isSetDurability();
+    if (this_present_durability || that_present_durability) {
+      if (!(this_present_durability && that_present_durability))
+        return false;
+      if (!this.durability.equals(that.durability))
+        return false;
+    }
+
     return true;
   }
 
@@ -475,6 +548,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDurability()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -522,6 +605,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetDurability()) {
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -609,6 +702,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // DURABILITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.durability = Durability.findByValue(iprot.readI32());
+              struct.setDurabilityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -653,6 +754,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.durability != null) {
+        if (struct.isSetDurability()) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -683,7 +791,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetAuthorizations()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetDurability()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
       if (struct.isSetMaxMemory()) {
         oprot.writeI64(struct.maxMemory);
       }
@@ -702,12 +813,15 @@ import org.slf4j.LoggerFactory;
           }
         }
       }
+      if (struct.isSetDurability()) {
+        oprot.writeI32(struct.durability.getValue());
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ConditionalWriterOptions struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
         struct.maxMemory = iprot.readI64();
         struct.setMaxMemoryIsSet(true);
@@ -733,6 +847,10 @@ import org.slf4j.LoggerFactory;
         }
         struct.setAuthorizationsIsSet(true);
       }
+      if (incoming.get(4)) {
+        struct.durability = Durability.findByValue(iprot.readI32());
+        struct.setDurabilityIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/java/org/apache/accumulo/proxy/thrift/Durability.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/thrift/Durability.java b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/Durability.java
new file mode 100644
index 0000000..cf729fc
--- /dev/null
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/Durability.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.proxy.thrift;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+@SuppressWarnings("all") public enum Durability implements org.apache.thrift.TEnum {
+  DEFAULT(0),
+  NONE(1),
+  LOG(2),
+  FLUSH(3),
+  SYNC(4);
+
+  private final int value;
+
+  private Durability(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static Durability findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return DEFAULT;
+      case 1:
+        return NONE;
+      case 2:
+        return LOG;
+      case 3:
+        return FLUSH;
+      case 4:
+        return SYNC;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/java/org/apache/accumulo/proxy/thrift/WriterOptions.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/thrift/WriterOptions.java b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/WriterOptions.java
index 5f000a8..97a57c8 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/thrift/WriterOptions.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/thrift/WriterOptions.java
@@ -55,6 +55,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("latencyMs", org.apache.thrift.protocol.TType.I64, (short)2);
   private static final org.apache.thrift.protocol.TField TIMEOUT_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("timeoutMs", org.apache.thrift.protocol.TType.I64, (short)3);
   private static final org.apache.thrift.protocol.TField THREADS_FIELD_DESC = new org.apache.thrift.protocol.TField("threads", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -66,13 +67,23 @@ import org.slf4j.LoggerFactory;
   public long latencyMs; // required
   public long timeoutMs; // required
   public int threads; // required
+  /**
+   * 
+   * @see Durability
+   */
+  public Durability durability; // optional
 
   /** 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 {
     MAX_MEMORY((short)1, "maxMemory"),
     LATENCY_MS((short)2, "latencyMs"),
     TIMEOUT_MS((short)3, "timeoutMs"),
-    THREADS((short)4, "threads");
+    THREADS((short)4, "threads"),
+    /**
+     * 
+     * @see Durability
+     */
+    DURABILITY((short)5, "durability");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -95,6 +106,8 @@ import org.slf4j.LoggerFactory;
           return TIMEOUT_MS;
         case 4: // THREADS
           return THREADS;
+        case 5: // DURABILITY
+          return DURABILITY;
         default:
           return null;
       }
@@ -140,6 +153,7 @@ import org.slf4j.LoggerFactory;
   private static final int __TIMEOUTMS_ISSET_ID = 2;
   private static final int __THREADS_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
+  private _Fields optionals[] = {_Fields.DURABILITY};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -151,6 +165,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.THREADS, new org.apache.thrift.meta_data.FieldMetaData("threads", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Durability.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriterOptions.class, metaDataMap);
   }
@@ -184,6 +200,9 @@ import org.slf4j.LoggerFactory;
     this.latencyMs = other.latencyMs;
     this.timeoutMs = other.timeoutMs;
     this.threads = other.threads;
+    if (other.isSetDurability()) {
+      this.durability = other.durability;
+    }
   }
 
   public WriterOptions deepCopy() {
@@ -200,6 +219,7 @@ import org.slf4j.LoggerFactory;
     this.timeoutMs = 0;
     setThreadsIsSet(false);
     this.threads = 0;
+    this.durability = null;
   }
 
   public long getMaxMemory() {
@@ -294,6 +314,38 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __THREADS_ISSET_ID, value);
   }
 
+  /**
+   * 
+   * @see Durability
+   */
+  public Durability getDurability() {
+    return this.durability;
+  }
+
+  /**
+   * 
+   * @see Durability
+   */
+  public WriterOptions setDurability(Durability durability) {
+    this.durability = durability;
+    return this;
+  }
+
+  public void unsetDurability() {
+    this.durability = null;
+  }
+
+  /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+  public boolean isSetDurability() {
+    return this.durability != null;
+  }
+
+  public void setDurabilityIsSet(boolean value) {
+    if (!value) {
+      this.durability = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case MAX_MEMORY:
@@ -328,6 +380,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case DURABILITY:
+      if (value == null) {
+        unsetDurability();
+      } else {
+        setDurability((Durability)value);
+      }
+      break;
+
     }
   }
 
@@ -345,6 +405,9 @@ import org.slf4j.LoggerFactory;
     case THREADS:
       return Integer.valueOf(getThreads());
 
+    case DURABILITY:
+      return getDurability();
+
     }
     throw new IllegalStateException();
   }
@@ -364,6 +427,8 @@ import org.slf4j.LoggerFactory;
       return isSetTimeoutMs();
     case THREADS:
       return isSetThreads();
+    case DURABILITY:
+      return isSetDurability();
     }
     throw new IllegalStateException();
   }
@@ -417,6 +482,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_durability = true && this.isSetDurability();
+    boolean that_present_durability = true && that.isSetDurability();
+    if (this_present_durability || that_present_durability) {
+      if (!(this_present_durability && that_present_durability))
+        return false;
+      if (!this.durability.equals(that.durability))
+        return false;
+    }
+
     return true;
   }
 
@@ -473,6 +547,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDurability()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -508,6 +592,16 @@ import org.slf4j.LoggerFactory;
     sb.append("threads:");
     sb.append(this.threads);
     first = false;
+    if (isSetDurability()) {
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -585,6 +679,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // DURABILITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.durability = Durability.findByValue(iprot.readI32());
+              struct.setDurabilityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -612,6 +714,13 @@ import org.slf4j.LoggerFactory;
       oprot.writeFieldBegin(THREADS_FIELD_DESC);
       oprot.writeI32(struct.threads);
       oprot.writeFieldEnd();
+      if (struct.durability != null) {
+        if (struct.isSetDurability()) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -642,7 +751,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetThreads()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetDurability()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
       if (struct.isSetMaxMemory()) {
         oprot.writeI64(struct.maxMemory);
       }
@@ -655,12 +767,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetThreads()) {
         oprot.writeI32(struct.threads);
       }
+      if (struct.isSetDurability()) {
+        oprot.writeI32(struct.durability.getValue());
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, WriterOptions struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
         struct.maxMemory = iprot.readI64();
         struct.setMaxMemoryIsSet(true);
@@ -677,6 +792,10 @@ import org.slf4j.LoggerFactory;
         struct.threads = iprot.readI32();
         struct.setThreadsIsSet(true);
       }
+      if (incoming.get(4)) {
+        struct.durability = Durability.findByValue(iprot.readI32());
+        struct.setDurabilityIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/python/ttypes.py
----------------------------------------------------------------------
diff --git a/proxy/src/main/python/ttypes.py b/proxy/src/main/python/ttypes.py
index 6a1b423..5e4c001 100644
--- a/proxy/src/main/python/ttypes.py
+++ b/proxy/src/main/python/ttypes.py
@@ -168,6 +168,29 @@ class ConditionalStatus:
     "INVISIBLE_VISIBILITY": 4,
   }
 
+class Durability:
+  DEFAULT = 0
+  NONE = 1
+  LOG = 2
+  FLUSH = 3
+  SYNC = 4
+
+  _VALUES_TO_NAMES = {
+    0: "DEFAULT",
+    1: "NONE",
+    2: "LOG",
+    3: "FLUSH",
+    4: "SYNC",
+  }
+
+  _NAMES_TO_VALUES = {
+    "DEFAULT": 0,
+    "NONE": 1,
+    "LOG": 2,
+    "FLUSH": 3,
+    "SYNC": 4,
+  }
+
 class CompactionType:
   MINOR = 0
   MERGE = 1
@@ -1704,6 +1727,7 @@ class ConditionalWriterOptions:
    - timeoutMs
    - threads
    - authorizations
+   - durability
   """
 
   thrift_spec = (
@@ -1712,13 +1736,15 @@ class ConditionalWriterOptions:
     (2, TType.I64, 'timeoutMs', None, None, ), # 2
     (3, TType.I32, 'threads', None, None, ), # 3
     (4, TType.SET, 'authorizations', (TType.STRING,None), None, ), # 4
+    (5, TType.I32, 'durability', None, None, ), # 5
   )
 
-  def __init__(self, maxMemory=None, timeoutMs=None, threads=None, authorizations=None,):
+  def __init__(self, maxMemory=None, timeoutMs=None, threads=None, authorizations=None, durability=None,):
     self.maxMemory = maxMemory
     self.timeoutMs = timeoutMs
     self.threads = threads
     self.authorizations = authorizations
+    self.durability = durability
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1754,6 +1780,11 @@ class ConditionalWriterOptions:
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.durability = iprot.readI32();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1783,6 +1814,10 @@ class ConditionalWriterOptions:
         oprot.writeString(iter99)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
+    if self.durability is not None:
+      oprot.writeFieldBegin('durability', TType.I32, 5)
+      oprot.writeI32(self.durability)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2201,6 +2236,7 @@ class WriterOptions:
    - latencyMs
    - timeoutMs
    - threads
+   - durability
   """
 
   thrift_spec = (
@@ -2209,13 +2245,15 @@ class WriterOptions:
     (2, TType.I64, 'latencyMs', None, None, ), # 2
     (3, TType.I64, 'timeoutMs', None, None, ), # 3
     (4, TType.I32, 'threads', None, None, ), # 4
+    (5, TType.I32, 'durability', None, None, ), # 5
   )
 
-  def __init__(self, maxMemory=None, latencyMs=None, timeoutMs=None, threads=None,):
+  def __init__(self, maxMemory=None, latencyMs=None, timeoutMs=None, threads=None, durability=None,):
     self.maxMemory = maxMemory
     self.latencyMs = latencyMs
     self.timeoutMs = timeoutMs
     self.threads = threads
+    self.durability = durability
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2246,6 +2284,11 @@ class WriterOptions:
           self.threads = iprot.readI32();
         else:
           iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.durability = iprot.readI32();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2272,6 +2315,10 @@ class WriterOptions:
       oprot.writeFieldBegin('threads', TType.I32, 4)
       oprot.writeI32(self.threads)
       oprot.writeFieldEnd()
+    if self.durability is not None:
+      oprot.writeFieldBegin('durability', TType.I32, 5)
+      oprot.writeI32(self.durability)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/ruby/proxy_types.rb
----------------------------------------------------------------------
diff --git a/proxy/src/main/ruby/proxy_types.rb b/proxy/src/main/ruby/proxy_types.rb
index 722e248..beeeee4 100644
--- a/proxy/src/main/ruby/proxy_types.rb
+++ b/proxy/src/main/ruby/proxy_types.rb
@@ -81,6 +81,16 @@ module Accumulo
     VALID_VALUES = Set.new([ACCEPTED, REJECTED, VIOLATED, UNKNOWN, INVISIBLE_VISIBILITY]).freeze
   end
 
+  module Durability
+    DEFAULT = 0
+    NONE = 1
+    LOG = 2
+    FLUSH = 3
+    SYNC = 4
+    VALUE_MAP = {0 => "DEFAULT", 1 => "NONE", 2 => "LOG", 3 => "FLUSH", 4 => "SYNC"}
+    VALID_VALUES = Set.new([DEFAULT, NONE, LOG, FLUSH, SYNC]).freeze
+  end
+
   module CompactionType
     MINOR = 0
     MERGE = 1
@@ -433,17 +443,22 @@ module Accumulo
     TIMEOUTMS = 2
     THREADS = 3
     AUTHORIZATIONS = 4
+    DURABILITY = 5
 
     FIELDS = {
       MAXMEMORY => {:type => ::Thrift::Types::I64, :name => 'maxMemory', :optional => true},
       TIMEOUTMS => {:type => ::Thrift::Types::I64, :name => 'timeoutMs', :optional => true},
       THREADS => {:type => ::Thrift::Types::I32, :name => 'threads', :optional => true},
-      AUTHORIZATIONS => {:type => ::Thrift::Types::SET, :name => 'authorizations', :element => {:type => ::Thrift::Types::STRING, :binary => true}, :optional => true}
+      AUTHORIZATIONS => {:type => ::Thrift::Types::SET, :name => 'authorizations', :element => {:type => ::Thrift::Types::STRING, :binary => true}, :optional => true},
+      DURABILITY => {:type => ::Thrift::Types::I32, :name => 'durability', :optional => true, :enum_class => ::Accumulo::Durability}
     }
 
     def struct_fields; FIELDS; end
 
     def validate
+      unless @durability.nil? || ::Accumulo::Durability::VALID_VALUES.include?(@durability)
+        raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field durability!')
+      end
     end
 
     ::Thrift::Struct.generate_accessors self
@@ -537,17 +552,22 @@ module Accumulo
     LATENCYMS = 2
     TIMEOUTMS = 3
     THREADS = 4
+    DURABILITY = 5
 
     FIELDS = {
       MAXMEMORY => {:type => ::Thrift::Types::I64, :name => 'maxMemory'},
       LATENCYMS => {:type => ::Thrift::Types::I64, :name => 'latencyMs'},
       TIMEOUTMS => {:type => ::Thrift::Types::I64, :name => 'timeoutMs'},
-      THREADS => {:type => ::Thrift::Types::I32, :name => 'threads'}
+      THREADS => {:type => ::Thrift::Types::I32, :name => 'threads'},
+      DURABILITY => {:type => ::Thrift::Types::I32, :name => 'durability', :optional => true, :enum_class => ::Accumulo::Durability}
     }
 
     def struct_fields; FIELDS; end
 
     def validate
+      unless @durability.nil? || ::Accumulo::Durability::VALID_VALUES.include?(@durability)
+        raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field durability!')
+      end
     end
 
     ::Thrift::Struct.generate_accessors self

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/thrift/proxy.thrift
----------------------------------------------------------------------
diff --git a/proxy/src/main/thrift/proxy.thrift b/proxy/src/main/thrift/proxy.thrift
index 6488f4f..fbd9c52 100644
--- a/proxy/src/main/thrift/proxy.thrift
+++ b/proxy/src/main/thrift/proxy.thrift
@@ -179,12 +179,23 @@ enum ConditionalStatus {
   INVISIBLE_VISIBILITY
 }
 
+//since 1.7.0
+enum Durability {
+	DEFAULT,
+	NONE,
+	LOG,
+	FLUSH,
+	SYNC
+}
+
 //since 1.6.0
 struct ConditionalWriterOptions {
    1:optional i64 maxMemory
    2:optional i64 timeoutMs
    3:optional i32 threads
    4:optional set<binary> authorizations;
+   //since 1.7.0
+   5:optional Durability durability;
 }
 
 struct ActiveScan {
@@ -234,6 +245,8 @@ struct WriterOptions {
  2:i64 latencyMs
  3:i64 timeoutMs
  4:i32 threads
+ //since 1.7.0
+ 5:optional Durability durability
 }
 
 enum IteratorScope {


[16/18] git commit: ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/3a0beab0
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/3a0beab0
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/3a0beab0

Branch: refs/heads/master
Commit: 3a0beab0d785f607b8f2cc73392467a830528b7f
Parents: f5b598e
Author: Eric C. Newton <er...@gmail.com>
Authored: Thu Sep 4 14:05:42 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/BatchWriterConfig.java |    7 +-
 .../apache/accumulo/core/client/Durability.java |   56 +-
 .../core/client/impl/ConditionalWriterImpl.java |    2 +-
 .../core/client/impl/DurabilityImpl.java        |   68 +
 .../client/impl/TabletServerBatchWriter.java    |    4 +-
 .../org/apache/accumulo/core/conf/Property.java |    6 +-
 .../core/client/BatchWriterConfigTest.java      |   16 +
 .../simple/client/ReadWriteExample.java         |   35 +-
 proxy/pom.xml                                   |    5 -
 proxy/src/main/cpp/AccumuloProxy.cpp            | 1454 ++++++++--------
 proxy/src/main/cpp/proxy_types.cpp              |  186 ++-
 proxy/src/main/cpp/proxy_types.h                |   50 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   22 +
 .../proxy/thrift/ConditionalWriterOptions.java  |  126 +-
 .../accumulo/proxy/thrift/Durability.java       |   70 +
 .../accumulo/proxy/thrift/WriterOptions.java    |  125 +-
 proxy/src/main/python/ttypes.py                 |   51 +-
 proxy/src/main/ruby/proxy_types.rb              |   24 +-
 proxy/src/main/thrift/proxy.thrift              |   13 +
 .../apache/accumulo/proxy/SimpleProxyIT.java    | 1575 ------------------
 .../proxy/TestProxyInstanceOperations.java      |   83 -
 .../accumulo/proxy/TestProxyReadWrite.java      |  478 ------
 .../proxy/TestProxySecurityOperations.java      |  157 --
 .../proxy/TestProxyTableOperations.java         |  212 ---
 .../java/org/apache/accumulo/master/Master.java |   35 +
 .../accumulo/tserver/TabletMutations.java       |    4 +-
 .../apache/accumulo/tserver/TabletServer.java   |   34 +-
 .../tserver/TabletServerResourceManager.java    |    7 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   19 +-
 .../tserver/log/TabletServerLogger.java         |    8 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |    4 +-
 .../accumulo/tserver/log/DfsLoggerTest.java     |   59 +
 .../accumulo/shell/commands/InsertCommand.java  |   27 +-
 test/pom.xml                                    |    4 +
 .../apache/accumulo/test/WrongTabletTest.java   |    8 +-
 .../test/performance/thrift/NullTserver.java    |   92 +-
 .../test/randomwalk/concurrent/Config.java      |    2 +-
 .../accumulo/proxy/ProxyDurabilityIT.java       |  141 ++
 .../apache/accumulo/proxy/SimpleProxyIT.java    | 1575 ++++++++++++++++++
 .../proxy/TestProxyInstanceOperations.java      |   83 +
 .../accumulo/proxy/TestProxyReadWrite.java      |  478 ++++++
 .../proxy/TestProxySecurityOperations.java      |  157 ++
 .../proxy/TestProxyTableOperations.java         |  212 +++
 .../accumulo/test/functional/BloomFilterIT.java |    2 +-
 .../accumulo/test/functional/DurabilityIT.java  |    3 +-
 45 files changed, 4296 insertions(+), 3483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index 270a89e..23ab39e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.StringUtils;
@@ -178,6 +179,10 @@ public class BatchWriterConfig implements Writable {
   }
 
   /**
+   * Change the durability for the BatchWriter session. The default durability is "default" which 
+   * is the table's durability setting.  If the durability is set to something other than the default,
+   * it will override the durability setting of the table.
+   * 
    * @param durability the Durability to be used by the BatchWriter
    * @since 1.7.0
    *
@@ -241,7 +246,7 @@ public class BatchWriterConfig implements Writable {
       } else if ("timeout".equals(key)) {
         timeout = Long.valueOf(value);
       } else if ("durability".equals(key)) {
-        durability = Durability.fromString(value);
+        durability = DurabilityImpl.fromString(value);
       } else {
         /* ignore any other properties */
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/Durability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Durability.java b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
index 6de666d..3e69cb2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Durability.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.client;
 
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
-
 /**
  * The value for the durability of a BatchWriter or ConditionalWriter.
  * @since 1.7.0
@@ -43,57 +41,5 @@ public enum Durability {
   /**
    * Write mutations to the write-ahead log, and ensure the data is saved to persistent storage.
    */
-  SYNC;
-
-  // for internal use only
-  public TDurability toThrift() {
-    switch (this) {
-      case DEFAULT:
-        return TDurability.DEFAULT;
-      case SYNC:
-        return TDurability.SYNC;
-      case FLUSH:
-        return TDurability.FLUSH;
-      case LOG:
-        return TDurability.LOG;
-      default:
-        return TDurability.NONE;
-    }
-  }
-
-  // for internal use only
-  static public Durability fromString(String value) {
-    try {
-      return Durability.valueOf(value.toUpperCase());
-    } catch (IllegalArgumentException ex) {
-      return Durability.SYNC;
-    }
-  }
-
-  // for internal use only
-  public static Durability fromThrift(TDurability tdurabilty) {
-    if (tdurabilty == null) {
-      return Durability.DEFAULT;
-    }
-    switch (tdurabilty) {
-      case DEFAULT:
-        return Durability.DEFAULT;
-      case SYNC:
-        return Durability.SYNC;
-      case FLUSH:
-        return Durability.FLUSH;
-      case LOG:
-        return Durability.LOG;
-      default:
-        return Durability.NONE;
-    }
-  }
-
-  // for internal use only
-  public Durability resolveDurability(Durability tabletDurability) {
-    if (this == Durability.DEFAULT) {
-      return tabletDurability;
-    }
-    return this;
-  }
+  SYNC
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 02954fa..e8af187 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -500,7 +500,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
     
     TConditionalSession tcs = client.startConditionalUpdate(tinfo, credentials.toThrift(instance), ByteBufferUtil.toByteBuffers(auths.getAuthorizations()),
-        tableId, durability.toThrift());
+        tableId, DurabilityImpl.toThrift(durability));
     
     synchronized (cachedSessionIDs) {
       SessionID sid = new SessionID();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
new file mode 100644
index 0000000..b2a0a98
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.client.Durability;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+
+public class DurabilityImpl {
+
+  public static TDurability toThrift(Durability durability) {
+    switch (durability) {
+      case DEFAULT:
+        return TDurability.DEFAULT;
+      case SYNC:
+        return TDurability.SYNC;
+      case FLUSH:
+        return TDurability.FLUSH;
+      case LOG:
+        return TDurability.LOG;
+      default:
+        return TDurability.NONE;
+    }
+  }
+
+  public static Durability fromString(String value) {
+    return Durability.valueOf(value.toUpperCase());
+  }
+
+  public static Durability fromThrift(TDurability tdurabilty) {
+    if (tdurabilty == null) {
+      return Durability.DEFAULT;
+    }
+    switch (tdurabilty) {
+      case DEFAULT:
+        return Durability.DEFAULT;
+      case SYNC:
+        return Durability.SYNC;
+      case FLUSH:
+        return Durability.FLUSH;
+      case LOG:
+        return Durability.LOG;
+      default:
+        return Durability.NONE;
+    }
+  }
+
+  public static Durability resolveDurabilty(Durability durability, Durability tabletDurability) {
+    if (durability == Durability.DEFAULT) {
+      return tabletDurability;
+    }
+    return durability;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index c68f842..5eec397 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -860,7 +860,7 @@ public class TabletServerBatchWriter {
             Entry<KeyExtent,List<Mutation>> entry = tabMuts.entrySet().iterator().next();
             
             try {
-              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift(), durability.toThrift());
+              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift(), DurabilityImpl.toThrift(durability));
             } catch (NotServingTabletException e) {
               allFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
               TabletLocator.getLocator(instance, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
@@ -870,7 +870,7 @@ public class TabletServerBatchWriter {
             timeoutTracker.madeProgress();
           } else {
             
-            long usid = client.startUpdate(tinfo, credentials.toThrift(instance), durability.toThrift());
+            long usid = client.startUpdate(tinfo, credentials.toThrift(instance), DurabilityImpl.toThrift(durability));
             
             List<TMutation> updates = new ArrayList<TMutation>();
             for (Entry<KeyExtent,List<Mutation>> entry : tabMuts.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6ac6fa9..35cd0a6 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -197,11 +197,15 @@ public enum Property {
   TSERV_INDEXCACHE_SIZE("tserver.cache.index.size", "512M", PropertyType.MEMORY, "Specifies the size of the cache for file indices."),
   TSERV_PORTSEARCH("tserver.port.search", "false", PropertyType.BOOLEAN, "if the ports above are in use, search higher ports until one is available"),
   TSERV_CLIENTPORT("tserver.port.client", "9997", PropertyType.PORT, "The port used for handling client connections on the tablet servers"),
+  @Deprecated
   TSERV_MUTATION_QUEUE_MAX("tserver.mutation.queue.max", "1M", PropertyType.MEMORY,
-      "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
+      "This setting is deprecated. See tserver.total.mutation.queue.max. " 
+          + "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
           + " max number of concurrent writer when configuring. When using Hadoop 2, Accumulo will call hsync() on the WAL . For a small number of "
           + "concurrent writers, increasing this buffer size decreases the frequncy of hsync calls. For a large number of concurrent writers a small buffers "
           + "size is ok because of group commit."),
+  TSERV_TOTAL_MUTATION_QUEUE_MAX("tserver.total.mutation.queue.max", "50M", PropertyType.MEMORY, 
+      "The amount of memory used to store write-ahead-log mutations before flushing them."),
   TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max", "30", PropertyType.COUNT,
       "To find a tablets split points, all index files are opened. This setting determines how many index "
           + "files can be opened at once. When there are more index files than this setting multiple passes "

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java b/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
index 231afe5..8f0812c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
@@ -40,12 +40,14 @@ public class BatchWriterConfigTest {
     long expectedMaxLatency = 120000l;
     long expectedTimeout = Long.MAX_VALUE;
     int expectedMaxWriteThreads = 3;
+    Durability expectedDurability = Durability.DEFAULT;
     
     BatchWriterConfig defaults = new BatchWriterConfig();
     assertEquals(expectedMaxMemory, defaults.getMaxMemory());
     assertEquals(expectedMaxLatency, defaults.getMaxLatency(TimeUnit.MILLISECONDS));
     assertEquals(expectedTimeout, defaults.getTimeout(TimeUnit.MILLISECONDS));
     assertEquals(expectedMaxWriteThreads, defaults.getMaxWriteThreads());
+    assertEquals(expectedDurability, defaults.getDurability());
   }
   
   @Test
@@ -55,11 +57,13 @@ public class BatchWriterConfigTest {
     bwConfig.setMaxLatency(22, TimeUnit.HOURS);
     bwConfig.setTimeout(33, TimeUnit.DAYS);
     bwConfig.setMaxWriteThreads(42);
+    bwConfig.setDurability(Durability.NONE);
     
     assertEquals(1123581321l, bwConfig.getMaxMemory());
     assertEquals(22 * 60 * 60 * 1000l, bwConfig.getMaxLatency(TimeUnit.MILLISECONDS));
     assertEquals(33 * 24 * 60 * 60 * 1000l, bwConfig.getTimeout(TimeUnit.MILLISECONDS));
     assertEquals(42, bwConfig.getMaxWriteThreads());
+    assertEquals(Durability.NONE, bwConfig.getDurability());
   }
   
   @Test
@@ -133,6 +137,7 @@ public class BatchWriterConfigTest {
     assertNotEquals(9898989l, bwDefaults.getTimeout(TimeUnit.MILLISECONDS));
     assertNotEquals(42, bwDefaults.getMaxWriteThreads());
     assertNotEquals(1123581321l, bwDefaults.getMaxMemory());
+    assertNotEquals(Durability.FLUSH, bwDefaults.getDurability());
     
     // test setting all fields
     BatchWriterConfig bwConfig = new BatchWriterConfig();
@@ -140,6 +145,7 @@ public class BatchWriterConfigTest {
     bwConfig.setTimeout(9898989l, TimeUnit.MILLISECONDS);
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321l);
+    bwConfig.setDurability(Durability.FLUSH);
     byte[] bytes = createBytes(bwConfig);
     checkBytes(bwConfig, bytes);
     
@@ -157,6 +163,12 @@ public class BatchWriterConfigTest {
     bytes = createBytes(bwConfig);
     assertEquals("     v#maxWriteThreads=24,timeout=3000", new String(bytes, StandardCharsets.UTF_8));
     checkBytes(bwConfig, bytes);
+    
+    // test human-readable durability
+    bwConfig = new BatchWriterConfig();
+    bwConfig.setDurability(Durability.LOG);
+    bytes = createBytes(bwConfig);
+    assertEquals("     e#durability=LOG", new String(bytes, StandardCharsets.UTF_8));
   }
 
   @Test
@@ -166,6 +178,10 @@ public class BatchWriterConfigTest {
     assertEquals(cfg1.hashCode(), cfg2.hashCode());
     cfg2.setMaxMemory(1);
     assertNotEquals(cfg1, cfg2);
+    cfg2 = new BatchWriterConfig();
+    cfg2.setDurability(Durability.FLUSH);
+    assertNotEquals(cfg1, cfg2);
+    assertNotEquals(cfg1.hashCode(), cfg2.hashCode());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
index 7bc44e8..2a42d29 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -42,16 +43,16 @@ public class ReadWriteExample {
   // defaults
   private static final String DEFAULT_AUTHS = "LEVEL1,GROUP1";
   private static final String DEFAULT_TABLE_NAME = "test";
-  
+
   private Connector conn;
-  
+
   static class DurabilityConverter implements IStringConverter<Durability> {
     @Override
     public Durability convert(String value) {
-      return Durability.fromString(value);
-    }    
+      return DurabilityImpl.fromString(value);
+    }
   }
-  
+
   static class Opts extends ClientOnDefaultTable {
     @Parameter(names = {"-C", "--createtable"}, description = "create table before doing anything")
     boolean createtable = false;
@@ -65,26 +66,26 @@ public class ReadWriteExample {
     boolean deleteEntries = false;
     @Parameter(names = {"--durability"}, description = "durabilty used for writes (none, log, flush or sync)", converter=DurabilityConverter.class)
     Durability durability = Durability.DEFAULT;
-    
+
     public Opts() {
       super(DEFAULT_TABLE_NAME);
       auths = new Authorizations(DEFAULT_AUTHS.split(","));
     }
   }
-  
+
   // hidden constructor
   private ReadWriteExample() {}
-  
+
   private void execute(Opts opts, ScannerOpts scanOpts) throws Exception {
     conn = opts.getConnector();
-    
+
     // add the authorizations to the user
     Authorizations userAuthorizations = conn.securityOperations().getUserAuthorizations(opts.principal);
     ByteArraySet auths = new ByteArraySet(userAuthorizations.getAuthorizations());
     auths.addAll(opts.auths.getAuthorizations());
     if (!auths.isEmpty())
       conn.securityOperations().changeUserAuthorizations(opts.principal, new Authorizations(auths));
-    
+
     // create table
     if (opts.createtable) {
       SortedSet<Text> partitionKeys = new TreeSet<Text>();
@@ -93,10 +94,10 @@ public class ReadWriteExample {
       conn.tableOperations().create(opts.getTableName());
       conn.tableOperations().addSplits(opts.getTableName(), partitionKeys);
     }
-    
+
     // send mutations
     createEntries(opts);
-    
+
     // read entries
     if (opts.readEntries) {
       // Note that the user needs to have the authorizations for the specified scan authorizations
@@ -106,24 +107,24 @@ public class ReadWriteExample {
       for (Entry<Key,Value> entry : scanner)
         System.out.println(entry.getKey().toString() + " -> " + entry.getValue().toString());
     }
-    
+
     // delete table
     if (opts.deletetable)
       conn.tableOperations().delete(opts.getTableName());
   }
-  
+
   private void createEntries(Opts opts) throws Exception {
     if (opts.createEntries || opts.deleteEntries) {
       BatchWriterConfig cfg = new BatchWriterConfig();
       cfg.setDurability(opts.durability);
       BatchWriter writer = conn.createBatchWriter(opts.getTableName(), cfg);
       ColumnVisibility cv = new ColumnVisibility(opts.auths.toString().replace(',', '|'));
-      
+
       Text cf = new Text("datatypes");
       Text cq = new Text("xml");
       byte[] row = {'h', 'e', 'l', 'l', 'o', '\0'};
       byte[] value = {'w', 'o', 'r', 'l', 'd', '\0'};
-      
+
       for (int i = 0; i < 10; i++) {
         row[row.length - 1] = (byte) i;
         Mutation m = new Mutation(new Text(row));
@@ -139,7 +140,7 @@ public class ReadWriteExample {
       writer.close();
     }
   }
-  
+
   public static void main(String[] args) throws Exception {
     ReadWriteExample rwe = new ReadWriteExample();
     Opts opts = new Opts();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/pom.xml
----------------------------------------------------------------------
diff --git a/proxy/pom.xml b/proxy/pom.xml
index 4f53450..9312d7b 100644
--- a/proxy/pom.xml
+++ b/proxy/pom.xml
@@ -77,11 +77,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-test</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>


[17/18] git commit: ACCUMULO-1957 test/whitespace cleanup

Posted by ec...@apache.org.
ACCUMULO-1957 test/whitespace cleanup


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/e0fe2ae6
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/e0fe2ae6
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/e0fe2ae6

Branch: refs/heads/master
Commit: e0fe2ae61d19108521047fb56dd11044ddbb3894
Parents: c56e300
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Sep 3 10:32:50 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/BatchWriterConfig.java | 32 ++++++------
 .../core/client/ConditionalWriterConfig.java    | 40 +++++++--------
 .../apache/accumulo/core/client/Durability.java |  4 +-
 .../apache/accumulo/tserver/TabletServer.java   | 12 ++---
 .../test/functional/SessionDurabilityIT.java    | 53 +++++++++++++++++++-
 5 files changed, 96 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e0fe2ae6/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index 1897552..270a89e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.util.StringUtils;
 
 /**
  * This object holds configuration settings used to instantiate a {@link BatchWriter}
- * 
+ *
  * @since 1.5.0
  */
 public class BatchWriterConfig implements Writable {
@@ -46,16 +46,16 @@ public class BatchWriterConfig implements Writable {
 
   private static final Integer DEFAULT_MAX_WRITE_THREADS = 3;
   private Integer maxWriteThreads = null;
-  
+
   private Durability durability = Durability.DEFAULT;
 
   /**
    * Sets the maximum memory to batch before writing. The smaller this value, the more frequently the {@link BatchWriter} will write.<br />
    * If set to a value smaller than a single mutation, then it will {@link BatchWriter#flush()} after each added mutation. Must be non-negative.
-   * 
+   *
    * <p>
    * <b>Default:</b> 50M
-   * 
+   *
    * @param maxMemory
    *          max size in bytes
    * @throws IllegalArgumentException
@@ -72,15 +72,15 @@ public class BatchWriterConfig implements Writable {
   /**
    * Sets the maximum amount of time to hold the data in memory before flushing it to servers.<br />
    * For no maximum, set to zero, or {@link Long#MAX_VALUE} with {@link TimeUnit#MILLISECONDS}.
-   * 
+   *
    * <p>
    * {@link TimeUnit#MICROSECONDS} or {@link TimeUnit#NANOSECONDS} will be truncated to the nearest {@link TimeUnit#MILLISECONDS}.<br />
    * If this truncation would result in making the value zero when it was specified as non-zero, then a minimum value of one {@link TimeUnit#MILLISECONDS} will
    * be used.
-   * 
+   *
    * <p>
    * <b>Default:</b> 120 seconds
-   * 
+   *
    * @param maxLatency
    *          the maximum latency, in the unit specified by the value of {@code timeUnit}
    * @param timeUnit
@@ -104,15 +104,15 @@ public class BatchWriterConfig implements Writable {
   /**
    * Sets the maximum amount of time an unresponsive server will be re-tried. When this timeout is exceeded, the {@link BatchWriter} should throw an exception.<br />
    * For no timeout, set to zero, or {@link Long#MAX_VALUE} with {@link TimeUnit#MILLISECONDS}.
-   * 
+   *
    * <p>
    * {@link TimeUnit#MICROSECONDS} or {@link TimeUnit#NANOSECONDS} will be truncated to the nearest {@link TimeUnit#MILLISECONDS}.<br />
    * If this truncation would result in making the value zero when it was specified as non-zero, then a minimum value of one {@link TimeUnit#MILLISECONDS} will
    * be used.
-   * 
+   *
    * <p>
    * <b>Default:</b> {@link Long#MAX_VALUE} (no timeout)
-   * 
+   *
    * @param timeout
    *          the timeout, in the unit specified by the value of {@code timeUnit}
    * @param timeUnit
@@ -135,10 +135,10 @@ public class BatchWriterConfig implements Writable {
 
   /**
    * Sets the maximum number of threads to use for writing data to the tablet servers.
-   * 
+   *
    * <p>
    * <b>Default:</b> 3
-   * 
+   *
    * @param maxWriteThreads
    *          the maximum threads to use
    * @throws IllegalArgumentException
@@ -168,7 +168,7 @@ public class BatchWriterConfig implements Writable {
   public int getMaxWriteThreads() {
     return maxWriteThreads != null ? maxWriteThreads : DEFAULT_MAX_WRITE_THREADS;
   }
-  
+
   /**
    * @since 1.7.0
    * @return the durability to be used by the BatchWriter
@@ -176,16 +176,16 @@ public class BatchWriterConfig implements Writable {
   public Durability getDurability() {
     return durability;
   }
- 
+
   /**
    * @param durability the Durability to be used by the BatchWriter
    * @since 1.7.0
-   * 
+   *
    */
   public void setDurability(Durability durability) {
     this.durability = durability;
   }
- 
+
   @Override
   public void write(DataOutput out) throws IOException {
     // write this out in a human-readable way

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e0fe2ae6/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
index 7bf568f..1280abd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
@@ -23,26 +23,26 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
- * 
+ *
  * @since 1.6.0
  */
 public class ConditionalWriterConfig {
-  
+
   private static final Long DEFAULT_TIMEOUT = Long.MAX_VALUE;
   private Long timeout = null;
-  
+
   private static final Integer DEFAULT_MAX_WRITE_THREADS = 3;
   private Integer maxWriteThreads = null;
-  
+
   private Authorizations auths = Authorizations.EMPTY;
-  
+
   private Durability durability = Durability.DEFAULT;
-  
+
   /**
    * A set of authorization labels that will be checked against the column visibility of each key in order to filter data. The authorizations passed in must be
    * a subset of the accumulo user's set of authorizations. If the accumulo user has authorizations (A1, A2) and authorizations (A2, A3) are passed, then an
    * exception will be thrown.
-   * 
+   *
    * <p>
    * Any condition that is not visible with this set of authorizations will fail.
    */
@@ -51,20 +51,20 @@ public class ConditionalWriterConfig {
     this.auths = auths;
     return this;
   }
-  
+
   /**
    * Sets the maximum amount of time an unresponsive server will be re-tried. When this timeout is exceeded, the {@link ConditionalWriter} should return the
    * mutation with an exception.<br />
    * For no timeout, set to zero, or {@link Long#MAX_VALUE} with {@link TimeUnit#MILLISECONDS}.
-   * 
+   *
    * <p>
    * {@link TimeUnit#MICROSECONDS} or {@link TimeUnit#NANOSECONDS} will be truncated to the nearest {@link TimeUnit#MILLISECONDS}.<br />
    * If this truncation would result in making the value zero when it was specified as non-zero, then a minimum value of one {@link TimeUnit#MILLISECONDS} will
    * be used.
-   * 
+   *
    * <p>
    * <b>Default:</b> {@link Long#MAX_VALUE} (no timeout)
-   * 
+   *
    * @param timeout
    *          the timeout, in the unit specified by the value of {@code timeUnit}
    * @param timeUnit
@@ -76,7 +76,7 @@ public class ConditionalWriterConfig {
   public ConditionalWriterConfig setTimeout(long timeout, TimeUnit timeUnit) {
     if (timeout < 0)
       throw new IllegalArgumentException("Negative timeout not allowed " + timeout);
-    
+
     if (timeout == 0)
       this.timeout = Long.MAX_VALUE;
     else
@@ -84,13 +84,13 @@ public class ConditionalWriterConfig {
       this.timeout = Math.max(1, timeUnit.toMillis(timeout));
     return this;
   }
-  
+
   /**
    * Sets the maximum number of threads to use for writing data to the tablet servers.
-   * 
+   *
    * <p>
    * <b>Default:</b> 3
-   * 
+   *
    * @param maxWriteThreads
    *          the maximum threads to use
    * @throws IllegalArgumentException
@@ -100,11 +100,11 @@ public class ConditionalWriterConfig {
   public ConditionalWriterConfig setMaxWriteThreads(int maxWriteThreads) {
     if (maxWriteThreads <= 0)
       throw new IllegalArgumentException("Max threads must be positive " + maxWriteThreads);
-    
+
     this.maxWriteThreads = maxWriteThreads;
     return this;
   }
-  
+
   /**
    * Sets the Durability for the mutation, if applied.
    * <p>
@@ -117,7 +117,7 @@ public class ConditionalWriterConfig {
     this.durability = durability;
     return this;
   }
-  
+
   public Authorizations getAuthorizations() {
     return auths;
   }
@@ -125,11 +125,11 @@ public class ConditionalWriterConfig {
   public long getTimeout(TimeUnit timeUnit) {
     return timeUnit.convert(timeout != null ? timeout : DEFAULT_TIMEOUT, TimeUnit.MILLISECONDS);
   }
-  
+
   public int getMaxWriteThreads() {
     return maxWriteThreads != null ? maxWriteThreads : DEFAULT_MAX_WRITE_THREADS;
   }
-  
+
   public Durability getDurability() {
     return durability;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e0fe2ae6/core/src/main/java/org/apache/accumulo/core/client/Durability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Durability.java b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
index 8f85aa2..e1dbf4a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Durability.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
@@ -4,7 +4,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 
 /**
  * The value for the durability of a BatchWriter or ConditionalWriter.
- * @since 1.7.0 
+ * @since 1.7.0
  */
 public enum Durability {
   // Note, the order of these is important; the "highest" Durability is used in group commits.
@@ -44,7 +44,7 @@ public enum Durability {
         return TDurability.NONE;
     }
   }
-  
+
   // for internal use only
   static public Durability fromString(String value) {
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e0fe2ae6/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 57de347..8a9c510 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -261,7 +261,7 @@ public class TabletServer implements Runnable {
   private ReplicationWorker replWorker = null;
   private final TabletStatsKeeper statsKeeper;
   private final AtomicInteger logIdGenerator = new AtomicInteger();
-  
+
   private final VolumeManager fs;
   public Instance getInstance() {
     return serverConfig.getInstance();
@@ -297,7 +297,7 @@ public class TabletServer implements Runnable {
   private String lockID;
 
   public static final AtomicLong seekCount = new AtomicLong(0);
-  
+
   private final AtomicLong totalMinorCompactions = new AtomicLong(0);
 
   public TabletServer(ServerConfigurationFactory conf, VolumeManager fs) {
@@ -739,7 +739,7 @@ public class TabletServer implements Runnable {
         sessionManager.unreserveSession(us);
       }
     }
-    
+
     private void flush(UpdateSession us) {
 
       int mutationCount = 0;
@@ -860,8 +860,8 @@ public class TabletServer implements Runnable {
 
             if (us.currentTablet != null && extent == us.currentTablet.getExtent()) {
               // because constraint violations may filter out some
-              // mutations, for proper accounting with the client code, 
-              // need to increment the count based on the original 
+              // mutations, for proper accounting with the client code,
+              // need to increment the count based on the original
               // number of mutations from the client NOT the filtered number
               us.successfulCommits.increment(us.currentTablet, us.queuedMutations.get(us.currentTablet).size());
             }
@@ -1101,7 +1101,7 @@ public class TabletServer implements Runnable {
               if (mutations.size() > 0) {
 
                 CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, sess.credentials), mutations);
-                
+
                 if (cs == null) {
                   for (ServerConditionalMutation scm : entry.getValue())
                     results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e0fe2ae6/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
index d777ee4..b0d0b23 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -5,9 +5,14 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriter.Status;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
@@ -66,7 +71,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
   private void writeSome(String tableName, int n, BatchWriterConfig cfg) throws Exception {
     Connector c = getConnector();
     BatchWriter bw = c.createBatchWriter(tableName, cfg);
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < n; i++) {
       Mutation m = new Mutation(i + "");
       m.put("", "", "");
       bw.addMutation(m);
@@ -74,6 +79,52 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     bw.close();
   }
   
+  @Test(timeout = 3 * 60 * 1000)
+  public void testConditionDurability() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    // table default is durable writes
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "sync");
+    // write without durability
+    ConditionalWriterConfig cfg = new ConditionalWriterConfig();
+    cfg.setDurability(Durability.NONE);
+    conditionWriteSome(tableName, 10, cfg);
+    // everything in there?
+    assertEquals(10, count(tableName));
+    // restart the server and verify the updates are lost
+    restartTServer();
+    assertEquals(0, count(tableName));
+  }
+  
+  @Test(timeout = 3 * 60 * 1000)
+  public void testConditionDurability2() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    // table default is durable writes
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "none");
+    // write with durability
+    ConditionalWriterConfig cfg = new ConditionalWriterConfig();
+    cfg.setDurability(Durability.SYNC);
+    conditionWriteSome(tableName, 10, cfg);
+    // everything in there?
+    assertEquals(10, count(tableName));
+    // restart the server and verify the updates are still there
+    restartTServer();
+    assertEquals(10, count(tableName));
+  }
+  
+  private void conditionWriteSome(String tableName, int n, ConditionalWriterConfig cfg) throws Exception {
+    Connector c = getConnector();
+    ConditionalWriter cw = c.createConditionalWriter(tableName, cfg);
+    for (int i = 0; i < n; i++) {
+      ConditionalMutation m = new ConditionalMutation((CharSequence)(i + ""), new Condition("", ""));
+      m.put("", "", "X");
+      assertEquals(Status.ACCEPTED, cw.write(m).getStatus());
+    }
+  }
+  
   private void restartTServer() throws Exception {
     for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
       cluster.killProcess(ServerType.TABLET_SERVER, proc);


[11/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java b/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
new file mode 100644
index 0000000..62e5b64
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
@@ -0,0 +1,1575 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.iterators.DevNull;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.user.SummingCombiner;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
+import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
+import org.apache.accumulo.proxy.thrift.ActiveCompaction;
+import org.apache.accumulo.proxy.thrift.ActiveScan;
+import org.apache.accumulo.proxy.thrift.BatchScanOptions;
+import org.apache.accumulo.proxy.thrift.Column;
+import org.apache.accumulo.proxy.thrift.ColumnUpdate;
+import org.apache.accumulo.proxy.thrift.CompactionReason;
+import org.apache.accumulo.proxy.thrift.CompactionType;
+import org.apache.accumulo.proxy.thrift.Condition;
+import org.apache.accumulo.proxy.thrift.ConditionalStatus;
+import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
+import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
+import org.apache.accumulo.proxy.thrift.DiskUsage;
+import org.apache.accumulo.proxy.thrift.IteratorScope;
+import org.apache.accumulo.proxy.thrift.IteratorSetting;
+import org.apache.accumulo.proxy.thrift.Key;
+import org.apache.accumulo.proxy.thrift.KeyValue;
+import org.apache.accumulo.proxy.thrift.MutationsRejectedException;
+import org.apache.accumulo.proxy.thrift.PartialKey;
+import org.apache.accumulo.proxy.thrift.Range;
+import org.apache.accumulo.proxy.thrift.ScanColumn;
+import org.apache.accumulo.proxy.thrift.ScanOptions;
+import org.apache.accumulo.proxy.thrift.ScanResult;
+import org.apache.accumulo.proxy.thrift.ScanState;
+import org.apache.accumulo.proxy.thrift.ScanType;
+import org.apache.accumulo.proxy.thrift.SystemPermission;
+import org.apache.accumulo.proxy.thrift.TableExistsException;
+import org.apache.accumulo.proxy.thrift.TableNotFoundException;
+import org.apache.accumulo.proxy.thrift.TablePermission;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.accumulo.proxy.thrift.UnknownScanner;
+import org.apache.accumulo.proxy.thrift.UnknownWriter;
+import org.apache.accumulo.proxy.thrift.WriterOptions;
+import org.apache.accumulo.server.util.PortUtils;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.TServer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+/**
+ * Call every method on the proxy and try to verify that it works.
+ */
+public class SimpleProxyIT {
+
+  public static File macTestFolder = new File(System.getProperty("user.dir") + "/target/" + SimpleProxyIT.class.getName());
+
+  private static MiniAccumuloCluster accumulo;
+  private static String secret = "superSecret";
+  private static Random random = new Random();
+  private static TServer proxyServer;
+  private static Thread thread;
+  private static int proxyPort;
+  private static org.apache.accumulo.proxy.thrift.AccumuloProxy.Client client;
+  private static String principal = "root";
+
+  private static Map<String,String> properties = new TreeMap<String,String>() {
+    private static final long serialVersionUID = 1L;
+
+    {
+      put("password", secret);
+    }
+  };
+  private static ByteBuffer creds = null;
+
+  private static Class<? extends TProtocolFactory> protocolClass;
+
+  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);
+    protocolFactories.add(org.apache.thrift.protocol.TTupleProtocol.Factory.class);
+    protocolFactories.add(org.apache.thrift.protocol.TCompactProtocol.Factory.class);
+
+    return protocolFactories.get(random.nextInt(protocolFactories.size()));
+  }
+
+  private static final AtomicInteger tableCounter = new AtomicInteger(0);
+
+  private static String makeTableName() {
+    return "test" + tableCounter.getAndIncrement();
+  }
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder(macTestFolder);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @BeforeClass
+  public static void setupMiniCluster() throws Exception {
+    FileUtils.deleteQuietly(macTestFolder);
+    macTestFolder.mkdirs();
+    MiniAccumuloConfig config = new MiniAccumuloConfig(macTestFolder, secret).setNumTservers(1);
+    accumulo = new MiniAccumuloCluster(config);
+    accumulo.start();
+    // wait for accumulo to be up and functional
+    ZooKeeperInstance zoo = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers());
+    Connector c = zoo.getConnector("root", new PasswordToken(secret.getBytes()));
+    for (@SuppressWarnings("unused") Entry<org.apache.accumulo.core.data.Key,Value> entry : c.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
+        ;
+
+    Properties props = new Properties();
+    props.put("instance", accumulo.getConfig().getInstanceName());
+    props.put("zookeepers", accumulo.getZooKeepers());
+    props.put("tokenClass", PasswordToken.class.getName());
+
+    protocolClass = getRandomProtocol();
+
+    proxyPort = PortUtils.getRandomFreePort();
+    proxyServer = Proxy.createProxyServer(org.apache.accumulo.proxy.thrift.AccumuloProxy.class, org.apache.accumulo.proxy.ProxyServer.class, proxyPort,
+        protocolClass, props);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxyServer.serve();
+      }
+    };
+    thread.start();
+    while (!proxyServer.isServing())
+      UtilWaitThread.sleep(100);
+    client = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
+    creds = client.login(principal, properties);
+  }
+
+  @Test(timeout = 10000)
+  public void security() throws Exception {
+    client.createLocalUser(creds, "user", s2bb(secret));
+    ByteBuffer badLogin = client.login("user", properties);
+    client.dropLocalUser(creds, "user");
+    final String table = makeTableName();
+    client.createTable(creds, table, false, TimeType.MILLIS);
+
+    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
+
+    try {
+      client.addConstraint(badLogin, table, NumericValueConstraint.class.getName());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.addSplits(badLogin, table, Collections.singleton(s2bb("1")));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.clearLocatorCache(badLogin, table);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.compactTable(badLogin, table, null, null, null, true, false);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.cancelCompaction(badLogin, table);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createTable(badLogin, table, false, TimeType.MILLIS);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.deleteTable(badLogin, table);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.deleteRows(badLogin, table, null, null);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.tableExists(badLogin, table);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.flushTable(badLogin, table, null, null, false);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getLocalityGroups(badLogin, table);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getMaxRow(badLogin, table, Collections.<ByteBuffer> emptySet(), null, false, null, false);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getTableProperties(badLogin, table);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.listSplits(badLogin, table, 10000);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.listTables(badLogin);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.listConstraints(badLogin, table);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.mergeTablets(badLogin, table, null, null);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.offlineTable(badLogin, table, false);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.onlineTable(badLogin, table, false);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.removeConstraint(badLogin, table, 0);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.removeTableProperty(badLogin, table, Property.TABLE_FILE_MAX.getKey());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.renameTable(badLogin, table, "someTableName");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
+      groups.put("group1", Collections.singleton("cf1"));
+      groups.put("group2", Collections.singleton("cf2"));
+      client.setLocalityGroups(badLogin, table, groups);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.setTableProperty(badLogin, table, Property.TABLE_FILE_MAX.getKey(), "0");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.tableIdMap(badLogin);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.getSiteConfiguration(badLogin);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getSystemConfiguration(badLogin);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getTabletServers(badLogin);
+      fail("exception not thrown");
+    } catch (TException ex) {}
+    try {
+      client.getActiveScans(badLogin, "fake");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getActiveCompactions(badLogin, "fakse");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.removeProperty(badLogin, "table.split.threshold");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.setProperty(badLogin, "table.split.threshold", "500M");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.testClassLoad(badLogin, DevNull.class.getName(), SortedKeyValueIterator.class.getName());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.authenticateUser(badLogin, "root", s2pp(secret));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      HashSet<ByteBuffer> auths = new HashSet<ByteBuffer>(Arrays.asList(s2bb("A"), s2bb("B")));
+      client.changeUserAuthorizations(badLogin, "stooge", auths);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.changeLocalUserPassword(badLogin, "stooge", s2bb(""));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createLocalUser(badLogin, "stooge", s2bb("password"));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.dropLocalUser(badLogin, "stooge");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getUserAuthorizations(badLogin, "stooge");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.grantSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.grantTablePermission(badLogin, "root", table, TablePermission.WRITE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.hasSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.hasTablePermission(badLogin, "root", table, TablePermission.WRITE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.listLocalUsers(badLogin);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.revokeSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.revokeTablePermission(badLogin, "root", table, TablePermission.ALTER_TABLE);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createScanner(badLogin, table, new ScanOptions());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createBatchScanner(badLogin, table, new BatchScanOptions());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.updateAndFlush(badLogin, table, new HashMap<ByteBuffer,List<ColumnUpdate>>());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createWriter(badLogin, table, new WriterOptions());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.attachIterator(badLogin, "slow", setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.checkIteratorConflicts(badLogin, table, setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      final String TABLE_TEST = makeTableName();
+      client.cloneTable(badLogin, table, TABLE_TEST, false, null, null);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.exportTable(badLogin, table, "/tmp");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.importTable(badLogin, "testify", "/tmp");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.getIteratorSetting(badLogin, table, "foo", IteratorScope.SCAN);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.listIterators(badLogin, table);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.removeIterator(badLogin, table, "name", EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.splitRangeByTablets(badLogin, table, client.getRowRange(ByteBuffer.wrap("row".getBytes())), 10);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      File importDir = tempFolder.newFolder("importDir");
+      File failuresDir = tempFolder.newFolder("failuresDir");
+      client.importDirectory(badLogin, table, importDir.getAbsolutePath(), failuresDir.getAbsolutePath(), true);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.pingTabletServer(badLogin, "fake");
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.login("badUser", properties);
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.testTableClassLoad(badLogin, table, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+    try {
+      client.createConditionalWriter(badLogin, table, new ConditionalWriterOptions());
+      fail("exception not thrown");
+    } catch (AccumuloSecurityException ex) {}
+  }
+
+  @Test(timeout = 10000)
+  public void tableNotFound() throws Exception {
+    final String doesNotExist = "doesNotExists";
+    try {
+      client.addConstraint(creds, doesNotExist, NumericValueConstraint.class.getName());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.addSplits(creds, doesNotExist, Collections.<ByteBuffer> emptySet());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
+    try {
+      client.attachIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.cancelCompaction(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.checkIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.clearLocatorCache(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      final String TABLE_TEST = makeTableName();
+      client.cloneTable(creds, doesNotExist, TABLE_TEST, false, null, null);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.compactTable(creds, doesNotExist, null, null, null, true, false);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.createBatchScanner(creds, doesNotExist, new BatchScanOptions());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.createScanner(creds, doesNotExist, new ScanOptions());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.createWriter(creds, doesNotExist, new WriterOptions());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.deleteRows(creds, doesNotExist, null, null);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.deleteTable(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.exportTable(creds, doesNotExist, "/tmp");
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.flushTable(creds, doesNotExist, null, null, false);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.getIteratorSetting(creds, doesNotExist, "foo", IteratorScope.SCAN);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.getLocalityGroups(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.getMaxRow(creds, doesNotExist, Collections.<ByteBuffer> emptySet(), null, false, null, false);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.getTableProperties(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.grantTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.hasTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      File importDir = tempFolder.newFolder("importDir");
+      File failuresDir = tempFolder.newFolder("failuresDir");
+      client.importDirectory(creds, doesNotExist, importDir.getAbsolutePath(), failuresDir.getAbsolutePath(), true);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.listConstraints(creds, doesNotExist);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.listSplits(creds, doesNotExist, 10000);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.mergeTablets(creds, doesNotExist, null, null);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.offlineTable(creds, doesNotExist, false);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.onlineTable(creds, doesNotExist, false);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.removeConstraint(creds, doesNotExist, 0);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.removeIterator(creds, doesNotExist, "name", EnumSet.allOf(IteratorScope.class));
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.removeTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.renameTable(creds, doesNotExist, "someTableName");
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.revokeTablePermission(creds, "root", doesNotExist, TablePermission.ALTER_TABLE);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.setTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey(), "0");
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.splitRangeByTablets(creds, doesNotExist, client.getRowRange(ByteBuffer.wrap("row".getBytes())), 10);
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.updateAndFlush(creds, doesNotExist, new HashMap<ByteBuffer,List<ColumnUpdate>>());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.getDiskUsage(creds, Collections.singleton(doesNotExist));
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.testTableClassLoad(creds, doesNotExist, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
+      fail("exception not thrown");
+    } catch (TableNotFoundException ex) {}
+    try {
+      client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
+    } catch (TableNotFoundException ex) {}
+  }
+  
+  @Test(timeout = 10 * 1000)
+  public void testExists() throws Exception {
+    client.createTable(creds, "ett1", false, TimeType.MILLIS);
+    client.createTable(creds, "ett2", false, TimeType.MILLIS);
+    try {
+      client.createTable(creds, "ett1", false, TimeType.MILLIS);
+      fail("exception not thrown");
+    } catch (TableExistsException tee) {}
+    try {
+      client.renameTable(creds, "ett1", "ett2");
+      fail("exception not thrown");
+    } catch (TableExistsException tee) {}
+    try {
+      client.cloneTable(creds, "ett1", "ett2", false, new HashMap<String,String>(), new HashSet<String>());
+      fail("exception not thrown");
+    } catch (TableExistsException tee) {}
+  }
+
+  @Test(timeout = 10000)
+  public void testUnknownScanner() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+
+    String scanner = client.createScanner(creds, TABLE_TEST, null);
+    assertFalse(client.hasNext(scanner));
+    client.closeScanner(scanner);
+
+    try {
+      client.hasNext(scanner);
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+
+    try {
+      client.closeScanner(scanner);
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+
+    try {
+      client.nextEntry("99999999");
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+    try {
+      client.nextK("99999999", 6);
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+    try {
+      client.hasNext("99999999");
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+    try {
+      client.hasNext(UUID.randomUUID().toString());
+      fail("exception not thrown");
+    } catch (UnknownScanner us) {}
+  }
+
+  @Test(timeout = 10000)
+  public void testUnknownWriter() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+
+    String writer = client.createWriter(creds, TABLE_TEST, null);
+    client.update(writer, mutation("row0", "cf", "cq", "value"));
+    client.flush(writer);
+    client.update(writer, mutation("row2", "cf", "cq", "value2"));
+    client.closeWriter(writer);
+
+    // this is a oneway call, so it does not throw exceptions
+    client.update(writer, mutation("row2", "cf", "cq", "value2"));
+
+    try {
+      client.flush(writer);
+      fail("exception not thrown");
+    } catch (UnknownWriter uw) {}
+    try {
+      client.flush("99999");
+      fail("exception not thrown");
+    } catch (UnknownWriter uw) {}
+    try {
+      client.flush(UUID.randomUUID().toString());
+      fail("exception not thrown");
+    } catch (UnknownWriter uw) {}
+    try {
+      client.closeWriter("99999");
+      fail("exception not thrown");
+    } catch (UnknownWriter uw) {}
+  }
+
+  @Test(timeout = 10000)
+  public void testDelete() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+    client.updateAndFlush(creds, TABLE_TEST, mutation("row0", "cf", "cq", "value"));
+
+    assertScan(new String[][] {{"row0", "cf", "cq", "value"}}, TABLE_TEST);
+
+    ColumnUpdate upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
+    upd.setDeleteCell(true);
+    Map<ByteBuffer,List<ColumnUpdate>> delete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
+
+    client.updateAndFlush(creds, TABLE_TEST, delete);
+
+    assertScan(new String[][] {}, TABLE_TEST);
+  }
+
+  @Test(timeout = 60000)
+  public void testInstanceOperations() throws Exception {
+    int tservers = 0;
+    for (String tserver : client.getTabletServers(creds)) {
+      client.pingTabletServer(creds, tserver);
+      tservers++;
+    }
+    assertTrue(tservers > 0);
+
+    // get something we know is in the site config
+    Map<String,String> cfg = client.getSiteConfiguration(creds);
+    assertTrue(cfg.get("instance.dfs.dir").startsWith(macTestFolder.getPath()));
+
+    // set a property in zookeeper
+    client.setProperty(creds, "table.split.threshold", "500M");
+
+    // check that we can read it
+    for (int i = 0; i < 5; i++) {
+      cfg = client.getSystemConfiguration(creds);
+      if ("500M".equals(cfg.get("table.split.threshold")))
+        break;
+      UtilWaitThread.sleep(200);
+    }
+    assertEquals("500M", cfg.get("table.split.threshold"));
+
+    // unset the setting, check that it's not what it was
+    client.removeProperty(creds, "table.split.threshold");
+    for (int i = 0; i < 5; i++) {
+      cfg = client.getSystemConfiguration(creds);
+      if (!"500M".equals(cfg.get("table.split.threshold")))
+        break;
+      UtilWaitThread.sleep(200);
+    }
+    assertNotEquals("500M", cfg.get("table.split.threshold"));
+
+    // try to load some classes via the proxy
+    assertTrue(client.testClassLoad(creds, DevNull.class.getName(), SortedKeyValueIterator.class.getName()));
+    assertFalse(client.testClassLoad(creds, "foo.bar", SortedKeyValueIterator.class.getName()));
+
+    // create a table that's very slow, so we can look for scans/compactions
+    client.createTable(creds, "slow", true, TimeType.MILLIS);
+    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
+    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
+
+    // Should take 10 seconds to read every record
+    for (int i = 0; i < 40; i++) {
+      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
+    }
+
+    // scan
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        String scanner;
+        try {
+          Client client2 = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
+          scanner = client2.createScanner(creds, "slow", null);
+          client2.nextK(scanner, 10);
+          client2.closeScanner(scanner);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+    t.start();
+
+    // look for the scan many times 
+    List<ActiveScan> scans = new ArrayList<ActiveScan>();
+    for (int i = 0; i < 100 && scans.isEmpty(); i++) {
+      for (String tserver : client.getTabletServers(creds)) {
+        List<ActiveScan> scansForServer = client.getActiveScans(creds, tserver);
+        for (ActiveScan scan : scansForServer) {
+          if ("root".equals(scan.getUser())) {
+            scans.add(scan);
+          }
+        }
+
+        if (!scans.isEmpty())
+          break;
+        UtilWaitThread.sleep(100);
+      }
+    }
+    t.join();
+
+    assertFalse(scans.isEmpty());
+    boolean found = false;
+    Map<String,String> map = null;
+    for (int i = 0; i < scans.size() && !found; i++) {
+      ActiveScan scan = scans.get(i);
+      if ("root".equals(scan.getUser())) {
+        assertTrue(ScanState.RUNNING.equals(scan.getState()) || ScanState.QUEUED.equals(scan.getState()));
+        assertEquals(ScanType.SINGLE, scan.getType());
+        assertEquals("slow", scan.getTable());
+
+        map = client.tableIdMap(creds);
+        assertEquals(map.get("slow"), scan.getExtent().tableId);
+        assertTrue(scan.getExtent().endRow == null);
+        assertTrue(scan.getExtent().prevEndRow == null);
+        found = true;
+      }
+    }
+
+    assertTrue("Could not find a scan against the 'slow' table", found);
+
+    // start a compaction
+    t = new Thread() {
+      @Override
+      public void run() {
+        try {
+          Client client2 = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
+          client2.compactTable(creds, "slow", null, null, null, true, true);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+    t.start();
+
+    final String desiredTableId = map.get("slow");
+
+    // try to catch it in the act
+    List<ActiveCompaction> compactions = new ArrayList<ActiveCompaction>();
+    for (int i = 0; i < 100 && compactions.isEmpty(); i++) {
+      // Iterate over the tservers
+      for (String tserver : client.getTabletServers(creds)) {
+        // And get the compactions on each
+        List<ActiveCompaction> compactionsOnServer = client.getActiveCompactions(creds, tserver);
+        for (ActiveCompaction compact : compactionsOnServer) {
+          // There might be other compactions occurring (e.g. on METADATA) in which
+          // case we want to prune out those that aren't for our slow table
+          if (desiredTableId.equals(compact.getExtent().tableId)) {
+            compactions.add(compact);
+          }
+        }
+
+        // If we found a compaction for the table we wanted, so we can stop looking
+        if (!compactions.isEmpty())
+          break;
+      }
+      UtilWaitThread.sleep(10);
+    }
+    t.join();
+
+    // verify the compaction information
+    assertFalse(compactions.isEmpty());
+    for (ActiveCompaction c : compactions) {
+      if (desiredTableId.equals(c.getExtent().tableId)) {
+        assertTrue(c.inputFiles.isEmpty());
+        assertEquals(CompactionType.MINOR, c.getType());
+        assertEquals(CompactionReason.USER, c.getReason());
+        assertEquals("", c.localityGroup);
+        assertTrue(c.outputFile.contains("default_tablet"));
+
+        return;
+      }
+    }
+    fail("Expection to find running compaction for table 'slow' but did not find one");
+  }
+
+  @Test
+  public void testSecurityOperations() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    // check password
+    assertTrue(client.authenticateUser(creds, "root", s2pp(secret)));
+    assertFalse(client.authenticateUser(creds, "root", s2pp("")));
+
+    // create a user
+    client.createLocalUser(creds, "stooge", s2bb("password"));
+    // change auths
+    Set<String> users = client.listLocalUsers(creds);
+    assertEquals(new HashSet<String>(Arrays.asList("root", "stooge")), users);
+    HashSet<ByteBuffer> auths = new HashSet<ByteBuffer>(Arrays.asList(s2bb("A"), s2bb("B")));
+    client.changeUserAuthorizations(creds, "stooge", auths);
+    List<ByteBuffer> update = client.getUserAuthorizations(creds, "stooge");
+    assertEquals(auths, new HashSet<ByteBuffer>(update));
+
+    // change password
+    client.changeLocalUserPassword(creds, "stooge", s2bb(""));
+    assertTrue(client.authenticateUser(creds, "stooge", s2pp("")));
+
+    // check permission failure
+    @SuppressWarnings("serial")
+    ByteBuffer stooge = client.login("stooge", new TreeMap<String,String>() {
+      {
+        put("password", "");
+      }
+    });
+
+    try {
+      client.createTable(stooge, "fail", true, TimeType.MILLIS);
+      fail("should not create the table");
+    } catch (AccumuloSecurityException ex) {
+      assertFalse(client.listTables(creds).contains("fail"));
+    }
+    // grant permissions and test
+    assertFalse(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
+    client.grantSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE);
+    assertTrue(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
+    client.createTable(stooge, "success", true, TimeType.MILLIS);
+    client.listTables(creds).contains("succcess");
+
+    // revoke permissions
+    client.revokeSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE);
+    assertFalse(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
+    try {
+      client.createTable(stooge, "fail", true, TimeType.MILLIS);
+      fail("should not create the table");
+    } catch (AccumuloSecurityException ex) {
+      assertFalse(client.listTables(creds).contains("fail"));
+    }
+    // create a table to test table permissions
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+    // denied!
+    try {
+      String scanner = client.createScanner(stooge, TABLE_TEST, null);
+      client.nextK(scanner, 100);
+      fail("stooge should not read table test");
+    } catch (AccumuloSecurityException ex) {}
+    // grant
+    assertFalse(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
+    client.grantTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ);
+    assertTrue(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
+    String scanner = client.createScanner(stooge, TABLE_TEST, null);
+    client.nextK(scanner, 10);
+    client.closeScanner(scanner);
+    // revoke
+    client.revokeTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ);
+    assertFalse(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
+    try {
+      scanner = client.createScanner(stooge, TABLE_TEST, null);
+      client.nextK(scanner, 100);
+      fail("stooge should not read table test");
+    } catch (AccumuloSecurityException ex) {}
+
+    // delete user
+    client.dropLocalUser(creds, "stooge");
+    users = client.listLocalUsers(creds);
+    assertEquals(1, users.size());
+
+  }
+
+  @Test
+  public void testBatchWriter() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
+
+    WriterOptions writerOptions = new WriterOptions();
+    writerOptions.setLatencyMs(10000);
+    writerOptions.setMaxMemory(2);
+    writerOptions.setThreads(1);
+    writerOptions.setTimeoutMs(100000);
+
+    String batchWriter = client.createWriter(creds, TABLE_TEST, writerOptions);
+    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
+    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
+    try {
+      client.flush(batchWriter);
+      fail("constraint did not fire");
+    } catch (MutationsRejectedException ex) {}
+    try {
+      client.closeWriter(batchWriter);
+      fail("constraint did not fire");
+    } catch (MutationsRejectedException e) {}
+
+    client.removeConstraint(creds, TABLE_TEST, 2);
+
+    assertScan(new String[][] {}, TABLE_TEST);
+
+    UtilWaitThread.sleep(2000);
+    
+    writerOptions = new WriterOptions();
+    writerOptions.setLatencyMs(10000);
+    writerOptions.setMaxMemory(3000);
+    writerOptions.setThreads(1);
+    writerOptions.setTimeoutMs(100000);
+
+    batchWriter = client.createWriter(creds, TABLE_TEST, writerOptions);
+
+    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
+    client.flush(batchWriter);
+    client.closeWriter(batchWriter);
+
+    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, TABLE_TEST);
+
+    client.deleteTable(creds, TABLE_TEST);
+  }
+
+  @Test
+  public void testTableOperations() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+    // constraints
+    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
+    assertEquals(2, client.listConstraints(creds, TABLE_TEST).size());
+
+    UtilWaitThread.sleep(2000);
+
+    client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "123"));
+
+    try {
+      client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "x"));
+      fail("constraint did not fire");
+    } catch (MutationsRejectedException ex) {}
+
+    client.removeConstraint(creds, TABLE_TEST, 2);
+
+    UtilWaitThread.sleep(2000);
+
+    assertEquals(1, client.listConstraints(creds, TABLE_TEST).size());
+
+    client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "x"));
+    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, TABLE_TEST);
+    // splits, merge
+    client.addSplits(creds, TABLE_TEST, new HashSet<ByteBuffer>(Arrays.asList(s2bb("a"), s2bb("m"), s2bb("z"))));
+    List<ByteBuffer> splits = client.listSplits(creds, TABLE_TEST, 1);
+    assertEquals(Arrays.asList(s2bb("m")), splits);
+    client.mergeTablets(creds, TABLE_TEST, null, s2bb("m"));
+    splits = client.listSplits(creds, TABLE_TEST, 10);
+    assertEquals(Arrays.asList(s2bb("m"), s2bb("z")), splits);
+    client.mergeTablets(creds, TABLE_TEST, null, null);
+    splits = client.listSplits(creds, TABLE_TEST, 10);
+    List<ByteBuffer> empty = Collections.emptyList();
+    assertEquals(empty, splits);
+    // iterators
+    client.deleteTable(creds, TABLE_TEST);
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+    HashMap<String,String> options = new HashMap<String,String>();
+    options.put("type", "STRING");
+    options.put("columns", "cf");
+    IteratorSetting setting = new IteratorSetting(10, TABLE_TEST, SummingCombiner.class.getName(), options);
+    client.attachIterator(creds, TABLE_TEST, setting, EnumSet.allOf(IteratorScope.class));
+    for (int i = 0; i < 10; i++) {
+      client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "1"));
+    }
+    assertScan(new String[][] {{"row1", "cf", "cq", "10"}}, TABLE_TEST);
+    try {
+      client.checkIteratorConflicts(creds, TABLE_TEST, setting, EnumSet.allOf(IteratorScope.class));
+      fail("checkIteratorConflicts did not throw an exception");
+    } catch (Exception ex) {}
+    client.deleteRows(creds, TABLE_TEST, null, null);
+    client.removeIterator(creds, TABLE_TEST, "test", EnumSet.allOf(IteratorScope.class));
+    String expected[][] = new String[10][];
+    for (int i = 0; i < 10; i++) {
+      client.updateAndFlush(creds, TABLE_TEST, mutation("row" + i, "cf", "cq", "" + i));
+      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
+      client.flushTable(creds, TABLE_TEST, null, null, true);
+    }
+    assertScan(expected, TABLE_TEST);
+    // clone
+    final String TABLE_TEST2 = makeTableName();
+    client.cloneTable(creds, TABLE_TEST, TABLE_TEST2, true, null, null);
+    assertScan(expected, TABLE_TEST2);
+    client.deleteTable(creds, TABLE_TEST2);
+
+    // don't know how to test this, call it just for fun
+    client.clearLocatorCache(creds, TABLE_TEST);
+
+    // compact
+    client.compactTable(creds, TABLE_TEST, null, null, null, true, true);
+    assertEquals(1, countFiles(TABLE_TEST));
+    assertScan(expected, TABLE_TEST);
+
+    // get disk usage
+    client.cloneTable(creds, TABLE_TEST, TABLE_TEST2, true, null, null);
+    Set<String> tablesToScan = new HashSet<String>();
+    tablesToScan.add(TABLE_TEST);
+    tablesToScan.add(TABLE_TEST2);
+    tablesToScan.add("foo");
+    client.createTable(creds, "foo", true, TimeType.MILLIS);
+    List<DiskUsage> diskUsage = (client.getDiskUsage(creds, tablesToScan));
+    assertEquals(2, diskUsage.size());
+    assertEquals(1, diskUsage.get(0).getTables().size());
+    assertEquals(2, diskUsage.get(1).getTables().size());
+    client.compactTable(creds, TABLE_TEST2, null, null, null, true, true);
+    diskUsage = (client.getDiskUsage(creds, tablesToScan));
+    assertEquals(3, diskUsage.size());
+    assertEquals(1, diskUsage.get(0).getTables().size());
+    assertEquals(1, diskUsage.get(1).getTables().size());
+    assertEquals(1, diskUsage.get(2).getTables().size());
+    client.deleteTable(creds, "foo");
+    client.deleteTable(creds, TABLE_TEST2);
+
+    // export/import
+    File dir = tempFolder.newFolder("test");
+    File destDir = tempFolder.newFolder("test_dest");
+    client.offlineTable(creds, TABLE_TEST, false);
+    client.exportTable(creds, TABLE_TEST, dir.getAbsolutePath());
+    // copy files to a new location
+    FileSystem fs = FileSystem.get(new Configuration());
+    FSDataInputStream is = fs.open(new Path(dir + "/distcp.txt"));
+    BufferedReader r = new BufferedReader(new InputStreamReader(is));
+    while (true) {
+      String line = r.readLine();
+      if (line == null)
+        break;
+      Path srcPath = new Path(line);
+      FileUtils.copyFile(new File(srcPath.toUri().getPath()), new File(destDir, srcPath.getName()));
+    }
+    client.deleteTable(creds, TABLE_TEST);
+    client.importTable(creds, "testify", destDir.getAbsolutePath());
+    assertScan(expected, "testify");
+    client.deleteTable(creds, "testify");
+
+    try {
+      // ACCUMULO-1558 a second import from the same dir should fail, the first import moved the files
+      client.importTable(creds, "testify2", destDir.getAbsolutePath());
+      fail();
+    } catch (Exception e) {}
+
+    assertFalse(client.listTables(creds).contains("testify2"));
+
+    // Locality groups
+    client.createTable(creds, "test", true, TimeType.MILLIS);
+    Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
+    groups.put("group1", Collections.singleton("cf1"));
+    groups.put("group2", Collections.singleton("cf2"));
+    client.setLocalityGroups(creds, "test", groups);
+    assertEquals(groups, client.getLocalityGroups(creds, "test"));
+    // table properties
+    Map<String,String> orig = client.getTableProperties(creds, "test");
+    client.setTableProperty(creds, "test", "table.split.threshold", "500M");
+    Map<String,String> update = client.getTableProperties(creds, "test");
+    assertEquals(update.get("table.split.threshold"), "500M");
+    client.removeTableProperty(creds, "test", "table.split.threshold");
+    update = client.getTableProperties(creds, "test");
+    assertEquals(orig, update);
+    // rename table
+    Map<String,String> tables = client.tableIdMap(creds);
+    client.renameTable(creds, "test", "bar");
+    Map<String,String> tables2 = client.tableIdMap(creds);
+    assertEquals(tables.get("test"), tables2.get("bar"));
+    // table exists
+    assertTrue(client.tableExists(creds, "bar"));
+    assertFalse(client.tableExists(creds, "test"));
+    // bulk import
+    String filename = dir + "/bulk/import/rfile.rf";
+    FileSKVWriter writer = FileOperations.getInstance().openWriter(filename, fs, fs.getConf(), DefaultConfiguration.getInstance());
+    writer.startDefaultLocalityGroup();
+    writer.append(new org.apache.accumulo.core.data.Key(new Text("a"), new Text("b"), new Text("c")), new Value("value".getBytes()));
+    writer.close();
+    fs.mkdirs(new Path(dir + "/bulk/fail"));
+    client.importDirectory(creds, "bar", dir + "/bulk/import", dir + "/bulk/fail", true);
+    String scanner = client.createScanner(creds, "bar", null);
+    ScanResult more = client.nextK(scanner, 100);
+    client.closeScanner(scanner);
+    assertEquals(1, more.results.size());
+    ByteBuffer maxRow = client.getMaxRow(creds, "bar", null, null, false, null, false);
+    assertEquals(s2bb("a"), maxRow);
+
+    assertFalse(client.testTableClassLoad(creds, "bar", "abc123", SortedKeyValueIterator.class.getName()));
+    assertTrue(client.testTableClassLoad(creds, "bar", VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName()));
+  }
+
+  private Condition newCondition(String cf, String cq) {
+    return new Condition(new Column(s2bb(cf), s2bb(cq), s2bb("")));
+  }
+
+  private Condition newCondition(String cf, String cq, String val) {
+    return newCondition(cf, cq).setValue(s2bb(val));
+  }
+
+  private Condition newCondition(String cf, String cq, long ts, String val) {
+    return newCondition(cf, cq).setValue(s2bb(val)).setTimestamp(ts);
+  }
+
+  private ColumnUpdate newColUpdate(String cf, String cq, String val) {
+    return new ColumnUpdate(s2bb(cf), s2bb(cq)).setValue(s2bb(val));
+  }
+
+  private ColumnUpdate newColUpdate(String cf, String cq, long ts, String val) {
+    return new ColumnUpdate(s2bb(cf), s2bb(cq)).setTimestamp(ts).setValue(s2bb(val));
+  }
+
+  private void assertScan(String[][] expected, String table) throws Exception {
+    String scid = client.createScanner(creds, table, new ScanOptions());
+    ScanResult keyValues = client.nextK(scid, expected.length + 1);
+
+    assertEquals(expected.length, keyValues.results.size());
+    assertFalse(keyValues.more);
+
+    for (int i = 0; i < keyValues.results.size(); i++) {
+      checkKey(expected[i][0], expected[i][1], expected[i][2], expected[i][3], keyValues.results.get(i));
+    }
+
+    client.closeScanner(scid);
+  }
+
+  @Test
+  public void testConditionalWriter() throws Exception {
+    final String TABLE_TEST = makeTableName();
+
+    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
+
+    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
+
+    String cwid = client.createConditionalWriter(creds, TABLE_TEST, new ConditionalWriterOptions());
+
+    Map<ByteBuffer,ConditionalUpdates> updates = new HashMap<ByteBuffer,ConditionalUpdates>();
+
+    updates.put(
+        s2bb("00345"),
+        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", 10, "1"),
+            newColUpdate("data", "img", "73435435"))));
+
+    Map<ByteBuffer,ConditionalStatus> results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "73435435"}, {"00345", "meta", "seq", "1"}}, TABLE_TEST);
+
+    // test not setting values on conditions
+    updates.clear();
+
+    updates.put(s2bb("00345"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", "2"))));
+    updates.put(s2bb("00346"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", "1"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(2, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00345")));
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00346")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "73435435"}, {"00345", "meta", "seq", "1"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
+
+    // test setting values on conditions
+    updates.clear();
+
+    updates.put(
+        s2bb("00345"),
+        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", "1")), Arrays.asList(newColUpdate("meta", "seq", 20, "2"),
+            newColUpdate("data", "img", "567890"))));
+
+    updates.put(s2bb("00346"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", "2")), Arrays.asList(newColUpdate("meta", "seq", "3"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(2, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00346")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "567890"}, {"00345", "meta", "seq", "2"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
+
+    // test setting timestamp on condition to a non-existant version
+    updates.clear();
+
+    updates.put(
+        s2bb("00345"),
+        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", 10, "2")), Arrays.asList(newColUpdate("meta", "seq", 30, "3"),
+            newColUpdate("data", "img", "1234567890"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00345")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "567890"}, {"00345", "meta", "seq", "2"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
+
+    // test setting timestamp to an existing version
+
+    updates.clear();
+
+    updates.put(
+        s2bb("00345"),
+        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", 20, "2")), Arrays.asList(newColUpdate("meta", "seq", 30, "3"),
+            newColUpdate("data", "img", "1234567890"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
+
+    // run test w/ condition that has iterators
+    // following should fail w/o iterator
+    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
+    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
+    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
+
+    updates.clear();
+    updates.put(s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "count", "3")), Arrays.asList(newColUpdate("data", "img", "1234567890"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}}, TABLE_TEST);
+
+    // following test w/ iterator setup should succeed
+    Condition iterCond = newCondition("data", "count", "3");
+    Map<String,String> props = new HashMap<String,String>();
+    props.put("type", "STRING");
+    props.put("columns", "data:count");
+    IteratorSetting is = new IteratorSetting(1, "sumc", SummingCombiner.class.getName(), props);
+    iterCond.setIterators(Arrays.asList(is));
+
+    updates.clear();
+    updates.put(s2bb("00347"), new ConditionalUpdates(Arrays.asList(iterCond), Arrays.asList(newColUpdate("data", "img", "1234567890"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
+
+    // test a mutation that violated a constraint
+    updates.clear();
+    updates.put(s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890")), Arrays.asList(newColUpdate("data", "count", "A"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.VIOLATED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
+
+    // run test with two conditions
+    // both conditions should fail
+    updates.clear();
+    updates.put(
+        s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "565"), newCondition("data", "count", "2")), Arrays.asList(
+            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
+
+    // one condition should fail
+    updates.clear();
+    updates.put(
+        s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890"), newCondition("data", "count", "2")), Arrays.asList(
+            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
+
+    // one condition should fail
+    updates.clear();
+    updates.put(
+        s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "565"), newCondition("data", "count", "1")), Arrays.asList(
+            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
+
+    // both conditions should succeed
+
+    ConditionalStatus result = client.updateRowConditionally(
+        creds,
+        TABLE_TEST,
+        s2bb("00347"),
+        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890"), newCondition("data", "count", "1")), Arrays.asList(
+            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
+
+    assertEquals(ConditionalStatus.ACCEPTED, result);
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}}, TABLE_TEST);
+
+    client.closeConditionalWriter(cwid);
+    try {
+      client.updateRowsConditionally(cwid, updates);
+      fail("conditional writer not closed");
+    } catch (UnknownWriter uk) {}
+
+    // run test with colvis
+    client.createLocalUser(creds, "cwuser", s2bb("bestpasswordever"));
+    client.changeUserAuthorizations(creds, "cwuser", Collections.singleton(s2bb("A")));
+    client.grantTablePermission(creds, "cwuser", TABLE_TEST, TablePermission.WRITE);
+    client.grantTablePermission(creds, "cwuser", TABLE_TEST, TablePermission.READ);
+
+    ByteBuffer cwuCreds = client.login("cwuser", Collections.singletonMap("password", "bestpasswordever"));
+
+    cwid = client.createConditionalWriter(cwuCreds, TABLE_TEST, new ConditionalWriterOptions().setAuthorizations(Collections.singleton(s2bb("A"))));
+
+    updates.clear();
+    updates.put(
+        s2bb("00348"),
+        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A")))), Arrays.asList(newColUpdate("data", "seq", "1"),
+            newColUpdate("data", "c", "1").setColVisibility(s2bb("A")))));
+    updates.put(s2bb("00349"),
+        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("B")))), Arrays.asList(newColUpdate("data", "seq", "1"))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(2, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00348")));
+    assertEquals(ConditionalStatus.INVISIBLE_VISIBILITY, results.get(s2bb("00349")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "1"}}, TABLE_TEST);
+
+    updates.clear();
+
+    updates.clear();
+    updates.put(
+        s2bb("00348"),
+        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A"))).setValue(s2bb("0"))), Arrays.asList(
+            newColUpdate("data", "seq", "2"), newColUpdate("data", "c", "2").setColVisibility(s2bb("A")))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00348")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "1"}}, TABLE_TEST);
+
+    updates.clear();
+    updates.put(
+        s2bb("00348"),
+        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A"))).setValue(s2bb("1"))), Arrays.asList(
+            newColUpdate("data", "seq", "2"), newColUpdate("data", "c", "2").setColVisibility(s2bb("A")))));
+
+    results = client.updateRowsConditionally(cwid, updates);
+
+    assertEquals(1, results.size());
+    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00348")));
+
+    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
+        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "2"}}, TABLE_TEST);
+
+    client.closeConditionalWriter(cwid);
+    try {
+      client.updateRowsConditionally(cwid, updates);
+      fail("conditional writer not closed");
+    } catch (UnknownWriter uk) {}
+
+    client.dropLocalUser(creds, "cwuser");
+
+  }
+
+  private void checkKey(String row, String cf, String cq, String val, KeyValue keyValue) {
+    assertEquals(row, ByteBufferUtil.toString(keyValue.key.row));
+    assertEquals(cf, ByteBufferUtil.toString(keyValue.key.colFamily));
+    assertEquals(cq, ByteBufferUtil.toString(keyValue.key.colQualifier));
+    assertEquals("", ByteBufferUtil.toString(keyValue.key.colVisibility));
+    assertEquals(val, ByteBufferUtil.toString(keyValue.value));
+  }
+
+  // scan metadata for file entries for the given table
+  private int countFiles(String table) throws Exception {
+    Map<String,String> tableIdMap = client.tableIdMap(creds);
+    String tableId = tableIdMap.get(table);
+    Key start = new Key();
+    start.row = s2bb(tableId + ";");
+    Key end = new Key();
+    end.row = s2bb(tableId + "<");
+    end = client.getFollowing(end, PartialKey.ROW);
+    ScanOptions opt = new ScanOptions();
+    opt.range = new Range(start, true, end, false);
+    opt.columns = Collections.singletonList(new ScanColumn(s2bb("file")));
+    String scanner = client.createScanner(creds, MetadataTable.NAME, opt);
+    int result = 0;
+    while (true) {
+      ScanResult more = client.nextK(scanner, 100);
+      result += more.getResults().size();
+      if (!more.more)
+        break;
+    }
+    return result;
+  }
+
+  private Map<ByteBuffer,List<ColumnUpdate>> mutation(String row, String cf, String cq, String value) {
+    ColumnUpdate upd = new ColumnUpdate(s2bb(cf), s2bb(cq));
+    upd.setValue(value.getBytes());
+    return Collections.singletonMap(s2bb(row), Collections.singletonList(upd));
+  }
+
+  private ByteBuffer s2bb(String cf) {
+    return ByteBuffer.wrap(cf.getBytes());
+  }
+
+  private Map<String,String> s2pp(String cf) {
+    Map<String,String> toRet = new TreeMap<String,String>();
+    toRet.put("password", cf);
+    return toRet;
+  }
+
+  static private ByteBuffer t2bb(Text t) {
+    return ByteBuffer.wrap(t.getBytes());
+  }
+  
+  @Test
+  public void testGetRowRange() throws Exception {
+    Range range = client.getRowRange(s2bb("xyzzy"));
+    org.apache.accumulo.core.data.Range range2 = new org.apache.accumulo.core.data.Range(new Text("xyzzy")); 
+    assertEquals(0, range.start.row.compareTo(t2bb(range2.getStartKey().getRow())));
+    assertEquals(0, range.stop.row.compareTo(t2bb(range2.getEndKey().getRow())));
+    assertEquals(range.startInclusive, range2.isStartKeyInclusive());
+    assertEquals(range.stopInclusive, range2.isEndKeyInclusive());
+    assertEquals(0, range.start.colFamily.compareTo(t2bb(range2.getStartKey().getColumnFamily())));
+    assertEquals(0, range.start.colQualifier.compareTo(t2bb(range2.getStartKey().getColumnQualifier())));
+    assertEquals(0, range.stop.colFamily.compareTo(t2bb(range2.getEndKey().getColumnFamily())));
+    assertEquals(0, range.stop.colQualifier.compareTo(t2bb(range2.getEndKey().getColumnQualifier())));
+    assertEquals(range.start.timestamp, range.start.timestamp);
+    assertEquals(range.stop.timestamp, range.stop.timestamp);
+  }
+  
+  @AfterClass
+  public static void tearDownMiniCluster() throws Exception {
+    accumulo.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
new file mode 100644
index 0000000..ad21f91
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Properties;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxyInstanceOperations {
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10197;
+  
+  @SuppressWarnings("serial")
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy.login("root", new TreeMap<String, String>() {{ put("password",""); }});
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Test
+  public void properties() throws TException {
+    tpc.proxy().setProperty(userpass, "test.systemprop", "whistletips");
+    
+    assertEquals(tpc.proxy().getSystemConfiguration(userpass).get("test.systemprop"), "whistletips");
+    tpc.proxy().removeProperty(userpass, "test.systemprop");
+    assertNull(tpc.proxy().getSystemConfiguration(userpass).get("test.systemprop"));
+    
+  }
+  
+  @Test
+  public void testClassLoad() throws TException {
+    assertTrue(tpc.proxy().testClassLoad(userpass, "org.apache.accumulo.core.iterators.user.RegExFilter", "org.apache.accumulo.core.iterators.Filter"));
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
new file mode 100644
index 0000000..c0049a0
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
@@ -0,0 +1,478 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.proxy.thrift.BatchScanOptions;
+import org.apache.accumulo.proxy.thrift.ColumnUpdate;
+import org.apache.accumulo.proxy.thrift.IteratorSetting;
+import org.apache.accumulo.proxy.thrift.Key;
+import org.apache.accumulo.proxy.thrift.KeyValue;
+import org.apache.accumulo.proxy.thrift.Range;
+import org.apache.accumulo.proxy.thrift.ScanColumn;
+import org.apache.accumulo.proxy.thrift.ScanOptions;
+import org.apache.accumulo.proxy.thrift.ScanResult;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxyReadWrite {
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10194;
+  protected static final String testtable = "testtable";
+  
+  @SuppressWarnings("serial")
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", new TreeMap<String, String>() {{put("password",""); }});
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Before
+  public void makeTestTable() throws Exception {
+    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
+  }
+  
+  @After
+  public void deleteTestTable() throws Exception {
+    tpc.proxy().deleteTable(userpass, testtable);
+  }
+  
+  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String value) {
+    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
+    update.setValue(value.getBytes());
+    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
+  }
+  
+  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String vis, String value) {
+    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
+    update.setValue(value.getBytes());
+    update.setColVisibility(vis.getBytes());
+    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
+  }
+  
+  /**
+   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a range so only the entries between -Inf...5 come back (there should be
+   * 50,000)
+   */
+  @Test
+  public void readWriteBatchOneShotWithRange() throws Exception {
+    int maxInserts = 100000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+      }
+    }
+    
+    Key stop = new Key();
+    stop.setRow("5".getBytes());
+    BatchScanOptions options = new BatchScanOptions();
+    options.ranges = Collections.singletonList(new Range(null, false, stop, false));
+    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      i += kvList.getResultsSize();
+      hasNext = kvList.isMore();
+    }
+    assertEquals(i, 50000);
+  }
+
+  /**
+   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a columnFamily so only the entries with specified column family come back (there should be
+   * 50,000)
+   */
+  @Test
+  public void readWriteBatchOneShotWithColumnFamilyOnly() throws Exception {
+    int maxInserts = 100000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+	
+      addMutation(mutations, String.format(format, i), "cf" + (i % 2) , "cq" + (i % 2), Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+      }
+    }
+    
+    BatchScanOptions options = new BatchScanOptions();
+
+	ScanColumn sc = new ScanColumn();
+	sc.colFamily = ByteBuffer.wrap("cf0".getBytes());
+
+    options.columns = Collections.singletonList(sc);
+    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      i += kvList.getResultsSize();
+      hasNext = kvList.isMore();
+    }
+    assertEquals(i, 50000);
+  }
+
+
+  /**
+   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a columnFamily + columnQualififer so only the entries with specified column 
+   * come back (there should be 50,000)
+   */
+  @Test
+  public void readWriteBatchOneShotWithFullColumn() throws Exception {
+    int maxInserts = 100000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+	
+      addMutation(mutations, String.format(format, i), "cf" + (i % 2) , "cq" + (i % 2), Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+      }
+    }
+    
+    BatchScanOptions options = new BatchScanOptions();
+
+	ScanColumn sc = new ScanColumn();
+	sc.colFamily = ByteBuffer.wrap("cf0".getBytes());
+	sc.colQualifier = ByteBuffer.wrap("cq0".getBytes());
+
+    options.columns = Collections.singletonList(sc);
+    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      i += kvList.getResultsSize();
+      hasNext = kvList.isMore();
+    }
+    assertEquals(i, 50000);
+  }
+
+
+  /**
+   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Filter the results so only the even numbers come back.
+   */
+  @Test
+  public void readWriteBatchOneShotWithFilterIterator() throws Exception {
+    int maxInserts = 10000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+      }
+      
+    }
+    
+    String regex = ".*[02468]";
+    
+    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
+    RegExFilter.setRegexs(is, regex, null, null, null, false);
+    
+    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
+    ScanOptions opts = new ScanOptions();
+    opts.iterators = Collections.singletonList(pis);
+    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      for (KeyValue kv : kvList.getResults()) {
+        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
+        
+        i += 2;
+      }
+      hasNext = kvList.isMore();
+    }
+  }
+  
+  @Test
+  public void readWriteOneShotWithRange() throws Exception {
+    int maxInserts = 100000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+      }
+    }
+    
+    Key stop = new Key();
+    stop.setRow("5".getBytes());
+    ScanOptions opts = new ScanOptions();
+    opts.range = new Range(null, false, stop, false);
+    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      i += kvList.getResultsSize();
+      hasNext = kvList.isMore();
+    }
+    assertEquals(i, 50000);
+  }
+  
+  /**
+   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Filter the results so only the even numbers come back.
+   */
+  @Test
+  public void readWriteOneShotWithFilterIterator() throws Exception {
+    int maxInserts = 10000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        
+        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+        mutations.clear();
+        
+      }
+      
+    }
+    
+    String regex = ".*[02468]";
+    
+    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
+    RegExFilter.setRegexs(is, regex, null, null, null, false);
+    
+    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
+    ScanOptions opts = new ScanOptions();
+    opts.iterators = Collections.singletonList(pis);
+    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      for (KeyValue kv : kvList.getResults()) {
+        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
+        
+        i += 2;
+      }
+      hasNext = kvList.isMore();
+    }
+  }
+  
+  // @Test
+  // This test takes kind of a long time. Enable it if you think you may have memory issues.
+  public void manyWritesAndReads() throws Exception {
+    int maxInserts = 1000000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$06d";
+    String writer = tpc.proxy().createWriter(userpass, testtable, null);
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        
+        tpc.proxy().update(writer, mutations);
+        mutations.clear();
+        
+      }
+      
+    }
+    
+    tpc.proxy().flush(writer);
+    tpc.proxy().closeWriter(writer);
+    
+    String cookie = tpc.proxy().createScanner(userpass, testtable, null);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      for (KeyValue kv : kvList.getResults()) {
+        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
+        i++;
+      }
+      hasNext = kvList.isMore();
+      if (hasNext)
+        assertEquals(k, kvList.getResults().size());
+    }
+    assertEquals(maxInserts, i);
+  }
+  
+  @Test
+  public void asynchReadWrite() throws Exception {
+    int maxInserts = 10000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    String writer = tpc.proxy().createWriter(userpass, testtable, null);
+    for (int i = 0; i < maxInserts; i++) {
+      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().update(writer, mutations);
+        mutations.clear();
+      }
+    }
+    
+    tpc.proxy().flush(writer);
+    tpc.proxy().closeWriter(writer);
+    
+    String regex = ".*[02468]";
+    
+    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
+    RegExFilter.setRegexs(is, regex, null, null, null, false);
+    
+    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
+    ScanOptions opts = new ScanOptions();
+    opts.iterators = Collections.singletonList(pis);
+    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    int numRead = 0;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      for (KeyValue kv : kvList.getResults()) {
+        assertEquals(i, Integer.parseInt(new String(kv.getKey().getRow())));
+        numRead++;
+        i += 2;
+      }
+      hasNext = kvList.isMore();
+    }
+    assertEquals(maxInserts / 2, numRead);
+  }
+  
+  @Test
+  public void testVisibility() throws Exception {
+    
+    Set<ByteBuffer> auths = new HashSet<ByteBuffer>();
+    auths.add(ByteBuffer.wrap("even".getBytes()));
+    tpc.proxy().changeUserAuthorizations(userpass, "root", auths);
+    
+    int maxInserts = 10000;
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    String format = "%1$05d";
+    String writer = tpc.proxy().createWriter(userpass, testtable, null);
+    for (int i = 0; i < maxInserts; i++) {
+      if (i % 2 == 0)
+        addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, "even", Util.randString(10));
+      else
+        addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, "odd", Util.randString(10));
+      
+      if (i % 1000 == 0 || i == maxInserts - 1) {
+        tpc.proxy().update(writer, mutations);
+        mutations.clear();
+      }
+    }
+    
+    tpc.proxy().flush(writer);
+    tpc.proxy().closeWriter(writer);
+    ScanOptions opts = new ScanOptions();
+    opts.authorizations = auths;
+    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
+    
+    int i = 0;
+    boolean hasNext = true;
+    
+    int k = 1000;
+    int numRead = 0;
+    while (hasNext) {
+      ScanResult kvList = tpc.proxy().nextK(cookie, k);
+      for (KeyValue kv : kvList.getResults()) {
+        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
+        i += 2;
+        numRead++;
+      }
+      hasNext = kvList.isMore();
+      
+    }
+    assertEquals(maxInserts / 2, numRead);
+  }
+  
+}


[13/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java b/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
deleted file mode 100644
index 62e5b64..0000000
--- a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
+++ /dev/null
@@ -1,1575 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.InputStreamReader;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.iterators.DevNull;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.user.SummingCombiner;
-import org.apache.accumulo.core.iterators.user.VersioningIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
-import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
-import org.apache.accumulo.proxy.thrift.AccumuloSecurityException;
-import org.apache.accumulo.proxy.thrift.ActiveCompaction;
-import org.apache.accumulo.proxy.thrift.ActiveScan;
-import org.apache.accumulo.proxy.thrift.BatchScanOptions;
-import org.apache.accumulo.proxy.thrift.Column;
-import org.apache.accumulo.proxy.thrift.ColumnUpdate;
-import org.apache.accumulo.proxy.thrift.CompactionReason;
-import org.apache.accumulo.proxy.thrift.CompactionType;
-import org.apache.accumulo.proxy.thrift.Condition;
-import org.apache.accumulo.proxy.thrift.ConditionalStatus;
-import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
-import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
-import org.apache.accumulo.proxy.thrift.DiskUsage;
-import org.apache.accumulo.proxy.thrift.IteratorScope;
-import org.apache.accumulo.proxy.thrift.IteratorSetting;
-import org.apache.accumulo.proxy.thrift.Key;
-import org.apache.accumulo.proxy.thrift.KeyValue;
-import org.apache.accumulo.proxy.thrift.MutationsRejectedException;
-import org.apache.accumulo.proxy.thrift.PartialKey;
-import org.apache.accumulo.proxy.thrift.Range;
-import org.apache.accumulo.proxy.thrift.ScanColumn;
-import org.apache.accumulo.proxy.thrift.ScanOptions;
-import org.apache.accumulo.proxy.thrift.ScanResult;
-import org.apache.accumulo.proxy.thrift.ScanState;
-import org.apache.accumulo.proxy.thrift.ScanType;
-import org.apache.accumulo.proxy.thrift.SystemPermission;
-import org.apache.accumulo.proxy.thrift.TableExistsException;
-import org.apache.accumulo.proxy.thrift.TableNotFoundException;
-import org.apache.accumulo.proxy.thrift.TablePermission;
-import org.apache.accumulo.proxy.thrift.TimeType;
-import org.apache.accumulo.proxy.thrift.UnknownScanner;
-import org.apache.accumulo.proxy.thrift.UnknownWriter;
-import org.apache.accumulo.proxy.thrift.WriterOptions;
-import org.apache.accumulo.server.util.PortUtils;
-import org.apache.accumulo.test.functional.SlowIterator;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.TServer;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-
-/**
- * Call every method on the proxy and try to verify that it works.
- */
-public class SimpleProxyIT {
-
-  public static File macTestFolder = new File(System.getProperty("user.dir") + "/target/" + SimpleProxyIT.class.getName());
-
-  private static MiniAccumuloCluster accumulo;
-  private static String secret = "superSecret";
-  private static Random random = new Random();
-  private static TServer proxyServer;
-  private static Thread thread;
-  private static int proxyPort;
-  private static org.apache.accumulo.proxy.thrift.AccumuloProxy.Client client;
-  private static String principal = "root";
-
-  private static Map<String,String> properties = new TreeMap<String,String>() {
-    private static final long serialVersionUID = 1L;
-
-    {
-      put("password", secret);
-    }
-  };
-  private static ByteBuffer creds = null;
-
-  private static Class<? extends TProtocolFactory> protocolClass;
-
-  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);
-    protocolFactories.add(org.apache.thrift.protocol.TTupleProtocol.Factory.class);
-    protocolFactories.add(org.apache.thrift.protocol.TCompactProtocol.Factory.class);
-
-    return protocolFactories.get(random.nextInt(protocolFactories.size()));
-  }
-
-  private static final AtomicInteger tableCounter = new AtomicInteger(0);
-
-  private static String makeTableName() {
-    return "test" + tableCounter.getAndIncrement();
-  }
-
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder(macTestFolder);
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @BeforeClass
-  public static void setupMiniCluster() throws Exception {
-    FileUtils.deleteQuietly(macTestFolder);
-    macTestFolder.mkdirs();
-    MiniAccumuloConfig config = new MiniAccumuloConfig(macTestFolder, secret).setNumTservers(1);
-    accumulo = new MiniAccumuloCluster(config);
-    accumulo.start();
-    // wait for accumulo to be up and functional
-    ZooKeeperInstance zoo = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers());
-    Connector c = zoo.getConnector("root", new PasswordToken(secret.getBytes()));
-    for (@SuppressWarnings("unused") Entry<org.apache.accumulo.core.data.Key,Value> entry : c.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
-        ;
-
-    Properties props = new Properties();
-    props.put("instance", accumulo.getConfig().getInstanceName());
-    props.put("zookeepers", accumulo.getZooKeepers());
-    props.put("tokenClass", PasswordToken.class.getName());
-
-    protocolClass = getRandomProtocol();
-
-    proxyPort = PortUtils.getRandomFreePort();
-    proxyServer = Proxy.createProxyServer(org.apache.accumulo.proxy.thrift.AccumuloProxy.class, org.apache.accumulo.proxy.ProxyServer.class, proxyPort,
-        protocolClass, props);
-    thread = new Thread() {
-      @Override
-      public void run() {
-        proxyServer.serve();
-      }
-    };
-    thread.start();
-    while (!proxyServer.isServing())
-      UtilWaitThread.sleep(100);
-    client = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
-    creds = client.login(principal, properties);
-  }
-
-  @Test(timeout = 10000)
-  public void security() throws Exception {
-    client.createLocalUser(creds, "user", s2bb(secret));
-    ByteBuffer badLogin = client.login("user", properties);
-    client.dropLocalUser(creds, "user");
-    final String table = makeTableName();
-    client.createTable(creds, table, false, TimeType.MILLIS);
-
-    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
-
-    try {
-      client.addConstraint(badLogin, table, NumericValueConstraint.class.getName());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.addSplits(badLogin, table, Collections.singleton(s2bb("1")));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.clearLocatorCache(badLogin, table);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.compactTable(badLogin, table, null, null, null, true, false);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.cancelCompaction(badLogin, table);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createTable(badLogin, table, false, TimeType.MILLIS);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.deleteTable(badLogin, table);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.deleteRows(badLogin, table, null, null);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.tableExists(badLogin, table);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.flushTable(badLogin, table, null, null, false);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getLocalityGroups(badLogin, table);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getMaxRow(badLogin, table, Collections.<ByteBuffer> emptySet(), null, false, null, false);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getTableProperties(badLogin, table);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.listSplits(badLogin, table, 10000);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.listTables(badLogin);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.listConstraints(badLogin, table);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.mergeTablets(badLogin, table, null, null);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.offlineTable(badLogin, table, false);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.onlineTable(badLogin, table, false);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.removeConstraint(badLogin, table, 0);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.removeTableProperty(badLogin, table, Property.TABLE_FILE_MAX.getKey());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.renameTable(badLogin, table, "someTableName");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
-      groups.put("group1", Collections.singleton("cf1"));
-      groups.put("group2", Collections.singleton("cf2"));
-      client.setLocalityGroups(badLogin, table, groups);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.setTableProperty(badLogin, table, Property.TABLE_FILE_MAX.getKey(), "0");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.tableIdMap(badLogin);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.getSiteConfiguration(badLogin);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getSystemConfiguration(badLogin);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getTabletServers(badLogin);
-      fail("exception not thrown");
-    } catch (TException ex) {}
-    try {
-      client.getActiveScans(badLogin, "fake");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getActiveCompactions(badLogin, "fakse");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.removeProperty(badLogin, "table.split.threshold");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.setProperty(badLogin, "table.split.threshold", "500M");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.testClassLoad(badLogin, DevNull.class.getName(), SortedKeyValueIterator.class.getName());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.authenticateUser(badLogin, "root", s2pp(secret));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      HashSet<ByteBuffer> auths = new HashSet<ByteBuffer>(Arrays.asList(s2bb("A"), s2bb("B")));
-      client.changeUserAuthorizations(badLogin, "stooge", auths);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.changeLocalUserPassword(badLogin, "stooge", s2bb(""));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createLocalUser(badLogin, "stooge", s2bb("password"));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.dropLocalUser(badLogin, "stooge");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getUserAuthorizations(badLogin, "stooge");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.grantSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.grantTablePermission(badLogin, "root", table, TablePermission.WRITE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.hasSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.hasTablePermission(badLogin, "root", table, TablePermission.WRITE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.listLocalUsers(badLogin);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.revokeSystemPermission(badLogin, "stooge", SystemPermission.CREATE_TABLE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.revokeTablePermission(badLogin, "root", table, TablePermission.ALTER_TABLE);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createScanner(badLogin, table, new ScanOptions());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createBatchScanner(badLogin, table, new BatchScanOptions());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.updateAndFlush(badLogin, table, new HashMap<ByteBuffer,List<ColumnUpdate>>());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createWriter(badLogin, table, new WriterOptions());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.attachIterator(badLogin, "slow", setting, EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.checkIteratorConflicts(badLogin, table, setting, EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      final String TABLE_TEST = makeTableName();
-      client.cloneTable(badLogin, table, TABLE_TEST, false, null, null);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.exportTable(badLogin, table, "/tmp");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.importTable(badLogin, "testify", "/tmp");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.getIteratorSetting(badLogin, table, "foo", IteratorScope.SCAN);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.listIterators(badLogin, table);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.removeIterator(badLogin, table, "name", EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.splitRangeByTablets(badLogin, table, client.getRowRange(ByteBuffer.wrap("row".getBytes())), 10);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      File importDir = tempFolder.newFolder("importDir");
-      File failuresDir = tempFolder.newFolder("failuresDir");
-      client.importDirectory(badLogin, table, importDir.getAbsolutePath(), failuresDir.getAbsolutePath(), true);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.pingTabletServer(badLogin, "fake");
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.login("badUser", properties);
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.testTableClassLoad(badLogin, table, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-    try {
-      client.createConditionalWriter(badLogin, table, new ConditionalWriterOptions());
-      fail("exception not thrown");
-    } catch (AccumuloSecurityException ex) {}
-  }
-
-  @Test(timeout = 10000)
-  public void tableNotFound() throws Exception {
-    final String doesNotExist = "doesNotExists";
-    try {
-      client.addConstraint(creds, doesNotExist, NumericValueConstraint.class.getName());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.addSplits(creds, doesNotExist, Collections.<ByteBuffer> emptySet());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    final IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "200"));
-    try {
-      client.attachIterator(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.cancelCompaction(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.checkIteratorConflicts(creds, doesNotExist, setting, EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.clearLocatorCache(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      final String TABLE_TEST = makeTableName();
-      client.cloneTable(creds, doesNotExist, TABLE_TEST, false, null, null);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.compactTable(creds, doesNotExist, null, null, null, true, false);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.createBatchScanner(creds, doesNotExist, new BatchScanOptions());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.createScanner(creds, doesNotExist, new ScanOptions());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.createWriter(creds, doesNotExist, new WriterOptions());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.deleteRows(creds, doesNotExist, null, null);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.deleteTable(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.exportTable(creds, doesNotExist, "/tmp");
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.flushTable(creds, doesNotExist, null, null, false);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.getIteratorSetting(creds, doesNotExist, "foo", IteratorScope.SCAN);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.getLocalityGroups(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.getMaxRow(creds, doesNotExist, Collections.<ByteBuffer> emptySet(), null, false, null, false);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.getTableProperties(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.grantTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.hasTablePermission(creds, "root", doesNotExist, TablePermission.WRITE);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      File importDir = tempFolder.newFolder("importDir");
-      File failuresDir = tempFolder.newFolder("failuresDir");
-      client.importDirectory(creds, doesNotExist, importDir.getAbsolutePath(), failuresDir.getAbsolutePath(), true);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.listConstraints(creds, doesNotExist);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.listSplits(creds, doesNotExist, 10000);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.mergeTablets(creds, doesNotExist, null, null);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.offlineTable(creds, doesNotExist, false);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.onlineTable(creds, doesNotExist, false);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.removeConstraint(creds, doesNotExist, 0);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.removeIterator(creds, doesNotExist, "name", EnumSet.allOf(IteratorScope.class));
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.removeTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.renameTable(creds, doesNotExist, "someTableName");
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.revokeTablePermission(creds, "root", doesNotExist, TablePermission.ALTER_TABLE);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.setTableProperty(creds, doesNotExist, Property.TABLE_FILE_MAX.getKey(), "0");
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.splitRangeByTablets(creds, doesNotExist, client.getRowRange(ByteBuffer.wrap("row".getBytes())), 10);
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.updateAndFlush(creds, doesNotExist, new HashMap<ByteBuffer,List<ColumnUpdate>>());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.getDiskUsage(creds, Collections.singleton(doesNotExist));
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.testTableClassLoad(creds, doesNotExist, VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName());
-      fail("exception not thrown");
-    } catch (TableNotFoundException ex) {}
-    try {
-      client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
-    } catch (TableNotFoundException ex) {}
-  }
-  
-  @Test(timeout = 10 * 1000)
-  public void testExists() throws Exception {
-    client.createTable(creds, "ett1", false, TimeType.MILLIS);
-    client.createTable(creds, "ett2", false, TimeType.MILLIS);
-    try {
-      client.createTable(creds, "ett1", false, TimeType.MILLIS);
-      fail("exception not thrown");
-    } catch (TableExistsException tee) {}
-    try {
-      client.renameTable(creds, "ett1", "ett2");
-      fail("exception not thrown");
-    } catch (TableExistsException tee) {}
-    try {
-      client.cloneTable(creds, "ett1", "ett2", false, new HashMap<String,String>(), new HashSet<String>());
-      fail("exception not thrown");
-    } catch (TableExistsException tee) {}
-  }
-
-  @Test(timeout = 10000)
-  public void testUnknownScanner() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-
-    String scanner = client.createScanner(creds, TABLE_TEST, null);
-    assertFalse(client.hasNext(scanner));
-    client.closeScanner(scanner);
-
-    try {
-      client.hasNext(scanner);
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-
-    try {
-      client.closeScanner(scanner);
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-
-    try {
-      client.nextEntry("99999999");
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-    try {
-      client.nextK("99999999", 6);
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-    try {
-      client.hasNext("99999999");
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-    try {
-      client.hasNext(UUID.randomUUID().toString());
-      fail("exception not thrown");
-    } catch (UnknownScanner us) {}
-  }
-
-  @Test(timeout = 10000)
-  public void testUnknownWriter() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-
-    String writer = client.createWriter(creds, TABLE_TEST, null);
-    client.update(writer, mutation("row0", "cf", "cq", "value"));
-    client.flush(writer);
-    client.update(writer, mutation("row2", "cf", "cq", "value2"));
-    client.closeWriter(writer);
-
-    // this is a oneway call, so it does not throw exceptions
-    client.update(writer, mutation("row2", "cf", "cq", "value2"));
-
-    try {
-      client.flush(writer);
-      fail("exception not thrown");
-    } catch (UnknownWriter uw) {}
-    try {
-      client.flush("99999");
-      fail("exception not thrown");
-    } catch (UnknownWriter uw) {}
-    try {
-      client.flush(UUID.randomUUID().toString());
-      fail("exception not thrown");
-    } catch (UnknownWriter uw) {}
-    try {
-      client.closeWriter("99999");
-      fail("exception not thrown");
-    } catch (UnknownWriter uw) {}
-  }
-
-  @Test(timeout = 10000)
-  public void testDelete() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-    client.updateAndFlush(creds, TABLE_TEST, mutation("row0", "cf", "cq", "value"));
-
-    assertScan(new String[][] {{"row0", "cf", "cq", "value"}}, TABLE_TEST);
-
-    ColumnUpdate upd = new ColumnUpdate(s2bb("cf"), s2bb("cq"));
-    upd.setDeleteCell(true);
-    Map<ByteBuffer,List<ColumnUpdate>> delete = Collections.singletonMap(s2bb("row0"), Collections.singletonList(upd));
-
-    client.updateAndFlush(creds, TABLE_TEST, delete);
-
-    assertScan(new String[][] {}, TABLE_TEST);
-  }
-
-  @Test(timeout = 60000)
-  public void testInstanceOperations() throws Exception {
-    int tservers = 0;
-    for (String tserver : client.getTabletServers(creds)) {
-      client.pingTabletServer(creds, tserver);
-      tservers++;
-    }
-    assertTrue(tservers > 0);
-
-    // get something we know is in the site config
-    Map<String,String> cfg = client.getSiteConfiguration(creds);
-    assertTrue(cfg.get("instance.dfs.dir").startsWith(macTestFolder.getPath()));
-
-    // set a property in zookeeper
-    client.setProperty(creds, "table.split.threshold", "500M");
-
-    // check that we can read it
-    for (int i = 0; i < 5; i++) {
-      cfg = client.getSystemConfiguration(creds);
-      if ("500M".equals(cfg.get("table.split.threshold")))
-        break;
-      UtilWaitThread.sleep(200);
-    }
-    assertEquals("500M", cfg.get("table.split.threshold"));
-
-    // unset the setting, check that it's not what it was
-    client.removeProperty(creds, "table.split.threshold");
-    for (int i = 0; i < 5; i++) {
-      cfg = client.getSystemConfiguration(creds);
-      if (!"500M".equals(cfg.get("table.split.threshold")))
-        break;
-      UtilWaitThread.sleep(200);
-    }
-    assertNotEquals("500M", cfg.get("table.split.threshold"));
-
-    // try to load some classes via the proxy
-    assertTrue(client.testClassLoad(creds, DevNull.class.getName(), SortedKeyValueIterator.class.getName()));
-    assertFalse(client.testClassLoad(creds, "foo.bar", SortedKeyValueIterator.class.getName()));
-
-    // create a table that's very slow, so we can look for scans/compactions
-    client.createTable(creds, "slow", true, TimeType.MILLIS);
-    IteratorSetting setting = new IteratorSetting(100, "slow", SlowIterator.class.getName(), Collections.singletonMap("sleepTime", "250"));
-    client.attachIterator(creds, "slow", setting, EnumSet.allOf(IteratorScope.class));
-
-    // Should take 10 seconds to read every record
-    for (int i = 0; i < 40; i++) {
-      client.updateAndFlush(creds, "slow", mutation("row" + i, "cf", "cq", "value"));
-    }
-
-    // scan
-    Thread t = new Thread() {
-      @Override
-      public void run() {
-        String scanner;
-        try {
-          Client client2 = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
-          scanner = client2.createScanner(creds, "slow", null);
-          client2.nextK(scanner, 10);
-          client2.closeScanner(scanner);
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-    t.start();
-
-    // look for the scan many times 
-    List<ActiveScan> scans = new ArrayList<ActiveScan>();
-    for (int i = 0; i < 100 && scans.isEmpty(); i++) {
-      for (String tserver : client.getTabletServers(creds)) {
-        List<ActiveScan> scansForServer = client.getActiveScans(creds, tserver);
-        for (ActiveScan scan : scansForServer) {
-          if ("root".equals(scan.getUser())) {
-            scans.add(scan);
-          }
-        }
-
-        if (!scans.isEmpty())
-          break;
-        UtilWaitThread.sleep(100);
-      }
-    }
-    t.join();
-
-    assertFalse(scans.isEmpty());
-    boolean found = false;
-    Map<String,String> map = null;
-    for (int i = 0; i < scans.size() && !found; i++) {
-      ActiveScan scan = scans.get(i);
-      if ("root".equals(scan.getUser())) {
-        assertTrue(ScanState.RUNNING.equals(scan.getState()) || ScanState.QUEUED.equals(scan.getState()));
-        assertEquals(ScanType.SINGLE, scan.getType());
-        assertEquals("slow", scan.getTable());
-
-        map = client.tableIdMap(creds);
-        assertEquals(map.get("slow"), scan.getExtent().tableId);
-        assertTrue(scan.getExtent().endRow == null);
-        assertTrue(scan.getExtent().prevEndRow == null);
-        found = true;
-      }
-    }
-
-    assertTrue("Could not find a scan against the 'slow' table", found);
-
-    // start a compaction
-    t = new Thread() {
-      @Override
-      public void run() {
-        try {
-          Client client2 = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
-          client2.compactTable(creds, "slow", null, null, null, true, true);
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-    t.start();
-
-    final String desiredTableId = map.get("slow");
-
-    // try to catch it in the act
-    List<ActiveCompaction> compactions = new ArrayList<ActiveCompaction>();
-    for (int i = 0; i < 100 && compactions.isEmpty(); i++) {
-      // Iterate over the tservers
-      for (String tserver : client.getTabletServers(creds)) {
-        // And get the compactions on each
-        List<ActiveCompaction> compactionsOnServer = client.getActiveCompactions(creds, tserver);
-        for (ActiveCompaction compact : compactionsOnServer) {
-          // There might be other compactions occurring (e.g. on METADATA) in which
-          // case we want to prune out those that aren't for our slow table
-          if (desiredTableId.equals(compact.getExtent().tableId)) {
-            compactions.add(compact);
-          }
-        }
-
-        // If we found a compaction for the table we wanted, so we can stop looking
-        if (!compactions.isEmpty())
-          break;
-      }
-      UtilWaitThread.sleep(10);
-    }
-    t.join();
-
-    // verify the compaction information
-    assertFalse(compactions.isEmpty());
-    for (ActiveCompaction c : compactions) {
-      if (desiredTableId.equals(c.getExtent().tableId)) {
-        assertTrue(c.inputFiles.isEmpty());
-        assertEquals(CompactionType.MINOR, c.getType());
-        assertEquals(CompactionReason.USER, c.getReason());
-        assertEquals("", c.localityGroup);
-        assertTrue(c.outputFile.contains("default_tablet"));
-
-        return;
-      }
-    }
-    fail("Expection to find running compaction for table 'slow' but did not find one");
-  }
-
-  @Test
-  public void testSecurityOperations() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    // check password
-    assertTrue(client.authenticateUser(creds, "root", s2pp(secret)));
-    assertFalse(client.authenticateUser(creds, "root", s2pp("")));
-
-    // create a user
-    client.createLocalUser(creds, "stooge", s2bb("password"));
-    // change auths
-    Set<String> users = client.listLocalUsers(creds);
-    assertEquals(new HashSet<String>(Arrays.asList("root", "stooge")), users);
-    HashSet<ByteBuffer> auths = new HashSet<ByteBuffer>(Arrays.asList(s2bb("A"), s2bb("B")));
-    client.changeUserAuthorizations(creds, "stooge", auths);
-    List<ByteBuffer> update = client.getUserAuthorizations(creds, "stooge");
-    assertEquals(auths, new HashSet<ByteBuffer>(update));
-
-    // change password
-    client.changeLocalUserPassword(creds, "stooge", s2bb(""));
-    assertTrue(client.authenticateUser(creds, "stooge", s2pp("")));
-
-    // check permission failure
-    @SuppressWarnings("serial")
-    ByteBuffer stooge = client.login("stooge", new TreeMap<String,String>() {
-      {
-        put("password", "");
-      }
-    });
-
-    try {
-      client.createTable(stooge, "fail", true, TimeType.MILLIS);
-      fail("should not create the table");
-    } catch (AccumuloSecurityException ex) {
-      assertFalse(client.listTables(creds).contains("fail"));
-    }
-    // grant permissions and test
-    assertFalse(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
-    client.grantSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE);
-    assertTrue(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
-    client.createTable(stooge, "success", true, TimeType.MILLIS);
-    client.listTables(creds).contains("succcess");
-
-    // revoke permissions
-    client.revokeSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE);
-    assertFalse(client.hasSystemPermission(creds, "stooge", SystemPermission.CREATE_TABLE));
-    try {
-      client.createTable(stooge, "fail", true, TimeType.MILLIS);
-      fail("should not create the table");
-    } catch (AccumuloSecurityException ex) {
-      assertFalse(client.listTables(creds).contains("fail"));
-    }
-    // create a table to test table permissions
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-    // denied!
-    try {
-      String scanner = client.createScanner(stooge, TABLE_TEST, null);
-      client.nextK(scanner, 100);
-      fail("stooge should not read table test");
-    } catch (AccumuloSecurityException ex) {}
-    // grant
-    assertFalse(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
-    client.grantTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ);
-    assertTrue(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
-    String scanner = client.createScanner(stooge, TABLE_TEST, null);
-    client.nextK(scanner, 10);
-    client.closeScanner(scanner);
-    // revoke
-    client.revokeTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ);
-    assertFalse(client.hasTablePermission(creds, "stooge", TABLE_TEST, TablePermission.READ));
-    try {
-      scanner = client.createScanner(stooge, TABLE_TEST, null);
-      client.nextK(scanner, 100);
-      fail("stooge should not read table test");
-    } catch (AccumuloSecurityException ex) {}
-
-    // delete user
-    client.dropLocalUser(creds, "stooge");
-    users = client.listLocalUsers(creds);
-    assertEquals(1, users.size());
-
-  }
-
-  @Test
-  public void testBatchWriter() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
-
-    WriterOptions writerOptions = new WriterOptions();
-    writerOptions.setLatencyMs(10000);
-    writerOptions.setMaxMemory(2);
-    writerOptions.setThreads(1);
-    writerOptions.setTimeoutMs(100000);
-
-    String batchWriter = client.createWriter(creds, TABLE_TEST, writerOptions);
-    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
-    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
-    try {
-      client.flush(batchWriter);
-      fail("constraint did not fire");
-    } catch (MutationsRejectedException ex) {}
-    try {
-      client.closeWriter(batchWriter);
-      fail("constraint did not fire");
-    } catch (MutationsRejectedException e) {}
-
-    client.removeConstraint(creds, TABLE_TEST, 2);
-
-    assertScan(new String[][] {}, TABLE_TEST);
-
-    UtilWaitThread.sleep(2000);
-    
-    writerOptions = new WriterOptions();
-    writerOptions.setLatencyMs(10000);
-    writerOptions.setMaxMemory(3000);
-    writerOptions.setThreads(1);
-    writerOptions.setTimeoutMs(100000);
-
-    batchWriter = client.createWriter(creds, TABLE_TEST, writerOptions);
-
-    client.update(batchWriter, mutation("row1", "cf", "cq", "x"));
-    client.flush(batchWriter);
-    client.closeWriter(batchWriter);
-
-    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, TABLE_TEST);
-
-    client.deleteTable(creds, TABLE_TEST);
-  }
-
-  @Test
-  public void testTableOperations() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-    // constraints
-    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
-    assertEquals(2, client.listConstraints(creds, TABLE_TEST).size());
-
-    UtilWaitThread.sleep(2000);
-
-    client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "123"));
-
-    try {
-      client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "x"));
-      fail("constraint did not fire");
-    } catch (MutationsRejectedException ex) {}
-
-    client.removeConstraint(creds, TABLE_TEST, 2);
-
-    UtilWaitThread.sleep(2000);
-
-    assertEquals(1, client.listConstraints(creds, TABLE_TEST).size());
-
-    client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "x"));
-    assertScan(new String[][] {{"row1", "cf", "cq", "x"}}, TABLE_TEST);
-    // splits, merge
-    client.addSplits(creds, TABLE_TEST, new HashSet<ByteBuffer>(Arrays.asList(s2bb("a"), s2bb("m"), s2bb("z"))));
-    List<ByteBuffer> splits = client.listSplits(creds, TABLE_TEST, 1);
-    assertEquals(Arrays.asList(s2bb("m")), splits);
-    client.mergeTablets(creds, TABLE_TEST, null, s2bb("m"));
-    splits = client.listSplits(creds, TABLE_TEST, 10);
-    assertEquals(Arrays.asList(s2bb("m"), s2bb("z")), splits);
-    client.mergeTablets(creds, TABLE_TEST, null, null);
-    splits = client.listSplits(creds, TABLE_TEST, 10);
-    List<ByteBuffer> empty = Collections.emptyList();
-    assertEquals(empty, splits);
-    // iterators
-    client.deleteTable(creds, TABLE_TEST);
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-    HashMap<String,String> options = new HashMap<String,String>();
-    options.put("type", "STRING");
-    options.put("columns", "cf");
-    IteratorSetting setting = new IteratorSetting(10, TABLE_TEST, SummingCombiner.class.getName(), options);
-    client.attachIterator(creds, TABLE_TEST, setting, EnumSet.allOf(IteratorScope.class));
-    for (int i = 0; i < 10; i++) {
-      client.updateAndFlush(creds, TABLE_TEST, mutation("row1", "cf", "cq", "1"));
-    }
-    assertScan(new String[][] {{"row1", "cf", "cq", "10"}}, TABLE_TEST);
-    try {
-      client.checkIteratorConflicts(creds, TABLE_TEST, setting, EnumSet.allOf(IteratorScope.class));
-      fail("checkIteratorConflicts did not throw an exception");
-    } catch (Exception ex) {}
-    client.deleteRows(creds, TABLE_TEST, null, null);
-    client.removeIterator(creds, TABLE_TEST, "test", EnumSet.allOf(IteratorScope.class));
-    String expected[][] = new String[10][];
-    for (int i = 0; i < 10; i++) {
-      client.updateAndFlush(creds, TABLE_TEST, mutation("row" + i, "cf", "cq", "" + i));
-      expected[i] = new String[] {"row" + i, "cf", "cq", "" + i};
-      client.flushTable(creds, TABLE_TEST, null, null, true);
-    }
-    assertScan(expected, TABLE_TEST);
-    // clone
-    final String TABLE_TEST2 = makeTableName();
-    client.cloneTable(creds, TABLE_TEST, TABLE_TEST2, true, null, null);
-    assertScan(expected, TABLE_TEST2);
-    client.deleteTable(creds, TABLE_TEST2);
-
-    // don't know how to test this, call it just for fun
-    client.clearLocatorCache(creds, TABLE_TEST);
-
-    // compact
-    client.compactTable(creds, TABLE_TEST, null, null, null, true, true);
-    assertEquals(1, countFiles(TABLE_TEST));
-    assertScan(expected, TABLE_TEST);
-
-    // get disk usage
-    client.cloneTable(creds, TABLE_TEST, TABLE_TEST2, true, null, null);
-    Set<String> tablesToScan = new HashSet<String>();
-    tablesToScan.add(TABLE_TEST);
-    tablesToScan.add(TABLE_TEST2);
-    tablesToScan.add("foo");
-    client.createTable(creds, "foo", true, TimeType.MILLIS);
-    List<DiskUsage> diskUsage = (client.getDiskUsage(creds, tablesToScan));
-    assertEquals(2, diskUsage.size());
-    assertEquals(1, diskUsage.get(0).getTables().size());
-    assertEquals(2, diskUsage.get(1).getTables().size());
-    client.compactTable(creds, TABLE_TEST2, null, null, null, true, true);
-    diskUsage = (client.getDiskUsage(creds, tablesToScan));
-    assertEquals(3, diskUsage.size());
-    assertEquals(1, diskUsage.get(0).getTables().size());
-    assertEquals(1, diskUsage.get(1).getTables().size());
-    assertEquals(1, diskUsage.get(2).getTables().size());
-    client.deleteTable(creds, "foo");
-    client.deleteTable(creds, TABLE_TEST2);
-
-    // export/import
-    File dir = tempFolder.newFolder("test");
-    File destDir = tempFolder.newFolder("test_dest");
-    client.offlineTable(creds, TABLE_TEST, false);
-    client.exportTable(creds, TABLE_TEST, dir.getAbsolutePath());
-    // copy files to a new location
-    FileSystem fs = FileSystem.get(new Configuration());
-    FSDataInputStream is = fs.open(new Path(dir + "/distcp.txt"));
-    BufferedReader r = new BufferedReader(new InputStreamReader(is));
-    while (true) {
-      String line = r.readLine();
-      if (line == null)
-        break;
-      Path srcPath = new Path(line);
-      FileUtils.copyFile(new File(srcPath.toUri().getPath()), new File(destDir, srcPath.getName()));
-    }
-    client.deleteTable(creds, TABLE_TEST);
-    client.importTable(creds, "testify", destDir.getAbsolutePath());
-    assertScan(expected, "testify");
-    client.deleteTable(creds, "testify");
-
-    try {
-      // ACCUMULO-1558 a second import from the same dir should fail, the first import moved the files
-      client.importTable(creds, "testify2", destDir.getAbsolutePath());
-      fail();
-    } catch (Exception e) {}
-
-    assertFalse(client.listTables(creds).contains("testify2"));
-
-    // Locality groups
-    client.createTable(creds, "test", true, TimeType.MILLIS);
-    Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
-    groups.put("group1", Collections.singleton("cf1"));
-    groups.put("group2", Collections.singleton("cf2"));
-    client.setLocalityGroups(creds, "test", groups);
-    assertEquals(groups, client.getLocalityGroups(creds, "test"));
-    // table properties
-    Map<String,String> orig = client.getTableProperties(creds, "test");
-    client.setTableProperty(creds, "test", "table.split.threshold", "500M");
-    Map<String,String> update = client.getTableProperties(creds, "test");
-    assertEquals(update.get("table.split.threshold"), "500M");
-    client.removeTableProperty(creds, "test", "table.split.threshold");
-    update = client.getTableProperties(creds, "test");
-    assertEquals(orig, update);
-    // rename table
-    Map<String,String> tables = client.tableIdMap(creds);
-    client.renameTable(creds, "test", "bar");
-    Map<String,String> tables2 = client.tableIdMap(creds);
-    assertEquals(tables.get("test"), tables2.get("bar"));
-    // table exists
-    assertTrue(client.tableExists(creds, "bar"));
-    assertFalse(client.tableExists(creds, "test"));
-    // bulk import
-    String filename = dir + "/bulk/import/rfile.rf";
-    FileSKVWriter writer = FileOperations.getInstance().openWriter(filename, fs, fs.getConf(), DefaultConfiguration.getInstance());
-    writer.startDefaultLocalityGroup();
-    writer.append(new org.apache.accumulo.core.data.Key(new Text("a"), new Text("b"), new Text("c")), new Value("value".getBytes()));
-    writer.close();
-    fs.mkdirs(new Path(dir + "/bulk/fail"));
-    client.importDirectory(creds, "bar", dir + "/bulk/import", dir + "/bulk/fail", true);
-    String scanner = client.createScanner(creds, "bar", null);
-    ScanResult more = client.nextK(scanner, 100);
-    client.closeScanner(scanner);
-    assertEquals(1, more.results.size());
-    ByteBuffer maxRow = client.getMaxRow(creds, "bar", null, null, false, null, false);
-    assertEquals(s2bb("a"), maxRow);
-
-    assertFalse(client.testTableClassLoad(creds, "bar", "abc123", SortedKeyValueIterator.class.getName()));
-    assertTrue(client.testTableClassLoad(creds, "bar", VersioningIterator.class.getName(), SortedKeyValueIterator.class.getName()));
-  }
-
-  private Condition newCondition(String cf, String cq) {
-    return new Condition(new Column(s2bb(cf), s2bb(cq), s2bb("")));
-  }
-
-  private Condition newCondition(String cf, String cq, String val) {
-    return newCondition(cf, cq).setValue(s2bb(val));
-  }
-
-  private Condition newCondition(String cf, String cq, long ts, String val) {
-    return newCondition(cf, cq).setValue(s2bb(val)).setTimestamp(ts);
-  }
-
-  private ColumnUpdate newColUpdate(String cf, String cq, String val) {
-    return new ColumnUpdate(s2bb(cf), s2bb(cq)).setValue(s2bb(val));
-  }
-
-  private ColumnUpdate newColUpdate(String cf, String cq, long ts, String val) {
-    return new ColumnUpdate(s2bb(cf), s2bb(cq)).setTimestamp(ts).setValue(s2bb(val));
-  }
-
-  private void assertScan(String[][] expected, String table) throws Exception {
-    String scid = client.createScanner(creds, table, new ScanOptions());
-    ScanResult keyValues = client.nextK(scid, expected.length + 1);
-
-    assertEquals(expected.length, keyValues.results.size());
-    assertFalse(keyValues.more);
-
-    for (int i = 0; i < keyValues.results.size(); i++) {
-      checkKey(expected[i][0], expected[i][1], expected[i][2], expected[i][3], keyValues.results.get(i));
-    }
-
-    client.closeScanner(scid);
-  }
-
-  @Test
-  public void testConditionalWriter() throws Exception {
-    final String TABLE_TEST = makeTableName();
-
-    client.createTable(creds, TABLE_TEST, true, TimeType.MILLIS);
-
-    client.addConstraint(creds, TABLE_TEST, NumericValueConstraint.class.getName());
-
-    String cwid = client.createConditionalWriter(creds, TABLE_TEST, new ConditionalWriterOptions());
-
-    Map<ByteBuffer,ConditionalUpdates> updates = new HashMap<ByteBuffer,ConditionalUpdates>();
-
-    updates.put(
-        s2bb("00345"),
-        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", 10, "1"),
-            newColUpdate("data", "img", "73435435"))));
-
-    Map<ByteBuffer,ConditionalStatus> results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "73435435"}, {"00345", "meta", "seq", "1"}}, TABLE_TEST);
-
-    // test not setting values on conditions
-    updates.clear();
-
-    updates.put(s2bb("00345"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", "2"))));
-    updates.put(s2bb("00346"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq")), Arrays.asList(newColUpdate("meta", "seq", "1"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(2, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00345")));
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00346")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "73435435"}, {"00345", "meta", "seq", "1"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
-
-    // test setting values on conditions
-    updates.clear();
-
-    updates.put(
-        s2bb("00345"),
-        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", "1")), Arrays.asList(newColUpdate("meta", "seq", 20, "2"),
-            newColUpdate("data", "img", "567890"))));
-
-    updates.put(s2bb("00346"), new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", "2")), Arrays.asList(newColUpdate("meta", "seq", "3"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(2, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00346")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "567890"}, {"00345", "meta", "seq", "2"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
-
-    // test setting timestamp on condition to a non-existant version
-    updates.clear();
-
-    updates.put(
-        s2bb("00345"),
-        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", 10, "2")), Arrays.asList(newColUpdate("meta", "seq", 30, "3"),
-            newColUpdate("data", "img", "1234567890"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00345")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "567890"}, {"00345", "meta", "seq", "2"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
-
-    // test setting timestamp to an existing version
-
-    updates.clear();
-
-    updates.put(
-        s2bb("00345"),
-        new ConditionalUpdates(Arrays.asList(newCondition("meta", "seq", 20, "2")), Arrays.asList(newColUpdate("meta", "seq", 30, "3"),
-            newColUpdate("data", "img", "1234567890"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00345")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"}}, TABLE_TEST);
-
-    // run test w/ condition that has iterators
-    // following should fail w/o iterator
-    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
-    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
-    client.updateAndFlush(creds, TABLE_TEST, Collections.singletonMap(s2bb("00347"), Arrays.asList(newColUpdate("data", "count", "1"))));
-
-    updates.clear();
-    updates.put(s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "count", "3")), Arrays.asList(newColUpdate("data", "img", "1234567890"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}}, TABLE_TEST);
-
-    // following test w/ iterator setup should succeed
-    Condition iterCond = newCondition("data", "count", "3");
-    Map<String,String> props = new HashMap<String,String>();
-    props.put("type", "STRING");
-    props.put("columns", "data:count");
-    IteratorSetting is = new IteratorSetting(1, "sumc", SummingCombiner.class.getName(), props);
-    iterCond.setIterators(Arrays.asList(is));
-
-    updates.clear();
-    updates.put(s2bb("00347"), new ConditionalUpdates(Arrays.asList(iterCond), Arrays.asList(newColUpdate("data", "img", "1234567890"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
-
-    // test a mutation that violated a constraint
-    updates.clear();
-    updates.put(s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890")), Arrays.asList(newColUpdate("data", "count", "A"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.VIOLATED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
-
-    // run test with two conditions
-    // both conditions should fail
-    updates.clear();
-    updates.put(
-        s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "565"), newCondition("data", "count", "2")), Arrays.asList(
-            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
-
-    // one condition should fail
-    updates.clear();
-    updates.put(
-        s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890"), newCondition("data", "count", "2")), Arrays.asList(
-            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
-
-    // one condition should fail
-    updates.clear();
-    updates.put(
-        s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "565"), newCondition("data", "count", "1")), Arrays.asList(
-            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00347")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "1"}, {"00347", "data", "img", "1234567890"}}, TABLE_TEST);
-
-    // both conditions should succeed
-
-    ConditionalStatus result = client.updateRowConditionally(
-        creds,
-        TABLE_TEST,
-        s2bb("00347"),
-        new ConditionalUpdates(Arrays.asList(newCondition("data", "img", "1234567890"), newCondition("data", "count", "1")), Arrays.asList(
-            newColUpdate("data", "count", "3"), newColUpdate("data", "img", "0987654321"))));
-
-    assertEquals(ConditionalStatus.ACCEPTED, result);
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}}, TABLE_TEST);
-
-    client.closeConditionalWriter(cwid);
-    try {
-      client.updateRowsConditionally(cwid, updates);
-      fail("conditional writer not closed");
-    } catch (UnknownWriter uk) {}
-
-    // run test with colvis
-    client.createLocalUser(creds, "cwuser", s2bb("bestpasswordever"));
-    client.changeUserAuthorizations(creds, "cwuser", Collections.singleton(s2bb("A")));
-    client.grantTablePermission(creds, "cwuser", TABLE_TEST, TablePermission.WRITE);
-    client.grantTablePermission(creds, "cwuser", TABLE_TEST, TablePermission.READ);
-
-    ByteBuffer cwuCreds = client.login("cwuser", Collections.singletonMap("password", "bestpasswordever"));
-
-    cwid = client.createConditionalWriter(cwuCreds, TABLE_TEST, new ConditionalWriterOptions().setAuthorizations(Collections.singleton(s2bb("A"))));
-
-    updates.clear();
-    updates.put(
-        s2bb("00348"),
-        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A")))), Arrays.asList(newColUpdate("data", "seq", "1"),
-            newColUpdate("data", "c", "1").setColVisibility(s2bb("A")))));
-    updates.put(s2bb("00349"),
-        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("B")))), Arrays.asList(newColUpdate("data", "seq", "1"))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(2, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00348")));
-    assertEquals(ConditionalStatus.INVISIBLE_VISIBILITY, results.get(s2bb("00349")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "1"}}, TABLE_TEST);
-
-    updates.clear();
-
-    updates.clear();
-    updates.put(
-        s2bb("00348"),
-        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A"))).setValue(s2bb("0"))), Arrays.asList(
-            newColUpdate("data", "seq", "2"), newColUpdate("data", "c", "2").setColVisibility(s2bb("A")))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.REJECTED, results.get(s2bb("00348")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "1"}}, TABLE_TEST);
-
-    updates.clear();
-    updates.put(
-        s2bb("00348"),
-        new ConditionalUpdates(Arrays.asList(new Condition(new Column(s2bb("data"), s2bb("c"), s2bb("A"))).setValue(s2bb("1"))), Arrays.asList(
-            newColUpdate("data", "seq", "2"), newColUpdate("data", "c", "2").setColVisibility(s2bb("A")))));
-
-    results = client.updateRowsConditionally(cwid, updates);
-
-    assertEquals(1, results.size());
-    assertEquals(ConditionalStatus.ACCEPTED, results.get(s2bb("00348")));
-
-    assertScan(new String[][] { {"00345", "data", "img", "1234567890"}, {"00345", "meta", "seq", "3"}, {"00346", "meta", "seq", "1"},
-        {"00347", "data", "count", "3"}, {"00347", "data", "img", "0987654321"}, {"00348", "data", "seq", "2"}}, TABLE_TEST);
-
-    client.closeConditionalWriter(cwid);
-    try {
-      client.updateRowsConditionally(cwid, updates);
-      fail("conditional writer not closed");
-    } catch (UnknownWriter uk) {}
-
-    client.dropLocalUser(creds, "cwuser");
-
-  }
-
-  private void checkKey(String row, String cf, String cq, String val, KeyValue keyValue) {
-    assertEquals(row, ByteBufferUtil.toString(keyValue.key.row));
-    assertEquals(cf, ByteBufferUtil.toString(keyValue.key.colFamily));
-    assertEquals(cq, ByteBufferUtil.toString(keyValue.key.colQualifier));
-    assertEquals("", ByteBufferUtil.toString(keyValue.key.colVisibility));
-    assertEquals(val, ByteBufferUtil.toString(keyValue.value));
-  }
-
-  // scan metadata for file entries for the given table
-  private int countFiles(String table) throws Exception {
-    Map<String,String> tableIdMap = client.tableIdMap(creds);
-    String tableId = tableIdMap.get(table);
-    Key start = new Key();
-    start.row = s2bb(tableId + ";");
-    Key end = new Key();
-    end.row = s2bb(tableId + "<");
-    end = client.getFollowing(end, PartialKey.ROW);
-    ScanOptions opt = new ScanOptions();
-    opt.range = new Range(start, true, end, false);
-    opt.columns = Collections.singletonList(new ScanColumn(s2bb("file")));
-    String scanner = client.createScanner(creds, MetadataTable.NAME, opt);
-    int result = 0;
-    while (true) {
-      ScanResult more = client.nextK(scanner, 100);
-      result += more.getResults().size();
-      if (!more.more)
-        break;
-    }
-    return result;
-  }
-
-  private Map<ByteBuffer,List<ColumnUpdate>> mutation(String row, String cf, String cq, String value) {
-    ColumnUpdate upd = new ColumnUpdate(s2bb(cf), s2bb(cq));
-    upd.setValue(value.getBytes());
-    return Collections.singletonMap(s2bb(row), Collections.singletonList(upd));
-  }
-
-  private ByteBuffer s2bb(String cf) {
-    return ByteBuffer.wrap(cf.getBytes());
-  }
-
-  private Map<String,String> s2pp(String cf) {
-    Map<String,String> toRet = new TreeMap<String,String>();
-    toRet.put("password", cf);
-    return toRet;
-  }
-
-  static private ByteBuffer t2bb(Text t) {
-    return ByteBuffer.wrap(t.getBytes());
-  }
-  
-  @Test
-  public void testGetRowRange() throws Exception {
-    Range range = client.getRowRange(s2bb("xyzzy"));
-    org.apache.accumulo.core.data.Range range2 = new org.apache.accumulo.core.data.Range(new Text("xyzzy")); 
-    assertEquals(0, range.start.row.compareTo(t2bb(range2.getStartKey().getRow())));
-    assertEquals(0, range.stop.row.compareTo(t2bb(range2.getEndKey().getRow())));
-    assertEquals(range.startInclusive, range2.isStartKeyInclusive());
-    assertEquals(range.stopInclusive, range2.isEndKeyInclusive());
-    assertEquals(0, range.start.colFamily.compareTo(t2bb(range2.getStartKey().getColumnFamily())));
-    assertEquals(0, range.start.colQualifier.compareTo(t2bb(range2.getStartKey().getColumnQualifier())));
-    assertEquals(0, range.stop.colFamily.compareTo(t2bb(range2.getEndKey().getColumnFamily())));
-    assertEquals(0, range.stop.colQualifier.compareTo(t2bb(range2.getEndKey().getColumnQualifier())));
-    assertEquals(range.start.timestamp, range.start.timestamp);
-    assertEquals(range.stop.timestamp, range.stop.timestamp);
-  }
-  
-  @AfterClass
-  public static void tearDownMiniCluster() throws Exception {
-    accumulo.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
deleted file mode 100644
index ad21f91..0000000
--- a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyInstanceOperations.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.Properties;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.server.TServer;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestProxyInstanceOperations {
-  protected static TServer proxy;
-  protected static Thread thread;
-  protected static TestProxyClient tpc;
-  protected static ByteBuffer userpass;
-  protected static final int port = 10197;
-  
-  @SuppressWarnings("serial")
-  @BeforeClass
-  public static void setup() throws Exception {
-    Properties prop = new Properties();
-    prop.setProperty("useMockInstance", "true");
-    prop.put("tokenClass", PasswordToken.class.getName());
-    
-    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
-        port, TCompactProtocol.Factory.class, prop);
-    thread = new Thread() {
-      @Override
-      public void run() {
-        proxy.serve();
-      }
-    };
-    thread.start();
-    tpc = new TestProxyClient("localhost", port);
-    userpass = tpc.proxy.login("root", new TreeMap<String, String>() {{ put("password",""); }});
-  }
-  
-  @AfterClass
-  public static void tearDown() throws InterruptedException {
-    proxy.stop();
-    thread.join();
-  }
-  
-  @Test
-  public void properties() throws TException {
-    tpc.proxy().setProperty(userpass, "test.systemprop", "whistletips");
-    
-    assertEquals(tpc.proxy().getSystemConfiguration(userpass).get("test.systemprop"), "whistletips");
-    tpc.proxy().removeProperty(userpass, "test.systemprop");
-    assertNull(tpc.proxy().getSystemConfiguration(userpass).get("test.systemprop"));
-    
-  }
-  
-  @Test
-  public void testClassLoad() throws TException {
-    assertTrue(tpc.proxy().testClassLoad(userpass, "org.apache.accumulo.core.iterators.user.RegExFilter", "org.apache.accumulo.core.iterators.Filter"));
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
deleted file mode 100644
index c0049a0..0000000
--- a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyReadWrite.java
+++ /dev/null
@@ -1,478 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.proxy;
-
-import static org.junit.Assert.assertEquals;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.proxy.thrift.BatchScanOptions;
-import org.apache.accumulo.proxy.thrift.ColumnUpdate;
-import org.apache.accumulo.proxy.thrift.IteratorSetting;
-import org.apache.accumulo.proxy.thrift.Key;
-import org.apache.accumulo.proxy.thrift.KeyValue;
-import org.apache.accumulo.proxy.thrift.Range;
-import org.apache.accumulo.proxy.thrift.ScanColumn;
-import org.apache.accumulo.proxy.thrift.ScanOptions;
-import org.apache.accumulo.proxy.thrift.ScanResult;
-import org.apache.accumulo.proxy.thrift.TimeType;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.server.TServer;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestProxyReadWrite {
-  protected static TServer proxy;
-  protected static Thread thread;
-  protected static TestProxyClient tpc;
-  protected static ByteBuffer userpass;
-  protected static final int port = 10194;
-  protected static final String testtable = "testtable";
-  
-  @SuppressWarnings("serial")
-  @BeforeClass
-  public static void setup() throws Exception {
-    Properties prop = new Properties();
-    prop.setProperty("useMockInstance", "true");
-    prop.put("tokenClass", PasswordToken.class.getName());
-    
-    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
-        port, TCompactProtocol.Factory.class, prop);
-    thread = new Thread() {
-      @Override
-      public void run() {
-        proxy.serve();
-      }
-    };
-    thread.start();
-    tpc = new TestProxyClient("localhost", port);
-    userpass = tpc.proxy().login("root", new TreeMap<String, String>() {{put("password",""); }});
-  }
-  
-  @AfterClass
-  public static void tearDown() throws InterruptedException {
-    proxy.stop();
-    thread.join();
-  }
-  
-  @Before
-  public void makeTestTable() throws Exception {
-    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
-  }
-  
-  @After
-  public void deleteTestTable() throws Exception {
-    tpc.proxy().deleteTable(userpass, testtable);
-  }
-  
-  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String value) {
-    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
-    update.setValue(value.getBytes());
-    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
-  }
-  
-  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String vis, String value) {
-    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
-    update.setValue(value.getBytes());
-    update.setColVisibility(vis.getBytes());
-    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
-  }
-  
-  /**
-   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a range so only the entries between -Inf...5 come back (there should be
-   * 50,000)
-   */
-  @Test
-  public void readWriteBatchOneShotWithRange() throws Exception {
-    int maxInserts = 100000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-      }
-    }
-    
-    Key stop = new Key();
-    stop.setRow("5".getBytes());
-    BatchScanOptions options = new BatchScanOptions();
-    options.ranges = Collections.singletonList(new Range(null, false, stop, false));
-    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      i += kvList.getResultsSize();
-      hasNext = kvList.isMore();
-    }
-    assertEquals(i, 50000);
-  }
-
-  /**
-   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a columnFamily so only the entries with specified column family come back (there should be
-   * 50,000)
-   */
-  @Test
-  public void readWriteBatchOneShotWithColumnFamilyOnly() throws Exception {
-    int maxInserts = 100000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-	
-      addMutation(mutations, String.format(format, i), "cf" + (i % 2) , "cq" + (i % 2), Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-      }
-    }
-    
-    BatchScanOptions options = new BatchScanOptions();
-
-	ScanColumn sc = new ScanColumn();
-	sc.colFamily = ByteBuffer.wrap("cf0".getBytes());
-
-    options.columns = Collections.singletonList(sc);
-    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      i += kvList.getResultsSize();
-      hasNext = kvList.isMore();
-    }
-    assertEquals(i, 50000);
-  }
-
-
-  /**
-   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Set a columnFamily + columnQualififer so only the entries with specified column 
-   * come back (there should be 50,000)
-   */
-  @Test
-  public void readWriteBatchOneShotWithFullColumn() throws Exception {
-    int maxInserts = 100000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-	
-      addMutation(mutations, String.format(format, i), "cf" + (i % 2) , "cq" + (i % 2), Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-      }
-    }
-    
-    BatchScanOptions options = new BatchScanOptions();
-
-	ScanColumn sc = new ScanColumn();
-	sc.colFamily = ByteBuffer.wrap("cf0".getBytes());
-	sc.colQualifier = ByteBuffer.wrap("cq0".getBytes());
-
-    options.columns = Collections.singletonList(sc);
-    String cookie = tpc.proxy().createBatchScanner(userpass, testtable, options);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      i += kvList.getResultsSize();
-      hasNext = kvList.isMore();
-    }
-    assertEquals(i, 50000);
-  }
-
-
-  /**
-   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Filter the results so only the even numbers come back.
-   */
-  @Test
-  public void readWriteBatchOneShotWithFilterIterator() throws Exception {
-    int maxInserts = 10000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-      }
-      
-    }
-    
-    String regex = ".*[02468]";
-    
-    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
-    RegExFilter.setRegexs(is, regex, null, null, null, false);
-    
-    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
-    ScanOptions opts = new ScanOptions();
-    opts.iterators = Collections.singletonList(pis);
-    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      for (KeyValue kv : kvList.getResults()) {
-        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
-        
-        i += 2;
-      }
-      hasNext = kvList.isMore();
-    }
-  }
-  
-  @Test
-  public void readWriteOneShotWithRange() throws Exception {
-    int maxInserts = 100000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-      }
-    }
-    
-    Key stop = new Key();
-    stop.setRow("5".getBytes());
-    ScanOptions opts = new ScanOptions();
-    opts.range = new Range(null, false, stop, false);
-    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      i += kvList.getResultsSize();
-      hasNext = kvList.isMore();
-    }
-    assertEquals(i, 50000);
-  }
-  
-  /**
-   * Insert 100000 cells which have as the row [0..99999] (padded with zeros). Filter the results so only the even numbers come back.
-   */
-  @Test
-  public void readWriteOneShotWithFilterIterator() throws Exception {
-    int maxInserts = 10000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        
-        tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-        mutations.clear();
-        
-      }
-      
-    }
-    
-    String regex = ".*[02468]";
-    
-    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
-    RegExFilter.setRegexs(is, regex, null, null, null, false);
-    
-    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
-    ScanOptions opts = new ScanOptions();
-    opts.iterators = Collections.singletonList(pis);
-    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      for (KeyValue kv : kvList.getResults()) {
-        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
-        
-        i += 2;
-      }
-      hasNext = kvList.isMore();
-    }
-  }
-  
-  // @Test
-  // This test takes kind of a long time. Enable it if you think you may have memory issues.
-  public void manyWritesAndReads() throws Exception {
-    int maxInserts = 1000000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$06d";
-    String writer = tpc.proxy().createWriter(userpass, testtable, null);
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        
-        tpc.proxy().update(writer, mutations);
-        mutations.clear();
-        
-      }
-      
-    }
-    
-    tpc.proxy().flush(writer);
-    tpc.proxy().closeWriter(writer);
-    
-    String cookie = tpc.proxy().createScanner(userpass, testtable, null);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      for (KeyValue kv : kvList.getResults()) {
-        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
-        i++;
-      }
-      hasNext = kvList.isMore();
-      if (hasNext)
-        assertEquals(k, kvList.getResults().size());
-    }
-    assertEquals(maxInserts, i);
-  }
-  
-  @Test
-  public void asynchReadWrite() throws Exception {
-    int maxInserts = 10000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    String writer = tpc.proxy().createWriter(userpass, testtable, null);
-    for (int i = 0; i < maxInserts; i++) {
-      addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().update(writer, mutations);
-        mutations.clear();
-      }
-    }
-    
-    tpc.proxy().flush(writer);
-    tpc.proxy().closeWriter(writer);
-    
-    String regex = ".*[02468]";
-    
-    org.apache.accumulo.core.client.IteratorSetting is = new org.apache.accumulo.core.client.IteratorSetting(50, regex, RegExFilter.class);
-    RegExFilter.setRegexs(is, regex, null, null, null, false);
-    
-    IteratorSetting pis = Util.iteratorSetting2ProxyIteratorSetting(is);
-    ScanOptions opts = new ScanOptions();
-    opts.iterators = Collections.singletonList(pis);
-    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    int numRead = 0;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      for (KeyValue kv : kvList.getResults()) {
-        assertEquals(i, Integer.parseInt(new String(kv.getKey().getRow())));
-        numRead++;
-        i += 2;
-      }
-      hasNext = kvList.isMore();
-    }
-    assertEquals(maxInserts / 2, numRead);
-  }
-  
-  @Test
-  public void testVisibility() throws Exception {
-    
-    Set<ByteBuffer> auths = new HashSet<ByteBuffer>();
-    auths.add(ByteBuffer.wrap("even".getBytes()));
-    tpc.proxy().changeUserAuthorizations(userpass, "root", auths);
-    
-    int maxInserts = 10000;
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    String format = "%1$05d";
-    String writer = tpc.proxy().createWriter(userpass, testtable, null);
-    for (int i = 0; i < maxInserts; i++) {
-      if (i % 2 == 0)
-        addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, "even", Util.randString(10));
-      else
-        addMutation(mutations, String.format(format, i), "cf" + i, "cq" + i, "odd", Util.randString(10));
-      
-      if (i % 1000 == 0 || i == maxInserts - 1) {
-        tpc.proxy().update(writer, mutations);
-        mutations.clear();
-      }
-    }
-    
-    tpc.proxy().flush(writer);
-    tpc.proxy().closeWriter(writer);
-    ScanOptions opts = new ScanOptions();
-    opts.authorizations = auths;
-    String cookie = tpc.proxy().createScanner(userpass, testtable, opts);
-    
-    int i = 0;
-    boolean hasNext = true;
-    
-    int k = 1000;
-    int numRead = 0;
-    while (hasNext) {
-      ScanResult kvList = tpc.proxy().nextK(cookie, k);
-      for (KeyValue kv : kvList.getResults()) {
-        assertEquals(Integer.parseInt(new String(kv.getKey().getRow())), i);
-        i += 2;
-        numRead++;
-      }
-      hasNext = kvList.isMore();
-      
-    }
-    assertEquals(maxInserts / 2, numRead);
-  }
-  
-}


[06/18] git commit: ACCUMULO-1957 rat checks, whitespace, removed debugging log statements

Posted by ec...@apache.org.
ACCUMULO-1957 rat checks, whitespace, removed debugging log statements


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f5b598e9
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f5b598e9
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f5b598e9

Branch: refs/heads/master
Commit: f5b598e989cd7c0b2199bb10bc0fc89ea833757a
Parents: 1a2c8d5
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Sep 3 13:41:56 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/client/Durability.java | 16 ++++++++++
 .../org/apache/accumulo/tserver/Mutations.java  | 16 ++++++++++
 .../apache/accumulo/tserver/log/DfsLogger.java  | 12 ++------
 .../accumulo/test/functional/DurabilityIT.java  | 10 +++---
 .../test/functional/SessionDurabilityIT.java    | 32 +++++++++++++++-----
 5 files changed, 64 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5b598e9/core/src/main/java/org/apache/accumulo/core/client/Durability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Durability.java b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
index e1dbf4a..6de666d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Durability.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.accumulo.core.client;
 
 import org.apache.accumulo.core.tabletserver.thrift.TDurability;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5b598e9/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
index aa4ec81..5ee1952 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.accumulo.tserver;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5b598e9/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index e166a60..d374492 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -146,7 +146,6 @@ public class DfsLogger {
         }
         workQueue.drainTo(work);
 
-        String durability = null;
         Method durabilityMethod = null;
         loop:
         for (LogWork logWork : work) {
@@ -161,12 +160,10 @@ public class DfsLogger {
               break;
             case SYNC:
               durabilityMethod = sync;
-              durability = logWork.durability.toString();
               break loop;
             case FLUSH:
               if (durabilityMethod == null) {
                 durabilityMethod = flush;
-                durability = logWork.durability.toString();
               }
               break;
           }
@@ -174,10 +171,7 @@ public class DfsLogger {
 
         try {
           if (durabilityMethod != null) {
-            log.debug("durability method " + durability);
             durabilityMethod.invoke(logFile);
-          } else {
-            log.debug("skipping flush/sync");
           }
         } catch (Exception ex) {
           log.warn("Exception syncing " + ex);
@@ -534,7 +528,7 @@ public class DfsLogger {
 
     if (durability == Durability.LOG)
       return null;
-    
+
     synchronized (closeLock) {
       // use a different lock for close check so that adding to work queue does not need
       // to wait on walog I/O operations
@@ -558,9 +552,9 @@ public class DfsLogger {
       LogFileValue value = new LogFileValue();
       value.mutations = tabletMutations.getMutations();
       data.add(new Pair<LogFileKey,LogFileValue>(key, value));
-      log.debug("Durability for " + tabletMutations.getDurability() + " (ordinal) " + tabletMutations.getDurability().ordinal() + " durability " + durability + " (ordinal) " + durability.ordinal());
-      if (tabletMutations.getDurability().ordinal() > durability.ordinal())
+      if (tabletMutations.getDurability().ordinal() > durability.ordinal()) {
         durability = tabletMutations.getDurability();
+      }
     }
     return logFileData(data, durability);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5b598e9/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 11f61c7..526f962 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -63,14 +63,14 @@ public class DurabilityIT extends ConfigurableMacIT {
     tableOps.setProperty(tableNames[3], Property.TABLE_DURABILITY.getKey(), "none");
     return tableNames;
   }
-  
+
   private void cleanup(String[] tableNames) throws Exception {
     Connector c = getConnector();
     for (String tableName : tableNames) {
       c.tableOperations().delete(tableName);
     }
   }
-  
+
   private void createTable(String tableName) throws Exception {
     TableOperations tableOps = getConnector().tableOperations();
     tableOps.create(tableName);
@@ -130,7 +130,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     assertTrue(N > readSome(tableNames[3]));
     cleanup(tableNames);
   }
-  
+
   @Test(timeout = 4 * 60 * 1000)
   public void testIncreaseDurability() throws Exception {
     Connector c = getConnector();
@@ -146,7 +146,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     restartTServer();
     assertTrue(N == readSome(tableName));
   }
-  
+
   private static Map<String, String> map(Iterable<Entry<String, String>> entries) {
     Map<String, String> result = new HashMap<String,String>();
     for (Entry<String,String> entry : entries) {
@@ -165,7 +165,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     c.tableOperations().create(tableName);
     props = map(c.tableOperations().getProperties(tableName));
     assertEquals("none", props.get(Property.TABLE_DURABILITY.getKey()));
-    
+
   }
 
   private long readSome(String table) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5b598e9/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
index b0d0b23..58e6007 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.accumulo.test.functional;
 
 import static org.junit.Assert.assertEquals;
@@ -23,14 +39,14 @@ import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.junit.Test;
 
 public class SessionDurabilityIT extends ConfigurableMacIT {
-  
+
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setNumTservers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
   }
-  
+
   @Test(timeout = 3 * 60 * 1000)
   public void nondurableTableHasDurableWrites() throws Exception {
     Connector c = getConnector();
@@ -47,7 +63,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     restartTServer();
     assertEquals(10, count(tableName));
   }
-  
+
   @Test(timeout = 3 * 60 * 1000)
   public void durableTableLosesNonDurableWrites() throws Exception {
     Connector c = getConnector();
@@ -63,7 +79,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     restartTServer();
     assertTrue(10 > count(tableName));
   }
-  
+
   private int count(String tableName) throws Exception {
     return FunctionalTestUtils.count(getConnector().createScanner(tableName, Authorizations.EMPTY));
   }
@@ -78,7 +94,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     }
     bw.close();
   }
-  
+
   @Test(timeout = 3 * 60 * 1000)
   public void testConditionDurability() throws Exception {
     Connector c = getConnector();
@@ -96,7 +112,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     restartTServer();
     assertEquals(0, count(tableName));
   }
-  
+
   @Test(timeout = 3 * 60 * 1000)
   public void testConditionDurability2() throws Exception {
     Connector c = getConnector();
@@ -114,7 +130,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     restartTServer();
     assertEquals(10, count(tableName));
   }
-  
+
   private void conditionWriteSome(String tableName, int n, ConditionalWriterConfig cfg) throws Exception {
     Connector c = getConnector();
     ConditionalWriter cw = c.createConditionalWriter(tableName, cfg);
@@ -124,7 +140,7 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
       assertEquals(Status.ACCEPTED, cw.write(m).getStatus());
     }
   }
-  
+
   private void restartTServer() throws Exception {
     for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
       cluster.killProcess(ServerType.TABLET_SERVER, proc);


[07/18] git commit: ACCUMULO-1957 more updates based on review comments

Posted by ec...@apache.org.
ACCUMULO-1957 more updates based on review comments


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/549d1c25
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/549d1c25
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/549d1c25

Branch: refs/heads/master
Commit: 549d1c25336c4201da802e2b2117bd252636979d
Parents: c2d95a1
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Sep 2 17:27:53 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  |  3 ++
 .../accumulo/test/functional/DurabilityIT.java  | 41 +++++++++++++++++++-
 .../test/functional/SessionDurabilityIT.java    | 13 +++++++
 3 files changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/549d1c25/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index f6829ea..e166a60 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -532,6 +532,9 @@ public class DfsLogger {
       }
     }
 
+    if (durability == Durability.LOG)
+      return null;
+    
     synchronized (closeLock) {
       // use a different lock for close check so that adding to work queue does not need
       // to wait on walog I/O operations

http://git-wip-us.apache.org/repos/asf/accumulo/blob/549d1c25/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 05e3bef..74df598 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -19,6 +19,8 @@ package org.apache.accumulo.test.functional;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -28,6 +30,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
@@ -60,7 +63,6 @@ public class DurabilityIT extends ConfigurableMacIT {
     tableOps.setProperty(tableNames[1], Property.TABLE_DURABILITY.getKey(), "flush");
     tableOps.setProperty(tableNames[2], Property.TABLE_DURABILITY.getKey(), "log");
     tableOps.setProperty(tableNames[3], Property.TABLE_DURABILITY.getKey(), "none");
-    UtilWaitThread.sleep(1000);
     return tableNames;
   }
   
@@ -130,6 +132,43 @@ public class DurabilityIT extends ConfigurableMacIT {
     assertTrue(N > readSome(tableNames[3], N));
     cleanup(tableNames);
   }
+  
+  @Test(timeout = 4 * 60 * 1000)
+  public void testIncreaseDurability() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "none");
+    UtilWaitThread.sleep(1000);
+    writeSome(tableName, N);
+    restartTServer();
+    assertTrue(N > readSome(tableName, N));
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "sync");
+    writeSome(tableName, N);
+    restartTServer();
+    assertTrue(N == readSome(tableName, N));
+  }
+  
+  private static Map<String, String> map(Iterable<Entry<String, String>> entries) {
+    Map<String, String> result = new HashMap<String,String>();
+    for (Entry<String,String> entry : entries) {
+      result.put(entry.getKey(), entry.getValue());
+    }
+    return result;
+  }
+
+  @Test(timeout = 4 * 60 * 1000)
+  public void testMetaDurability() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.instanceOperations().setProperty(Property.TABLE_DURABILITY.getKey(), "none");
+    Map<String, String> props = map(c.tableOperations().getProperties(MetadataTable.NAME));
+    assertEquals("sync", props.get(Property.TABLE_DURABILITY.getKey()));
+    c.tableOperations().create(tableName);
+    props = map(c.tableOperations().getProperties(tableName));
+    assertEquals("none", props.get(Property.TABLE_DURABILITY.getKey()));
+    
+  }
 
   private long readSome(String table, long n) throws Exception {
     long count = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/549d1c25/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
index 1f84327..91041a9 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -44,6 +44,19 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     assertEquals(10, count(tableName));
   }
   
+  @Test
+  public void durableTableLosesNonDurableWrites() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "sync");
+    BatchWriterConfig cfg = new BatchWriterConfig();
+    cfg.setDurability(Durability.NONE);
+    write(tableName, 10, cfg);
+    restartTServer();
+    assertTrue(10 > count(tableName));
+  }
+  
   private int count(String tableName) throws Exception {
     Connector c = getConnector();
     Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);


[10/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
new file mode 100644
index 0000000..e0b17ac
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.proxy.thrift.SystemPermission;
+import org.apache.accumulo.proxy.thrift.TablePermission;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxySecurityOperations {
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10196;
+  protected static final String testtable = "testtable";
+  protected static final String testuser = "VonJines";
+  protected static final ByteBuffer testpw = ByteBuffer.wrap("fiveones".getBytes());
+  
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
+      private static final long serialVersionUID = 1L;
+      
+      {
+        put("password", "");
+      }
+    });
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Before
+  public void makeTestTableAndUser() throws Exception {
+    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
+    tpc.proxy().createLocalUser(userpass, testuser, testpw);
+  }
+  
+  @After
+  public void deleteTestTable() throws Exception {
+    tpc.proxy().deleteTable(userpass, testtable);
+    tpc.proxy().dropLocalUser(userpass, testuser);
+  }
+  
+  @Test
+  public void create() throws TException {
+    tpc.proxy().createLocalUser(userpass, testuser + "2", testpw);
+    assertTrue(tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
+    tpc.proxy().dropLocalUser(userpass, testuser + "2");
+    assertTrue(!tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
+  }
+  
+  @Test
+  public void authenticate() throws TException {
+    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
+    assertFalse(tpc.proxy().authenticateUser(userpass, "EvilUser", bb2pp(testpw)));
+    
+    tpc.proxy().changeLocalUserPassword(userpass, testuser, ByteBuffer.wrap("newpass".getBytes()));
+    assertFalse(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
+    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(ByteBuffer.wrap("newpass".getBytes()))));
+    
+  }
+  
+  @Test
+  public void tablePermissions() throws TException {
+    tpc.proxy().grantTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
+    assertTrue(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
+    
+    tpc.proxy().revokeTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
+    assertFalse(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
+    
+  }
+  
+  @Test
+  public void systemPermissions() throws TException {
+    tpc.proxy().grantSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
+    assertTrue(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
+    
+    tpc.proxy().revokeSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
+    assertFalse(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
+    
+  }
+  
+  @Test
+  public void auths() throws TException {
+    HashSet<ByteBuffer> newauths = new HashSet<ByteBuffer>();
+    newauths.add(ByteBuffer.wrap("BBR".getBytes()));
+    newauths.add(ByteBuffer.wrap("Barney".getBytes()));
+    tpc.proxy().changeUserAuthorizations(userpass, testuser, newauths);
+    List<ByteBuffer> actualauths = tpc.proxy().getUserAuthorizations(userpass, testuser);
+    assertEquals(actualauths.size(), newauths.size());
+    
+    for (ByteBuffer auth : actualauths) {
+      assertTrue(newauths.contains(auth));
+    }
+  }
+  
+  private Map<String,String> bb2pp(ByteBuffer cf) {
+    Map<String,String> toRet = new TreeMap<String,String>();
+    toRet.put("password", ByteBufferUtil.toString(cf));
+    return toRet;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
new file mode 100644
index 0000000..87d3454
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.proxy.thrift.ColumnUpdate;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxyTableOperations {
+  
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10195;
+  protected static final String testtable = "testtable";
+  
+  @SuppressWarnings("serial")
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
+      {
+        put("password", "");
+      }
+    });
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Before
+  public void makeTestTable() throws Exception {
+    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
+  }
+  
+  @After
+  public void deleteTestTable() throws Exception {
+    tpc.proxy().deleteTable(userpass, testtable);
+  }
+  
+  @Test
+  public void createExistsDelete() throws TException {
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().createTable(userpass, "testtable2", true, TimeType.MILLIS);
+    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().deleteTable(userpass, "testtable2");
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+  }
+  
+  @Test
+  public void listRename() throws TException {
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().renameTable(userpass, testtable, "testtable2");
+    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().renameTable(userpass, "testtable2", testtable);
+    assertTrue(tpc.proxy().listTables(userpass).contains("testtable"));
+    
+  }
+  
+  // This test does not yet function because the backing Mock instance does not yet support merging
+  @Test
+  public void merge() throws TException {
+    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
+    splits.add(ByteBuffer.wrap("a".getBytes()));
+    splits.add(ByteBuffer.wrap("c".getBytes()));
+    splits.add(ByteBuffer.wrap("z".getBytes()));
+    tpc.proxy().addSplits(userpass, testtable, splits);
+    
+    tpc.proxy().mergeTablets(userpass, testtable, ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap("d".getBytes()));
+    
+    splits.remove(ByteBuffer.wrap("c".getBytes()));
+    
+    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
+    
+    for (ByteBuffer split : tableSplits)
+      assertTrue(splits.contains(split));
+    assertTrue(tableSplits.size() == splits.size());
+    
+  }
+  
+  @Test
+  public void splits() throws TException {
+    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
+    splits.add(ByteBuffer.wrap("a".getBytes()));
+    splits.add(ByteBuffer.wrap("b".getBytes()));
+    splits.add(ByteBuffer.wrap("z".getBytes()));
+    tpc.proxy().addSplits(userpass, testtable, splits);
+    
+    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
+    
+    for (ByteBuffer split : tableSplits)
+      assertTrue(splits.contains(split));
+    assertTrue(tableSplits.size() == splits.size());
+  }
+  
+  @Test
+  public void constraints() throws TException {
+    int cid = tpc.proxy().addConstraint(userpass, testtable, "org.apache.accumulo.TestConstraint");
+    Map<String,Integer> constraints = tpc.proxy().listConstraints(userpass, testtable);
+    assertEquals((int) constraints.get("org.apache.accumulo.TestConstraint"), cid);
+    tpc.proxy().removeConstraint(userpass, testtable, cid);
+    constraints = tpc.proxy().listConstraints(userpass, testtable);
+    assertNull(constraints.get("org.apache.accumulo.TestConstraint"));
+  }
+  
+  @Test
+  public void localityGroups() throws TException {
+    Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
+    Set<String> group1 = new HashSet<String>();
+    group1.add("cf1");
+    groups.put("group1", group1);
+    Set<String> group2 = new HashSet<String>();
+    group2.add("cf2");
+    group2.add("cf3");
+    groups.put("group2", group2);
+    tpc.proxy().setLocalityGroups(userpass, testtable, groups);
+    
+    Map<String,Set<String>> actualGroups = tpc.proxy().getLocalityGroups(userpass, testtable);
+    
+    assertEquals(groups.size(), actualGroups.size());
+    for (String groupName : groups.keySet()) {
+      assertTrue(actualGroups.containsKey(groupName));
+      assertEquals(groups.get(groupName).size(), actualGroups.get(groupName).size());
+      for (String cf : groups.get(groupName)) {
+        assertTrue(actualGroups.get(groupName).contains(cf));
+      }
+    }
+  }
+  
+  @Test
+  public void tableProperties() throws TException {
+    tpc.proxy().setTableProperty(userpass, testtable, "test.property1", "wharrrgarbl");
+    assertEquals(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"), "wharrrgarbl");
+    tpc.proxy().removeTableProperty(userpass, testtable, "test.property1");
+    assertNull(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"));
+  }
+  
+  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String value) {
+    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
+    update.setValue(value.getBytes());
+    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
+  }
+  
+  @Test
+  public void tableOperationsRowMethods() throws TException {
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    for (int i = 0; i < 10; i++) {
+      addMutation(mutations, "" + i, "cf", "cq", "");
+    }
+    tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+    
+    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("9".getBytes()));
+    
+    tpc.proxy().deleteRows(userpass, testtable, ByteBuffer.wrap("51".getBytes()), ByteBuffer.wrap("99".getBytes()));
+    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("5".getBytes()));
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 8f6b830..6ee671e 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -57,7 +57,7 @@ public class BloomFilterIT extends ConfigurableMacIT {
     siteConfig.put(Property.TABLE_BLOOM_SIZE.getKey(), "2000000");
     siteConfig.put(Property.TABLE_BLOOM_ERRORRATE.getKey(), "1%");
     siteConfig.put(Property.TABLE_BLOOM_LOAD_THRESHOLD.getKey(), "0");
-    siteConfig.put(Property.TSERV_MUTATION_QUEUE_MAX.getKey(), "10M");
+    siteConfig.put(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "10M");
     siteConfig.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "64K");
     cfg.setSiteConfig(siteConfig );
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 526f962..4632083 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -165,7 +165,8 @@ public class DurabilityIT extends ConfigurableMacIT {
     c.tableOperations().create(tableName);
     props = map(c.tableOperations().getProperties(tableName));
     assertEquals("none", props.get(Property.TABLE_DURABILITY.getKey()));
-
+    restartTServer();
+    assertTrue(c.tableOperations().exists(tableName));
   }
 
   private long readSome(String table) throws Exception {


[04/18] git commit: ACCUMULO-1957 whitespace

Posted by ec...@apache.org.
ACCUMULO-1957 whitespace


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/a25796c4
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/a25796c4
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/a25796c4

Branch: refs/heads/master
Commit: a25796c4b18734ad87cb5e5df64a15574647fd13
Parents: e3aa7ea
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Aug 27 15:38:42 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:58 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/core/conf/Property.java |  6 +++---
 .../apache/accumulo/core/conf/PropertyType.java |  2 +-
 .../accumulo/tserver/TabletMutations.java       |  6 +++---
 .../apache/accumulo/tserver/log/DfsLogger.java  |  8 ++++----
 .../accumulo/tserver/tablet/Durability.java     |  4 ++--
 .../accumulo/test/functional/DurabilityIT.java  | 20 ++++++++++----------
 6 files changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 72d9aa1..018e83f 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -390,9 +390,9 @@ public enum Property {
           + ",org.apache.accumulo.core.file.keyfunctor.ColumnFamilyFunctor, and org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor are"
           + " allowable values. One can extend any of the above mentioned classes to perform specialized parsing of the key. "),
   TABLE_BLOOM_HASHTYPE("table.bloom.hash.type", "murmur", PropertyType.STRING, "The bloom filter hash type"),
-  TABLE_DURABILITY("table.durability", "sync", PropertyType.DURABILITY, "The durability used to write to the write-ahead log." + 
-      " Legal values are: none, which skips the write-ahead log; " + 
-      "flush, which pushes data to the file system; and " + 
+  TABLE_DURABILITY("table.durability", "sync", PropertyType.DURABILITY, "The durability used to write to the write-ahead log." +
+      " Legal values are: none, which skips the write-ahead log; " +
+      "flush, which pushes data to the file system; and " +
       "sync, which ensures the data is written to disk."),
   TABLE_FAILURES_IGNORE("table.failures.ignore", "false", PropertyType.BOOLEAN,
       "If you want queries for your table to hang or fail when data is missing from the system, "

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index 5d5dd5f..00202cf 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -67,7 +67,7 @@ public enum PropertyType {
 
   CLASSNAME("java class", "[\\w$.]*", "A fully qualified java class name representing a class on the classpath.\n"
       + "An example is 'java.lang.String', rather than 'String'"),
-      
+
   DURABILITY("durability", "(?:none|log|flush|sync)", "One of 'none', 'flush' or 'sync'."),
 
   STRING("string", ".*",

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
index a30fa02..edd12aa 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
@@ -41,13 +41,13 @@ public class TabletMutations {
   public int getTid() {
     return tid;
   }
-  
+
   public int getSeq() {
     return seq;
   }
-  
+
   public Durability getDurability() {
     return durability;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index d907ee7..443ba2e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -69,7 +69,7 @@ import com.google.common.base.Joiner;
 
 /**
  * Wrap a connection to a logger.
- * 
+ *
  */
 public class DfsLogger {
   // Package private so that LogSorter can find this
@@ -146,7 +146,7 @@ public class DfsLogger {
           continue;
         }
         workQueue.drainTo(work);
-        
+
         Method durabilityMethod = null;
         loop:
         for (LogWork logWork : work) {
@@ -238,7 +238,7 @@ public class DfsLogger {
     // filename is unique
     return getFileName().hashCode();
   }
-  
+
   private final ServerResources conf;
   private FSDataOutputStream logFile;
   private DataOutputStream encryptingLogFile = null;
@@ -474,7 +474,7 @@ public class DfsLogger {
       log.error("WAL work queue not empty after sync thread exited");
       throw new IllegalStateException("WAL work queue not empty after sync thread exited");
     }
-    
+
     if (encryptingLogFile != null)
       try {
         logFile.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
index 675b196..a88e377 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Durability.java
@@ -21,7 +21,7 @@ public enum Durability {
   LOG,
   FLUSH,
   SYNC;
-  
+
   static public Durability fromString(String value) {
     try {
       return Durability.valueOf(value.toUpperCase());
@@ -29,5 +29,5 @@ public enum Durability {
       return Durability.SYNC;
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a25796c4/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index b4d9c83..590c7c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -37,24 +37,24 @@ import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
 public class DurabilityIT extends ConfigurableMacIT {
-  
+
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.useMiniDFS(true);
     cfg.setNumTservers(1);
   }
-  
+
   static final long N = 100000;
-  
+
   String tableNames[] = null;
-  
+
   void init() throws Exception {
     synchronized (this) {
       if (tableNames == null) {
         tableNames = getUniqueNames(4);
         Connector c = getConnector();
         TableOperations tableOps = c.tableOperations();
-        tableOps.create(tableNames[0]); 
+        tableOps.create(tableNames[0]);
         tableOps.create(tableNames[1]);
         tableOps.create(tableNames[2]);
         tableOps.create(tableNames[3]);
@@ -81,7 +81,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     assertTrue(t1 > t2);
     assertTrue(t2 > t3);
   }
-  
+
   @Test(timeout = 4 * 60 * 1000)
   public void testSync() throws Exception {
     init();
@@ -96,7 +96,7 @@ public class DurabilityIT extends ConfigurableMacIT {
   public void testFlush() throws Exception {
     init();
     // flush table won't lose anything since we're not losing power/dfs
-    getConnector().tableOperations().deleteRows(tableNames[1], null, null); 
+    getConnector().tableOperations().deleteRows(tableNames[1], null, null);
     writeSome(tableNames[1], N);
     restartTServer();
     assertEquals(N, readSome(tableNames[1], N));
@@ -106,12 +106,12 @@ public class DurabilityIT extends ConfigurableMacIT {
   public void testLog() throws Exception {
     init();
     // we're probably going to lose something the the log setting
-    getConnector().tableOperations().deleteRows(tableNames[2], null, null); 
+    getConnector().tableOperations().deleteRows(tableNames[2], null, null);
     writeSome(tableNames[2], N);
     restartTServer();
     assertTrue(N > readSome(tableNames[2], N));
   }
-  
+
   @Test(timeout = 4 * 60 * 1000)
   public void testNone() throws Exception {
     init();
@@ -159,5 +159,5 @@ public class DurabilityIT extends ConfigurableMacIT {
     c.tableOperations().flush(table, null, null, true);
     return result;
   }
-  
+
 }


[15/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/main/cpp/AccumuloProxy.cpp
----------------------------------------------------------------------
diff --git a/proxy/src/main/cpp/AccumuloProxy.cpp b/proxy/src/main/cpp/AccumuloProxy.cpp
index 083c3ca..2667770 100644
--- a/proxy/src/main/cpp/AccumuloProxy.cpp
+++ b/proxy/src/main/cpp/AccumuloProxy.cpp
@@ -56,17 +56,17 @@ uint32_t AccumuloProxy_login_args::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->loginProperties.clear();
-            uint32_t _size123;
-            ::apache::thrift::protocol::TType _ktype124;
-            ::apache::thrift::protocol::TType _vtype125;
-            xfer += iprot->readMapBegin(_ktype124, _vtype125, _size123);
-            uint32_t _i127;
-            for (_i127 = 0; _i127 < _size123; ++_i127)
+            uint32_t _size125;
+            ::apache::thrift::protocol::TType _ktype126;
+            ::apache::thrift::protocol::TType _vtype127;
+            xfer += iprot->readMapBegin(_ktype126, _vtype127, _size125);
+            uint32_t _i129;
+            for (_i129 = 0; _i129 < _size125; ++_i129)
             {
-              std::string _key128;
-              xfer += iprot->readString(_key128);
-              std::string& _val129 = this->loginProperties[_key128];
-              xfer += iprot->readString(_val129);
+              std::string _key130;
+              xfer += iprot->readString(_key130);
+              std::string& _val131 = this->loginProperties[_key130];
+              xfer += iprot->readString(_val131);
             }
             xfer += iprot->readMapEnd();
           }
@@ -98,11 +98,11 @@ uint32_t AccumuloProxy_login_args::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("loginProperties", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->loginProperties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter130;
-    for (_iter130 = this->loginProperties.begin(); _iter130 != this->loginProperties.end(); ++_iter130)
+    std::map<std::string, std::string> ::const_iterator _iter132;
+    for (_iter132 = this->loginProperties.begin(); _iter132 != this->loginProperties.end(); ++_iter132)
     {
-      xfer += oprot->writeString(_iter130->first);
-      xfer += oprot->writeString(_iter130->second);
+      xfer += oprot->writeString(_iter132->first);
+      xfer += oprot->writeString(_iter132->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -124,11 +124,11 @@ uint32_t AccumuloProxy_login_pargs::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("loginProperties", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->loginProperties)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter131;
-    for (_iter131 = (*(this->loginProperties)).begin(); _iter131 != (*(this->loginProperties)).end(); ++_iter131)
+    std::map<std::string, std::string> ::const_iterator _iter133;
+    for (_iter133 = (*(this->loginProperties)).begin(); _iter133 != (*(this->loginProperties)).end(); ++_iter133)
     {
-      xfer += oprot->writeString(_iter131->first);
-      xfer += oprot->writeString(_iter131->second);
+      xfer += oprot->writeString(_iter133->first);
+      xfer += oprot->writeString(_iter133->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -549,15 +549,15 @@ uint32_t AccumuloProxy_addSplits_args::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->splits.clear();
-            uint32_t _size132;
-            ::apache::thrift::protocol::TType _etype135;
-            xfer += iprot->readSetBegin(_etype135, _size132);
-            uint32_t _i136;
-            for (_i136 = 0; _i136 < _size132; ++_i136)
+            uint32_t _size134;
+            ::apache::thrift::protocol::TType _etype137;
+            xfer += iprot->readSetBegin(_etype137, _size134);
+            uint32_t _i138;
+            for (_i138 = 0; _i138 < _size134; ++_i138)
             {
-              std::string _elem137;
-              xfer += iprot->readBinary(_elem137);
-              this->splits.insert(_elem137);
+              std::string _elem139;
+              xfer += iprot->readBinary(_elem139);
+              this->splits.insert(_elem139);
             }
             xfer += iprot->readSetEnd();
           }
@@ -593,10 +593,10 @@ uint32_t AccumuloProxy_addSplits_args::write(::apache::thrift::protocol::TProtoc
   xfer += oprot->writeFieldBegin("splits", ::apache::thrift::protocol::T_SET, 3);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->splits.size()));
-    std::set<std::string> ::const_iterator _iter138;
-    for (_iter138 = this->splits.begin(); _iter138 != this->splits.end(); ++_iter138)
+    std::set<std::string> ::const_iterator _iter140;
+    for (_iter140 = this->splits.begin(); _iter140 != this->splits.end(); ++_iter140)
     {
-      xfer += oprot->writeBinary((*_iter138));
+      xfer += oprot->writeBinary((*_iter140));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -622,10 +622,10 @@ uint32_t AccumuloProxy_addSplits_pargs::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("splits", ::apache::thrift::protocol::T_SET, 3);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->splits)).size()));
-    std::set<std::string> ::const_iterator _iter139;
-    for (_iter139 = (*(this->splits)).begin(); _iter139 != (*(this->splits)).end(); ++_iter139)
+    std::set<std::string> ::const_iterator _iter141;
+    for (_iter141 = (*(this->splits)).begin(); _iter141 != (*(this->splits)).end(); ++_iter141)
     {
-      xfer += oprot->writeBinary((*_iter139));
+      xfer += oprot->writeBinary((*_iter141));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -820,17 +820,17 @@ uint32_t AccumuloProxy_attachIterator_args::read(::apache::thrift::protocol::TPr
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->scopes.clear();
-            uint32_t _size140;
-            ::apache::thrift::protocol::TType _etype143;
-            xfer += iprot->readSetBegin(_etype143, _size140);
-            uint32_t _i144;
-            for (_i144 = 0; _i144 < _size140; ++_i144)
+            uint32_t _size142;
+            ::apache::thrift::protocol::TType _etype145;
+            xfer += iprot->readSetBegin(_etype145, _size142);
+            uint32_t _i146;
+            for (_i146 = 0; _i146 < _size142; ++_i146)
             {
-              IteratorScope::type _elem145;
-              int32_t ecast146;
-              xfer += iprot->readI32(ecast146);
-              _elem145 = (IteratorScope::type)ecast146;
-              this->scopes.insert(_elem145);
+              IteratorScope::type _elem147;
+              int32_t ecast148;
+              xfer += iprot->readI32(ecast148);
+              _elem147 = (IteratorScope::type)ecast148;
+              this->scopes.insert(_elem147);
             }
             xfer += iprot->readSetEnd();
           }
@@ -870,10 +870,10 @@ uint32_t AccumuloProxy_attachIterator_args::write(::apache::thrift::protocol::TP
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->scopes.size()));
-    std::set<IteratorScope::type> ::const_iterator _iter147;
-    for (_iter147 = this->scopes.begin(); _iter147 != this->scopes.end(); ++_iter147)
+    std::set<IteratorScope::type> ::const_iterator _iter149;
+    for (_iter149 = this->scopes.begin(); _iter149 != this->scopes.end(); ++_iter149)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter147));
+      xfer += oprot->writeI32((int32_t)(*_iter149));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -903,10 +903,10 @@ uint32_t AccumuloProxy_attachIterator_pargs::write(::apache::thrift::protocol::T
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>((*(this->scopes)).size()));
-    std::set<IteratorScope::type> ::const_iterator _iter148;
-    for (_iter148 = (*(this->scopes)).begin(); _iter148 != (*(this->scopes)).end(); ++_iter148)
+    std::set<IteratorScope::type> ::const_iterator _iter150;
+    for (_iter150 = (*(this->scopes)).begin(); _iter150 != (*(this->scopes)).end(); ++_iter150)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter148));
+      xfer += oprot->writeI32((int32_t)(*_iter150));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -1101,17 +1101,17 @@ uint32_t AccumuloProxy_checkIteratorConflicts_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->scopes.clear();
-            uint32_t _size149;
-            ::apache::thrift::protocol::TType _etype152;
-            xfer += iprot->readSetBegin(_etype152, _size149);
-            uint32_t _i153;
-            for (_i153 = 0; _i153 < _size149; ++_i153)
+            uint32_t _size151;
+            ::apache::thrift::protocol::TType _etype154;
+            xfer += iprot->readSetBegin(_etype154, _size151);
+            uint32_t _i155;
+            for (_i155 = 0; _i155 < _size151; ++_i155)
             {
-              IteratorScope::type _elem154;
-              int32_t ecast155;
-              xfer += iprot->readI32(ecast155);
-              _elem154 = (IteratorScope::type)ecast155;
-              this->scopes.insert(_elem154);
+              IteratorScope::type _elem156;
+              int32_t ecast157;
+              xfer += iprot->readI32(ecast157);
+              _elem156 = (IteratorScope::type)ecast157;
+              this->scopes.insert(_elem156);
             }
             xfer += iprot->readSetEnd();
           }
@@ -1151,10 +1151,10 @@ uint32_t AccumuloProxy_checkIteratorConflicts_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->scopes.size()));
-    std::set<IteratorScope::type> ::const_iterator _iter156;
-    for (_iter156 = this->scopes.begin(); _iter156 != this->scopes.end(); ++_iter156)
+    std::set<IteratorScope::type> ::const_iterator _iter158;
+    for (_iter158 = this->scopes.begin(); _iter158 != this->scopes.end(); ++_iter158)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter156));
+      xfer += oprot->writeI32((int32_t)(*_iter158));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -1184,10 +1184,10 @@ uint32_t AccumuloProxy_checkIteratorConflicts_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>((*(this->scopes)).size()));
-    std::set<IteratorScope::type> ::const_iterator _iter157;
-    for (_iter157 = (*(this->scopes)).begin(); _iter157 != (*(this->scopes)).end(); ++_iter157)
+    std::set<IteratorScope::type> ::const_iterator _iter159;
+    for (_iter159 = (*(this->scopes)).begin(); _iter159 != (*(this->scopes)).end(); ++_iter159)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter157));
+      xfer += oprot->writeI32((int32_t)(*_iter159));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -1568,17 +1568,17 @@ uint32_t AccumuloProxy_cloneTable_args::read(::apache::thrift::protocol::TProtoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->propertiesToSet.clear();
-            uint32_t _size158;
-            ::apache::thrift::protocol::TType _ktype159;
-            ::apache::thrift::protocol::TType _vtype160;
-            xfer += iprot->readMapBegin(_ktype159, _vtype160, _size158);
-            uint32_t _i162;
-            for (_i162 = 0; _i162 < _size158; ++_i162)
+            uint32_t _size160;
+            ::apache::thrift::protocol::TType _ktype161;
+            ::apache::thrift::protocol::TType _vtype162;
+            xfer += iprot->readMapBegin(_ktype161, _vtype162, _size160);
+            uint32_t _i164;
+            for (_i164 = 0; _i164 < _size160; ++_i164)
             {
-              std::string _key163;
-              xfer += iprot->readString(_key163);
-              std::string& _val164 = this->propertiesToSet[_key163];
-              xfer += iprot->readString(_val164);
+              std::string _key165;
+              xfer += iprot->readString(_key165);
+              std::string& _val166 = this->propertiesToSet[_key165];
+              xfer += iprot->readString(_val166);
             }
             xfer += iprot->readMapEnd();
           }
@@ -1591,15 +1591,15 @@ uint32_t AccumuloProxy_cloneTable_args::read(::apache::thrift::protocol::TProtoc
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->propertiesToExclude.clear();
-            uint32_t _size165;
-            ::apache::thrift::protocol::TType _etype168;
-            xfer += iprot->readSetBegin(_etype168, _size165);
-            uint32_t _i169;
-            for (_i169 = 0; _i169 < _size165; ++_i169)
+            uint32_t _size167;
+            ::apache::thrift::protocol::TType _etype170;
+            xfer += iprot->readSetBegin(_etype170, _size167);
+            uint32_t _i171;
+            for (_i171 = 0; _i171 < _size167; ++_i171)
             {
-              std::string _elem170;
-              xfer += iprot->readString(_elem170);
-              this->propertiesToExclude.insert(_elem170);
+              std::string _elem172;
+              xfer += iprot->readString(_elem172);
+              this->propertiesToExclude.insert(_elem172);
             }
             xfer += iprot->readSetEnd();
           }
@@ -1643,11 +1643,11 @@ uint32_t AccumuloProxy_cloneTable_args::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("propertiesToSet", ::apache::thrift::protocol::T_MAP, 5);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->propertiesToSet.size()));
-    std::map<std::string, std::string> ::const_iterator _iter171;
-    for (_iter171 = this->propertiesToSet.begin(); _iter171 != this->propertiesToSet.end(); ++_iter171)
+    std::map<std::string, std::string> ::const_iterator _iter173;
+    for (_iter173 = this->propertiesToSet.begin(); _iter173 != this->propertiesToSet.end(); ++_iter173)
     {
-      xfer += oprot->writeString(_iter171->first);
-      xfer += oprot->writeString(_iter171->second);
+      xfer += oprot->writeString(_iter173->first);
+      xfer += oprot->writeString(_iter173->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -1656,10 +1656,10 @@ uint32_t AccumuloProxy_cloneTable_args::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("propertiesToExclude", ::apache::thrift::protocol::T_SET, 6);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->propertiesToExclude.size()));
-    std::set<std::string> ::const_iterator _iter172;
-    for (_iter172 = this->propertiesToExclude.begin(); _iter172 != this->propertiesToExclude.end(); ++_iter172)
+    std::set<std::string> ::const_iterator _iter174;
+    for (_iter174 = this->propertiesToExclude.begin(); _iter174 != this->propertiesToExclude.end(); ++_iter174)
     {
-      xfer += oprot->writeString((*_iter172));
+      xfer += oprot->writeString((*_iter174));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -1693,11 +1693,11 @@ uint32_t AccumuloProxy_cloneTable_pargs::write(::apache::thrift::protocol::TProt
   xfer += oprot->writeFieldBegin("propertiesToSet", ::apache::thrift::protocol::T_MAP, 5);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->propertiesToSet)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter173;
-    for (_iter173 = (*(this->propertiesToSet)).begin(); _iter173 != (*(this->propertiesToSet)).end(); ++_iter173)
+    std::map<std::string, std::string> ::const_iterator _iter175;
+    for (_iter175 = (*(this->propertiesToSet)).begin(); _iter175 != (*(this->propertiesToSet)).end(); ++_iter175)
     {
-      xfer += oprot->writeString(_iter173->first);
-      xfer += oprot->writeString(_iter173->second);
+      xfer += oprot->writeString(_iter175->first);
+      xfer += oprot->writeString(_iter175->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -1706,10 +1706,10 @@ uint32_t AccumuloProxy_cloneTable_pargs::write(::apache::thrift::protocol::TProt
   xfer += oprot->writeFieldBegin("propertiesToExclude", ::apache::thrift::protocol::T_SET, 6);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->propertiesToExclude)).size()));
-    std::set<std::string> ::const_iterator _iter174;
-    for (_iter174 = (*(this->propertiesToExclude)).begin(); _iter174 != (*(this->propertiesToExclude)).end(); ++_iter174)
+    std::set<std::string> ::const_iterator _iter176;
+    for (_iter176 = (*(this->propertiesToExclude)).begin(); _iter176 != (*(this->propertiesToExclude)).end(); ++_iter176)
     {
-      xfer += oprot->writeString((*_iter174));
+      xfer += oprot->writeString((*_iter176));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -1932,14 +1932,14 @@ uint32_t AccumuloProxy_compactTable_args::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->iterators.clear();
-            uint32_t _size175;
-            ::apache::thrift::protocol::TType _etype178;
-            xfer += iprot->readListBegin(_etype178, _size175);
-            this->iterators.resize(_size175);
-            uint32_t _i179;
-            for (_i179 = 0; _i179 < _size175; ++_i179)
+            uint32_t _size177;
+            ::apache::thrift::protocol::TType _etype180;
+            xfer += iprot->readListBegin(_etype180, _size177);
+            this->iterators.resize(_size177);
+            uint32_t _i181;
+            for (_i181 = 0; _i181 < _size177; ++_i181)
             {
-              xfer += this->iterators[_i179].read(iprot);
+              xfer += this->iterators[_i181].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1999,10 +1999,10 @@ uint32_t AccumuloProxy_compactTable_args::write(::apache::thrift::protocol::TPro
   xfer += oprot->writeFieldBegin("iterators", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->iterators.size()));
-    std::vector<IteratorSetting> ::const_iterator _iter180;
-    for (_iter180 = this->iterators.begin(); _iter180 != this->iterators.end(); ++_iter180)
+    std::vector<IteratorSetting> ::const_iterator _iter182;
+    for (_iter182 = this->iterators.begin(); _iter182 != this->iterators.end(); ++_iter182)
     {
-      xfer += (*_iter180).write(oprot);
+      xfer += (*_iter182).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2044,10 +2044,10 @@ uint32_t AccumuloProxy_compactTable_pargs::write(::apache::thrift::protocol::TPr
   xfer += oprot->writeFieldBegin("iterators", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->iterators)).size()));
-    std::vector<IteratorSetting> ::const_iterator _iter181;
-    for (_iter181 = (*(this->iterators)).begin(); _iter181 != (*(this->iterators)).end(); ++_iter181)
+    std::vector<IteratorSetting> ::const_iterator _iter183;
+    for (_iter183 = (*(this->iterators)).begin(); _iter183 != (*(this->iterators)).end(); ++_iter183)
     {
-      xfer += (*_iter181).write(oprot);
+      xfer += (*_iter183).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2466,9 +2466,9 @@ uint32_t AccumuloProxy_createTable_args::read(::apache::thrift::protocol::TProto
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast182;
-          xfer += iprot->readI32(ecast182);
-          this->type = (TimeType::type)ecast182;
+          int32_t ecast184;
+          xfer += iprot->readI32(ecast184);
+          this->type = (TimeType::type)ecast184;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3672,15 +3672,15 @@ uint32_t AccumuloProxy_getDiskUsage_args::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->tables.clear();
-            uint32_t _size183;
-            ::apache::thrift::protocol::TType _etype186;
-            xfer += iprot->readSetBegin(_etype186, _size183);
-            uint32_t _i187;
-            for (_i187 = 0; _i187 < _size183; ++_i187)
+            uint32_t _size185;
+            ::apache::thrift::protocol::TType _etype188;
+            xfer += iprot->readSetBegin(_etype188, _size185);
+            uint32_t _i189;
+            for (_i189 = 0; _i189 < _size185; ++_i189)
             {
-              std::string _elem188;
-              xfer += iprot->readString(_elem188);
-              this->tables.insert(_elem188);
+              std::string _elem190;
+              xfer += iprot->readString(_elem190);
+              this->tables.insert(_elem190);
             }
             xfer += iprot->readSetEnd();
           }
@@ -3712,10 +3712,10 @@ uint32_t AccumuloProxy_getDiskUsage_args::write(::apache::thrift::protocol::TPro
   xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tables.size()));
-    std::set<std::string> ::const_iterator _iter189;
-    for (_iter189 = this->tables.begin(); _iter189 != this->tables.end(); ++_iter189)
+    std::set<std::string> ::const_iterator _iter191;
+    for (_iter191 = this->tables.begin(); _iter191 != this->tables.end(); ++_iter191)
     {
-      xfer += oprot->writeString((*_iter189));
+      xfer += oprot->writeString((*_iter191));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -3737,10 +3737,10 @@ uint32_t AccumuloProxy_getDiskUsage_pargs::write(::apache::thrift::protocol::TPr
   xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tables)).size()));
-    std::set<std::string> ::const_iterator _iter190;
-    for (_iter190 = (*(this->tables)).begin(); _iter190 != (*(this->tables)).end(); ++_iter190)
+    std::set<std::string> ::const_iterator _iter192;
+    for (_iter192 = (*(this->tables)).begin(); _iter192 != (*(this->tables)).end(); ++_iter192)
     {
-      xfer += oprot->writeString((*_iter190));
+      xfer += oprot->writeString((*_iter192));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -3775,14 +3775,14 @@ uint32_t AccumuloProxy_getDiskUsage_result::read(::apache::thrift::protocol::TPr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size191;
-            ::apache::thrift::protocol::TType _etype194;
-            xfer += iprot->readListBegin(_etype194, _size191);
-            this->success.resize(_size191);
-            uint32_t _i195;
-            for (_i195 = 0; _i195 < _size191; ++_i195)
+            uint32_t _size193;
+            ::apache::thrift::protocol::TType _etype196;
+            xfer += iprot->readListBegin(_etype196, _size193);
+            this->success.resize(_size193);
+            uint32_t _i197;
+            for (_i197 = 0; _i197 < _size193; ++_i197)
             {
-              xfer += this->success[_i195].read(iprot);
+              xfer += this->success[_i197].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3837,10 +3837,10 @@ uint32_t AccumuloProxy_getDiskUsage_result::write(::apache::thrift::protocol::TP
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<DiskUsage> ::const_iterator _iter196;
-      for (_iter196 = this->success.begin(); _iter196 != this->success.end(); ++_iter196)
+      std::vector<DiskUsage> ::const_iterator _iter198;
+      for (_iter198 = this->success.begin(); _iter198 != this->success.end(); ++_iter198)
       {
-        xfer += (*_iter196).write(oprot);
+        xfer += (*_iter198).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3887,14 +3887,14 @@ uint32_t AccumuloProxy_getDiskUsage_presult::read(::apache::thrift::protocol::TP
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size197;
-            ::apache::thrift::protocol::TType _etype200;
-            xfer += iprot->readListBegin(_etype200, _size197);
-            (*(this->success)).resize(_size197);
-            uint32_t _i201;
-            for (_i201 = 0; _i201 < _size197; ++_i201)
+            uint32_t _size199;
+            ::apache::thrift::protocol::TType _etype202;
+            xfer += iprot->readListBegin(_etype202, _size199);
+            (*(this->success)).resize(_size199);
+            uint32_t _i203;
+            for (_i203 = 0; _i203 < _size199; ++_i203)
             {
-              xfer += (*(this->success))[_i201].read(iprot);
+              xfer += (*(this->success))[_i203].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4045,27 +4045,27 @@ uint32_t AccumuloProxy_getLocalityGroups_result::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size202;
-            ::apache::thrift::protocol::TType _ktype203;
-            ::apache::thrift::protocol::TType _vtype204;
-            xfer += iprot->readMapBegin(_ktype203, _vtype204, _size202);
-            uint32_t _i206;
-            for (_i206 = 0; _i206 < _size202; ++_i206)
+            uint32_t _size204;
+            ::apache::thrift::protocol::TType _ktype205;
+            ::apache::thrift::protocol::TType _vtype206;
+            xfer += iprot->readMapBegin(_ktype205, _vtype206, _size204);
+            uint32_t _i208;
+            for (_i208 = 0; _i208 < _size204; ++_i208)
             {
-              std::string _key207;
-              xfer += iprot->readString(_key207);
-              std::set<std::string> & _val208 = this->success[_key207];
+              std::string _key209;
+              xfer += iprot->readString(_key209);
+              std::set<std::string> & _val210 = this->success[_key209];
               {
-                _val208.clear();
-                uint32_t _size209;
-                ::apache::thrift::protocol::TType _etype212;
-                xfer += iprot->readSetBegin(_etype212, _size209);
-                uint32_t _i213;
-                for (_i213 = 0; _i213 < _size209; ++_i213)
+                _val210.clear();
+                uint32_t _size211;
+                ::apache::thrift::protocol::TType _etype214;
+                xfer += iprot->readSetBegin(_etype214, _size211);
+                uint32_t _i215;
+                for (_i215 = 0; _i215 < _size211; ++_i215)
                 {
-                  std::string _elem214;
-                  xfer += iprot->readString(_elem214);
-                  _val208.insert(_elem214);
+                  std::string _elem216;
+                  xfer += iprot->readString(_elem216);
+                  _val210.insert(_elem216);
                 }
                 xfer += iprot->readSetEnd();
               }
@@ -4123,16 +4123,16 @@ uint32_t AccumuloProxy_getLocalityGroups_result::write(::apache::thrift::protoco
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_SET, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::set<std::string> > ::const_iterator _iter215;
-      for (_iter215 = this->success.begin(); _iter215 != this->success.end(); ++_iter215)
+      std::map<std::string, std::set<std::string> > ::const_iterator _iter217;
+      for (_iter217 = this->success.begin(); _iter217 != this->success.end(); ++_iter217)
       {
-        xfer += oprot->writeString(_iter215->first);
+        xfer += oprot->writeString(_iter217->first);
         {
-          xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter215->second.size()));
-          std::set<std::string> ::const_iterator _iter216;
-          for (_iter216 = _iter215->second.begin(); _iter216 != _iter215->second.end(); ++_iter216)
+          xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter217->second.size()));
+          std::set<std::string> ::const_iterator _iter218;
+          for (_iter218 = _iter217->second.begin(); _iter218 != _iter217->second.end(); ++_iter218)
           {
-            xfer += oprot->writeString((*_iter216));
+            xfer += oprot->writeString((*_iter218));
           }
           xfer += oprot->writeSetEnd();
         }
@@ -4182,27 +4182,27 @@ uint32_t AccumuloProxy_getLocalityGroups_presult::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size217;
-            ::apache::thrift::protocol::TType _ktype218;
-            ::apache::thrift::protocol::TType _vtype219;
-            xfer += iprot->readMapBegin(_ktype218, _vtype219, _size217);
-            uint32_t _i221;
-            for (_i221 = 0; _i221 < _size217; ++_i221)
+            uint32_t _size219;
+            ::apache::thrift::protocol::TType _ktype220;
+            ::apache::thrift::protocol::TType _vtype221;
+            xfer += iprot->readMapBegin(_ktype220, _vtype221, _size219);
+            uint32_t _i223;
+            for (_i223 = 0; _i223 < _size219; ++_i223)
             {
-              std::string _key222;
-              xfer += iprot->readString(_key222);
-              std::set<std::string> & _val223 = (*(this->success))[_key222];
+              std::string _key224;
+              xfer += iprot->readString(_key224);
+              std::set<std::string> & _val225 = (*(this->success))[_key224];
               {
-                _val223.clear();
-                uint32_t _size224;
-                ::apache::thrift::protocol::TType _etype227;
-                xfer += iprot->readSetBegin(_etype227, _size224);
-                uint32_t _i228;
-                for (_i228 = 0; _i228 < _size224; ++_i228)
+                _val225.clear();
+                uint32_t _size226;
+                ::apache::thrift::protocol::TType _etype229;
+                xfer += iprot->readSetBegin(_etype229, _size226);
+                uint32_t _i230;
+                for (_i230 = 0; _i230 < _size226; ++_i230)
                 {
-                  std::string _elem229;
-                  xfer += iprot->readString(_elem229);
-                  _val223.insert(_elem229);
+                  std::string _elem231;
+                  xfer += iprot->readString(_elem231);
+                  _val225.insert(_elem231);
                 }
                 xfer += iprot->readSetEnd();
               }
@@ -4296,9 +4296,9 @@ uint32_t AccumuloProxy_getIteratorSetting_args::read(::apache::thrift::protocol:
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast230;
-          xfer += iprot->readI32(ecast230);
-          this->scope = (IteratorScope::type)ecast230;
+          int32_t ecast232;
+          xfer += iprot->readI32(ecast232);
+          this->scope = (IteratorScope::type)ecast232;
           this->__isset.scope = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -4562,15 +4562,15 @@ uint32_t AccumuloProxy_getMaxRow_args::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->auths.clear();
-            uint32_t _size231;
-            ::apache::thrift::protocol::TType _etype234;
-            xfer += iprot->readSetBegin(_etype234, _size231);
-            uint32_t _i235;
-            for (_i235 = 0; _i235 < _size231; ++_i235)
+            uint32_t _size233;
+            ::apache::thrift::protocol::TType _etype236;
+            xfer += iprot->readSetBegin(_etype236, _size233);
+            uint32_t _i237;
+            for (_i237 = 0; _i237 < _size233; ++_i237)
             {
-              std::string _elem236;
-              xfer += iprot->readBinary(_elem236);
-              this->auths.insert(_elem236);
+              std::string _elem238;
+              xfer += iprot->readBinary(_elem238);
+              this->auths.insert(_elem238);
             }
             xfer += iprot->readSetEnd();
           }
@@ -4638,10 +4638,10 @@ uint32_t AccumuloProxy_getMaxRow_args::write(::apache::thrift::protocol::TProtoc
   xfer += oprot->writeFieldBegin("auths", ::apache::thrift::protocol::T_SET, 3);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->auths.size()));
-    std::set<std::string> ::const_iterator _iter237;
-    for (_iter237 = this->auths.begin(); _iter237 != this->auths.end(); ++_iter237)
+    std::set<std::string> ::const_iterator _iter239;
+    for (_iter239 = this->auths.begin(); _iter239 != this->auths.end(); ++_iter239)
     {
-      xfer += oprot->writeBinary((*_iter237));
+      xfer += oprot->writeBinary((*_iter239));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -4683,10 +4683,10 @@ uint32_t AccumuloProxy_getMaxRow_pargs::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("auths", ::apache::thrift::protocol::T_SET, 3);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->auths)).size()));
-    std::set<std::string> ::const_iterator _iter238;
-    for (_iter238 = (*(this->auths)).begin(); _iter238 != (*(this->auths)).end(); ++_iter238)
+    std::set<std::string> ::const_iterator _iter240;
+    for (_iter240 = (*(this->auths)).begin(); _iter240 != (*(this->auths)).end(); ++_iter240)
     {
-      xfer += oprot->writeBinary((*_iter238));
+      xfer += oprot->writeBinary((*_iter240));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -4975,17 +4975,17 @@ uint32_t AccumuloProxy_getTableProperties_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size239;
-            ::apache::thrift::protocol::TType _ktype240;
-            ::apache::thrift::protocol::TType _vtype241;
-            xfer += iprot->readMapBegin(_ktype240, _vtype241, _size239);
-            uint32_t _i243;
-            for (_i243 = 0; _i243 < _size239; ++_i243)
+            uint32_t _size241;
+            ::apache::thrift::protocol::TType _ktype242;
+            ::apache::thrift::protocol::TType _vtype243;
+            xfer += iprot->readMapBegin(_ktype242, _vtype243, _size241);
+            uint32_t _i245;
+            for (_i245 = 0; _i245 < _size241; ++_i245)
             {
-              std::string _key244;
-              xfer += iprot->readString(_key244);
-              std::string& _val245 = this->success[_key244];
-              xfer += iprot->readString(_val245);
+              std::string _key246;
+              xfer += iprot->readString(_key246);
+              std::string& _val247 = this->success[_key246];
+              xfer += iprot->readString(_val247);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5040,11 +5040,11 @@ uint32_t AccumuloProxy_getTableProperties_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter246;
-      for (_iter246 = this->success.begin(); _iter246 != this->success.end(); ++_iter246)
+      std::map<std::string, std::string> ::const_iterator _iter248;
+      for (_iter248 = this->success.begin(); _iter248 != this->success.end(); ++_iter248)
       {
-        xfer += oprot->writeString(_iter246->first);
-        xfer += oprot->writeString(_iter246->second);
+        xfer += oprot->writeString(_iter248->first);
+        xfer += oprot->writeString(_iter248->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -5091,17 +5091,17 @@ uint32_t AccumuloProxy_getTableProperties_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size247;
-            ::apache::thrift::protocol::TType _ktype248;
-            ::apache::thrift::protocol::TType _vtype249;
-            xfer += iprot->readMapBegin(_ktype248, _vtype249, _size247);
-            uint32_t _i251;
-            for (_i251 = 0; _i251 < _size247; ++_i251)
+            uint32_t _size249;
+            ::apache::thrift::protocol::TType _ktype250;
+            ::apache::thrift::protocol::TType _vtype251;
+            xfer += iprot->readMapBegin(_ktype250, _vtype251, _size249);
+            uint32_t _i253;
+            for (_i253 = 0; _i253 < _size249; ++_i253)
             {
-              std::string _key252;
-              xfer += iprot->readString(_key252);
-              std::string& _val253 = (*(this->success))[_key252];
-              xfer += iprot->readString(_val253);
+              std::string _key254;
+              xfer += iprot->readString(_key254);
+              std::string& _val255 = (*(this->success))[_key254];
+              xfer += iprot->readString(_val255);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5768,14 +5768,14 @@ uint32_t AccumuloProxy_listSplits_result::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size254;
-            ::apache::thrift::protocol::TType _etype257;
-            xfer += iprot->readListBegin(_etype257, _size254);
-            this->success.resize(_size254);
-            uint32_t _i258;
-            for (_i258 = 0; _i258 < _size254; ++_i258)
+            uint32_t _size256;
+            ::apache::thrift::protocol::TType _etype259;
+            xfer += iprot->readListBegin(_etype259, _size256);
+            this->success.resize(_size256);
+            uint32_t _i260;
+            for (_i260 = 0; _i260 < _size256; ++_i260)
             {
-              xfer += iprot->readBinary(this->success[_i258]);
+              xfer += iprot->readBinary(this->success[_i260]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5830,10 +5830,10 @@ uint32_t AccumuloProxy_listSplits_result::write(::apache::thrift::protocol::TPro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter259;
-      for (_iter259 = this->success.begin(); _iter259 != this->success.end(); ++_iter259)
+      std::vector<std::string> ::const_iterator _iter261;
+      for (_iter261 = this->success.begin(); _iter261 != this->success.end(); ++_iter261)
       {
-        xfer += oprot->writeBinary((*_iter259));
+        xfer += oprot->writeBinary((*_iter261));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5880,14 +5880,14 @@ uint32_t AccumuloProxy_listSplits_presult::read(::apache::thrift::protocol::TPro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size260;
-            ::apache::thrift::protocol::TType _etype263;
-            xfer += iprot->readListBegin(_etype263, _size260);
-            (*(this->success)).resize(_size260);
-            uint32_t _i264;
-            for (_i264 = 0; _i264 < _size260; ++_i264)
+            uint32_t _size262;
+            ::apache::thrift::protocol::TType _etype265;
+            xfer += iprot->readListBegin(_etype265, _size262);
+            (*(this->success)).resize(_size262);
+            uint32_t _i266;
+            for (_i266 = 0; _i266 < _size262; ++_i266)
             {
-              xfer += iprot->readBinary((*(this->success))[_i264]);
+              xfer += iprot->readBinary((*(this->success))[_i266]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6022,15 +6022,15 @@ uint32_t AccumuloProxy_listTables_result::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->success.clear();
-            uint32_t _size265;
-            ::apache::thrift::protocol::TType _etype268;
-            xfer += iprot->readSetBegin(_etype268, _size265);
-            uint32_t _i269;
-            for (_i269 = 0; _i269 < _size265; ++_i269)
+            uint32_t _size267;
+            ::apache::thrift::protocol::TType _etype270;
+            xfer += iprot->readSetBegin(_etype270, _size267);
+            uint32_t _i271;
+            for (_i271 = 0; _i271 < _size267; ++_i271)
             {
-              std::string _elem270;
-              xfer += iprot->readString(_elem270);
-              this->success.insert(_elem270);
+              std::string _elem272;
+              xfer += iprot->readString(_elem272);
+              this->success.insert(_elem272);
             }
             xfer += iprot->readSetEnd();
           }
@@ -6061,10 +6061,10 @@ uint32_t AccumuloProxy_listTables_result::write(::apache::thrift::protocol::TPro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_SET, 0);
     {
       xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::set<std::string> ::const_iterator _iter271;
-      for (_iter271 = this->success.begin(); _iter271 != this->success.end(); ++_iter271)
+      std::set<std::string> ::const_iterator _iter273;
+      for (_iter273 = this->success.begin(); _iter273 != this->success.end(); ++_iter273)
       {
-        xfer += oprot->writeString((*_iter271));
+        xfer += oprot->writeString((*_iter273));
       }
       xfer += oprot->writeSetEnd();
     }
@@ -6099,15 +6099,15 @@ uint32_t AccumuloProxy_listTables_presult::read(::apache::thrift::protocol::TPro
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             (*(this->success)).clear();
-            uint32_t _size272;
-            ::apache::thrift::protocol::TType _etype275;
-            xfer += iprot->readSetBegin(_etype275, _size272);
-            uint32_t _i276;
-            for (_i276 = 0; _i276 < _size272; ++_i276)
+            uint32_t _size274;
+            ::apache::thrift::protocol::TType _etype277;
+            xfer += iprot->readSetBegin(_etype277, _size274);
+            uint32_t _i278;
+            for (_i278 = 0; _i278 < _size274; ++_i278)
             {
-              std::string _elem277;
-              xfer += iprot->readString(_elem277);
-              (*(this->success)).insert(_elem277);
+              std::string _elem279;
+              xfer += iprot->readString(_elem279);
+              (*(this->success)).insert(_elem279);
             }
             xfer += iprot->readSetEnd();
           }
@@ -6234,29 +6234,29 @@ uint32_t AccumuloProxy_listIterators_result::read(::apache::thrift::protocol::TP
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size278;
-            ::apache::thrift::protocol::TType _ktype279;
-            ::apache::thrift::protocol::TType _vtype280;
-            xfer += iprot->readMapBegin(_ktype279, _vtype280, _size278);
-            uint32_t _i282;
-            for (_i282 = 0; _i282 < _size278; ++_i282)
+            uint32_t _size280;
+            ::apache::thrift::protocol::TType _ktype281;
+            ::apache::thrift::protocol::TType _vtype282;
+            xfer += iprot->readMapBegin(_ktype281, _vtype282, _size280);
+            uint32_t _i284;
+            for (_i284 = 0; _i284 < _size280; ++_i284)
             {
-              std::string _key283;
-              xfer += iprot->readString(_key283);
-              std::set<IteratorScope::type> & _val284 = this->success[_key283];
+              std::string _key285;
+              xfer += iprot->readString(_key285);
+              std::set<IteratorScope::type> & _val286 = this->success[_key285];
               {
-                _val284.clear();
-                uint32_t _size285;
-                ::apache::thrift::protocol::TType _etype288;
-                xfer += iprot->readSetBegin(_etype288, _size285);
-                uint32_t _i289;
-                for (_i289 = 0; _i289 < _size285; ++_i289)
+                _val286.clear();
+                uint32_t _size287;
+                ::apache::thrift::protocol::TType _etype290;
+                xfer += iprot->readSetBegin(_etype290, _size287);
+                uint32_t _i291;
+                for (_i291 = 0; _i291 < _size287; ++_i291)
                 {
-                  IteratorScope::type _elem290;
-                  int32_t ecast291;
-                  xfer += iprot->readI32(ecast291);
-                  _elem290 = (IteratorScope::type)ecast291;
-                  _val284.insert(_elem290);
+                  IteratorScope::type _elem292;
+                  int32_t ecast293;
+                  xfer += iprot->readI32(ecast293);
+                  _elem292 = (IteratorScope::type)ecast293;
+                  _val286.insert(_elem292);
                 }
                 xfer += iprot->readSetEnd();
               }
@@ -6314,16 +6314,16 @@ uint32_t AccumuloProxy_listIterators_result::write(::apache::thrift::protocol::T
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_SET, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::set<IteratorScope::type> > ::const_iterator _iter292;
-      for (_iter292 = this->success.begin(); _iter292 != this->success.end(); ++_iter292)
+      std::map<std::string, std::set<IteratorScope::type> > ::const_iterator _iter294;
+      for (_iter294 = this->success.begin(); _iter294 != this->success.end(); ++_iter294)
       {
-        xfer += oprot->writeString(_iter292->first);
+        xfer += oprot->writeString(_iter294->first);
         {
-          xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(_iter292->second.size()));
-          std::set<IteratorScope::type> ::const_iterator _iter293;
-          for (_iter293 = _iter292->second.begin(); _iter293 != _iter292->second.end(); ++_iter293)
+          xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(_iter294->second.size()));
+          std::set<IteratorScope::type> ::const_iterator _iter295;
+          for (_iter295 = _iter294->second.begin(); _iter295 != _iter294->second.end(); ++_iter295)
           {
-            xfer += oprot->writeI32((int32_t)(*_iter293));
+            xfer += oprot->writeI32((int32_t)(*_iter295));
           }
           xfer += oprot->writeSetEnd();
         }
@@ -6373,29 +6373,29 @@ uint32_t AccumuloProxy_listIterators_presult::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size294;
-            ::apache::thrift::protocol::TType _ktype295;
-            ::apache::thrift::protocol::TType _vtype296;
-            xfer += iprot->readMapBegin(_ktype295, _vtype296, _size294);
-            uint32_t _i298;
-            for (_i298 = 0; _i298 < _size294; ++_i298)
+            uint32_t _size296;
+            ::apache::thrift::protocol::TType _ktype297;
+            ::apache::thrift::protocol::TType _vtype298;
+            xfer += iprot->readMapBegin(_ktype297, _vtype298, _size296);
+            uint32_t _i300;
+            for (_i300 = 0; _i300 < _size296; ++_i300)
             {
-              std::string _key299;
-              xfer += iprot->readString(_key299);
-              std::set<IteratorScope::type> & _val300 = (*(this->success))[_key299];
+              std::string _key301;
+              xfer += iprot->readString(_key301);
+              std::set<IteratorScope::type> & _val302 = (*(this->success))[_key301];
               {
-                _val300.clear();
-                uint32_t _size301;
-                ::apache::thrift::protocol::TType _etype304;
-                xfer += iprot->readSetBegin(_etype304, _size301);
-                uint32_t _i305;
-                for (_i305 = 0; _i305 < _size301; ++_i305)
+                _val302.clear();
+                uint32_t _size303;
+                ::apache::thrift::protocol::TType _etype306;
+                xfer += iprot->readSetBegin(_etype306, _size303);
+                uint32_t _i307;
+                for (_i307 = 0; _i307 < _size303; ++_i307)
                 {
-                  IteratorScope::type _elem306;
-                  int32_t ecast307;
-                  xfer += iprot->readI32(ecast307);
-                  _elem306 = (IteratorScope::type)ecast307;
-                  _val300.insert(_elem306);
+                  IteratorScope::type _elem308;
+                  int32_t ecast309;
+                  xfer += iprot->readI32(ecast309);
+                  _elem308 = (IteratorScope::type)ecast309;
+                  _val302.insert(_elem308);
                 }
                 xfer += iprot->readSetEnd();
               }
@@ -6549,17 +6549,17 @@ uint32_t AccumuloProxy_listConstraints_result::read(::apache::thrift::protocol::
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size308;
-            ::apache::thrift::protocol::TType _ktype309;
-            ::apache::thrift::protocol::TType _vtype310;
-            xfer += iprot->readMapBegin(_ktype309, _vtype310, _size308);
-            uint32_t _i312;
-            for (_i312 = 0; _i312 < _size308; ++_i312)
+            uint32_t _size310;
+            ::apache::thrift::protocol::TType _ktype311;
+            ::apache::thrift::protocol::TType _vtype312;
+            xfer += iprot->readMapBegin(_ktype311, _vtype312, _size310);
+            uint32_t _i314;
+            for (_i314 = 0; _i314 < _size310; ++_i314)
             {
-              std::string _key313;
-              xfer += iprot->readString(_key313);
-              int32_t& _val314 = this->success[_key313];
-              xfer += iprot->readI32(_val314);
+              std::string _key315;
+              xfer += iprot->readString(_key315);
+              int32_t& _val316 = this->success[_key315];
+              xfer += iprot->readI32(_val316);
             }
             xfer += iprot->readMapEnd();
           }
@@ -6614,11 +6614,11 @@ uint32_t AccumuloProxy_listConstraints_result::write(::apache::thrift::protocol:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, int32_t> ::const_iterator _iter315;
-      for (_iter315 = this->success.begin(); _iter315 != this->success.end(); ++_iter315)
+      std::map<std::string, int32_t> ::const_iterator _iter317;
+      for (_iter317 = this->success.begin(); _iter317 != this->success.end(); ++_iter317)
       {
-        xfer += oprot->writeString(_iter315->first);
-        xfer += oprot->writeI32(_iter315->second);
+        xfer += oprot->writeString(_iter317->first);
+        xfer += oprot->writeI32(_iter317->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -6665,17 +6665,17 @@ uint32_t AccumuloProxy_listConstraints_presult::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size316;
-            ::apache::thrift::protocol::TType _ktype317;
-            ::apache::thrift::protocol::TType _vtype318;
-            xfer += iprot->readMapBegin(_ktype317, _vtype318, _size316);
-            uint32_t _i320;
-            for (_i320 = 0; _i320 < _size316; ++_i320)
+            uint32_t _size318;
+            ::apache::thrift::protocol::TType _ktype319;
+            ::apache::thrift::protocol::TType _vtype320;
+            xfer += iprot->readMapBegin(_ktype319, _vtype320, _size318);
+            uint32_t _i322;
+            for (_i322 = 0; _i322 < _size318; ++_i322)
             {
-              std::string _key321;
-              xfer += iprot->readString(_key321);
-              int32_t& _val322 = (*(this->success))[_key321];
-              xfer += iprot->readI32(_val322);
+              std::string _key323;
+              xfer += iprot->readString(_key323);
+              int32_t& _val324 = (*(this->success))[_key323];
+              xfer += iprot->readI32(_val324);
             }
             xfer += iprot->readMapEnd();
           }
@@ -7720,17 +7720,17 @@ uint32_t AccumuloProxy_removeIterator_args::read(::apache::thrift::protocol::TPr
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->scopes.clear();
-            uint32_t _size323;
-            ::apache::thrift::protocol::TType _etype326;
-            xfer += iprot->readSetBegin(_etype326, _size323);
-            uint32_t _i327;
-            for (_i327 = 0; _i327 < _size323; ++_i327)
+            uint32_t _size325;
+            ::apache::thrift::protocol::TType _etype328;
+            xfer += iprot->readSetBegin(_etype328, _size325);
+            uint32_t _i329;
+            for (_i329 = 0; _i329 < _size325; ++_i329)
             {
-              IteratorScope::type _elem328;
-              int32_t ecast329;
-              xfer += iprot->readI32(ecast329);
-              _elem328 = (IteratorScope::type)ecast329;
-              this->scopes.insert(_elem328);
+              IteratorScope::type _elem330;
+              int32_t ecast331;
+              xfer += iprot->readI32(ecast331);
+              _elem330 = (IteratorScope::type)ecast331;
+              this->scopes.insert(_elem330);
             }
             xfer += iprot->readSetEnd();
           }
@@ -7770,10 +7770,10 @@ uint32_t AccumuloProxy_removeIterator_args::write(::apache::thrift::protocol::TP
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->scopes.size()));
-    std::set<IteratorScope::type> ::const_iterator _iter330;
-    for (_iter330 = this->scopes.begin(); _iter330 != this->scopes.end(); ++_iter330)
+    std::set<IteratorScope::type> ::const_iterator _iter332;
+    for (_iter332 = this->scopes.begin(); _iter332 != this->scopes.end(); ++_iter332)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter330));
+      xfer += oprot->writeI32((int32_t)(*_iter332));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -7803,10 +7803,10 @@ uint32_t AccumuloProxy_removeIterator_pargs::write(::apache::thrift::protocol::T
   xfer += oprot->writeFieldBegin("scopes", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>((*(this->scopes)).size()));
-    std::set<IteratorScope::type> ::const_iterator _iter331;
-    for (_iter331 = (*(this->scopes)).begin(); _iter331 != (*(this->scopes)).end(); ++_iter331)
+    std::set<IteratorScope::type> ::const_iterator _iter333;
+    for (_iter333 = (*(this->scopes)).begin(); _iter333 != (*(this->scopes)).end(); ++_iter333)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter331));
+      xfer += oprot->writeI32((int32_t)(*_iter333));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -8481,27 +8481,27 @@ uint32_t AccumuloProxy_setLocalityGroups_args::read(::apache::thrift::protocol::
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->groups.clear();
-            uint32_t _size332;
-            ::apache::thrift::protocol::TType _ktype333;
-            ::apache::thrift::protocol::TType _vtype334;
-            xfer += iprot->readMapBegin(_ktype333, _vtype334, _size332);
-            uint32_t _i336;
-            for (_i336 = 0; _i336 < _size332; ++_i336)
+            uint32_t _size334;
+            ::apache::thrift::protocol::TType _ktype335;
+            ::apache::thrift::protocol::TType _vtype336;
+            xfer += iprot->readMapBegin(_ktype335, _vtype336, _size334);
+            uint32_t _i338;
+            for (_i338 = 0; _i338 < _size334; ++_i338)
             {
-              std::string _key337;
-              xfer += iprot->readString(_key337);
-              std::set<std::string> & _val338 = this->groups[_key337];
+              std::string _key339;
+              xfer += iprot->readString(_key339);
+              std::set<std::string> & _val340 = this->groups[_key339];
               {
-                _val338.clear();
-                uint32_t _size339;
-                ::apache::thrift::protocol::TType _etype342;
-                xfer += iprot->readSetBegin(_etype342, _size339);
-                uint32_t _i343;
-                for (_i343 = 0; _i343 < _size339; ++_i343)
+                _val340.clear();
+                uint32_t _size341;
+                ::apache::thrift::protocol::TType _etype344;
+                xfer += iprot->readSetBegin(_etype344, _size341);
+                uint32_t _i345;
+                for (_i345 = 0; _i345 < _size341; ++_i345)
                 {
-                  std::string _elem344;
-                  xfer += iprot->readString(_elem344);
-                  _val338.insert(_elem344);
+                  std::string _elem346;
+                  xfer += iprot->readString(_elem346);
+                  _val340.insert(_elem346);
                 }
                 xfer += iprot->readSetEnd();
               }
@@ -8540,16 +8540,16 @@ uint32_t AccumuloProxy_setLocalityGroups_args::write(::apache::thrift::protocol:
   xfer += oprot->writeFieldBegin("groups", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_SET, static_cast<uint32_t>(this->groups.size()));
-    std::map<std::string, std::set<std::string> > ::const_iterator _iter345;
-    for (_iter345 = this->groups.begin(); _iter345 != this->groups.end(); ++_iter345)
+    std::map<std::string, std::set<std::string> > ::const_iterator _iter347;
+    for (_iter347 = this->groups.begin(); _iter347 != this->groups.end(); ++_iter347)
     {
-      xfer += oprot->writeString(_iter345->first);
+      xfer += oprot->writeString(_iter347->first);
       {
-        xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter345->second.size()));
-        std::set<std::string> ::const_iterator _iter346;
-        for (_iter346 = _iter345->second.begin(); _iter346 != _iter345->second.end(); ++_iter346)
+        xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter347->second.size()));
+        std::set<std::string> ::const_iterator _iter348;
+        for (_iter348 = _iter347->second.begin(); _iter348 != _iter347->second.end(); ++_iter348)
         {
-          xfer += oprot->writeString((*_iter346));
+          xfer += oprot->writeString((*_iter348));
         }
         xfer += oprot->writeSetEnd();
       }
@@ -8578,16 +8578,16 @@ uint32_t AccumuloProxy_setLocalityGroups_pargs::write(::apache::thrift::protocol
   xfer += oprot->writeFieldBegin("groups", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_SET, static_cast<uint32_t>((*(this->groups)).size()));
-    std::map<std::string, std::set<std::string> > ::const_iterator _iter347;
-    for (_iter347 = (*(this->groups)).begin(); _iter347 != (*(this->groups)).end(); ++_iter347)
+    std::map<std::string, std::set<std::string> > ::const_iterator _iter349;
+    for (_iter349 = (*(this->groups)).begin(); _iter349 != (*(this->groups)).end(); ++_iter349)
     {
-      xfer += oprot->writeString(_iter347->first);
+      xfer += oprot->writeString(_iter349->first);
       {
-        xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter347->second.size()));
-        std::set<std::string> ::const_iterator _iter348;
-        for (_iter348 = _iter347->second.begin(); _iter348 != _iter347->second.end(); ++_iter348)
+        xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter349->second.size()));
+        std::set<std::string> ::const_iterator _iter350;
+        for (_iter350 = _iter349->second.begin(); _iter350 != _iter349->second.end(); ++_iter350)
         {
-          xfer += oprot->writeString((*_iter348));
+          xfer += oprot->writeString((*_iter350));
         }
         xfer += oprot->writeSetEnd();
       }
@@ -9125,15 +9125,15 @@ uint32_t AccumuloProxy_splitRangeByTablets_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->success.clear();
-            uint32_t _size349;
-            ::apache::thrift::protocol::TType _etype352;
-            xfer += iprot->readSetBegin(_etype352, _size349);
-            uint32_t _i353;
-            for (_i353 = 0; _i353 < _size349; ++_i353)
+            uint32_t _size351;
+            ::apache::thrift::protocol::TType _etype354;
+            xfer += iprot->readSetBegin(_etype354, _size351);
+            uint32_t _i355;
+            for (_i355 = 0; _i355 < _size351; ++_i355)
             {
-              Range _elem354;
-              xfer += _elem354.read(iprot);
-              this->success.insert(_elem354);
+              Range _elem356;
+              xfer += _elem356.read(iprot);
+              this->success.insert(_elem356);
             }
             xfer += iprot->readSetEnd();
           }
@@ -9188,10 +9188,10 @@ uint32_t AccumuloProxy_splitRangeByTablets_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_SET, 0);
     {
       xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::set<Range> ::const_iterator _iter355;
-      for (_iter355 = this->success.begin(); _iter355 != this->success.end(); ++_iter355)
+      std::set<Range> ::const_iterator _iter357;
+      for (_iter357 = this->success.begin(); _iter357 != this->success.end(); ++_iter357)
       {
-        xfer += (*_iter355).write(oprot);
+        xfer += (*_iter357).write(oprot);
       }
       xfer += oprot->writeSetEnd();
     }
@@ -9238,15 +9238,15 @@ uint32_t AccumuloProxy_splitRangeByTablets_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             (*(this->success)).clear();
-            uint32_t _size356;
-            ::apache::thrift::protocol::TType _etype359;
-            xfer += iprot->readSetBegin(_etype359, _size356);
-            uint32_t _i360;
-            for (_i360 = 0; _i360 < _size356; ++_i360)
+            uint32_t _size358;
+            ::apache::thrift::protocol::TType _etype361;
+            xfer += iprot->readSetBegin(_etype361, _size358);
+            uint32_t _i362;
+            for (_i362 = 0; _i362 < _size358; ++_i362)
             {
-              Range _elem361;
-              xfer += _elem361.read(iprot);
-              (*(this->success)).insert(_elem361);
+              Range _elem363;
+              xfer += _elem363.read(iprot);
+              (*(this->success)).insert(_elem363);
             }
             xfer += iprot->readSetEnd();
           }
@@ -9559,17 +9559,17 @@ uint32_t AccumuloProxy_tableIdMap_result::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size362;
-            ::apache::thrift::protocol::TType _ktype363;
-            ::apache::thrift::protocol::TType _vtype364;
-            xfer += iprot->readMapBegin(_ktype363, _vtype364, _size362);
-            uint32_t _i366;
-            for (_i366 = 0; _i366 < _size362; ++_i366)
+            uint32_t _size364;
+            ::apache::thrift::protocol::TType _ktype365;
+            ::apache::thrift::protocol::TType _vtype366;
+            xfer += iprot->readMapBegin(_ktype365, _vtype366, _size364);
+            uint32_t _i368;
+            for (_i368 = 0; _i368 < _size364; ++_i368)
             {
-              std::string _key367;
-              xfer += iprot->readString(_key367);
-              std::string& _val368 = this->success[_key367];
-              xfer += iprot->readString(_val368);
+              std::string _key369;
+              xfer += iprot->readString(_key369);
+              std::string& _val370 = this->success[_key369];
+              xfer += iprot->readString(_val370);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9600,11 +9600,11 @@ uint32_t AccumuloProxy_tableIdMap_result::write(::apache::thrift::protocol::TPro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter369;
-      for (_iter369 = this->success.begin(); _iter369 != this->success.end(); ++_iter369)
+      std::map<std::string, std::string> ::const_iterator _iter371;
+      for (_iter371 = this->success.begin(); _iter371 != this->success.end(); ++_iter371)
       {
-        xfer += oprot->writeString(_iter369->first);
-        xfer += oprot->writeString(_iter369->second);
+        xfer += oprot->writeString(_iter371->first);
+        xfer += oprot->writeString(_iter371->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -9639,17 +9639,17 @@ uint32_t AccumuloProxy_tableIdMap_presult::read(::apache::thrift::protocol::TPro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size370;
-            ::apache::thrift::protocol::TType _ktype371;
-            ::apache::thrift::protocol::TType _vtype372;
-            xfer += iprot->readMapBegin(_ktype371, _vtype372, _size370);
-            uint32_t _i374;
-            for (_i374 = 0; _i374 < _size370; ++_i374)
+            uint32_t _size372;
+            ::apache::thrift::protocol::TType _ktype373;
+            ::apache::thrift::protocol::TType _vtype374;
+            xfer += iprot->readMapBegin(_ktype373, _vtype374, _size372);
+            uint32_t _i376;
+            for (_i376 = 0; _i376 < _size372; ++_i376)
             {
-              std::string _key375;
-              xfer += iprot->readString(_key375);
-              std::string& _val376 = (*(this->success))[_key375];
-              xfer += iprot->readString(_val376);
+              std::string _key377;
+              xfer += iprot->readString(_key377);
+              std::string& _val378 = (*(this->success))[_key377];
+              xfer += iprot->readString(_val378);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10244,14 +10244,14 @@ uint32_t AccumuloProxy_getActiveScans_result::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size377;
-            ::apache::thrift::protocol::TType _etype380;
-            xfer += iprot->readListBegin(_etype380, _size377);
-            this->success.resize(_size377);
-            uint32_t _i381;
-            for (_i381 = 0; _i381 < _size377; ++_i381)
+            uint32_t _size379;
+            ::apache::thrift::protocol::TType _etype382;
+            xfer += iprot->readListBegin(_etype382, _size379);
+            this->success.resize(_size379);
+            uint32_t _i383;
+            for (_i383 = 0; _i383 < _size379; ++_i383)
             {
-              xfer += this->success[_i381].read(iprot);
+              xfer += this->success[_i383].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10298,10 +10298,10 @@ uint32_t AccumuloProxy_getActiveScans_result::write(::apache::thrift::protocol::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<ActiveScan> ::const_iterator _iter382;
-      for (_iter382 = this->success.begin(); _iter382 != this->success.end(); ++_iter382)
+      std::vector<ActiveScan> ::const_iterator _iter384;
+      for (_iter384 = this->success.begin(); _iter384 != this->success.end(); ++_iter384)
       {
-        xfer += (*_iter382).write(oprot);
+        xfer += (*_iter384).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10344,14 +10344,14 @@ uint32_t AccumuloProxy_getActiveScans_presult::read(::apache::thrift::protocol::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size383;
-            ::apache::thrift::protocol::TType _etype386;
-            xfer += iprot->readListBegin(_etype386, _size383);
-            (*(this->success)).resize(_size383);
-            uint32_t _i387;
-            for (_i387 = 0; _i387 < _size383; ++_i387)
+            uint32_t _size385;
+            ::apache::thrift::protocol::TType _etype388;
+            xfer += iprot->readListBegin(_etype388, _size385);
+            (*(this->success)).resize(_size385);
+            uint32_t _i389;
+            for (_i389 = 0; _i389 < _size385; ++_i389)
             {
-              xfer += (*(this->success))[_i387].read(iprot);
+              xfer += (*(this->success))[_i389].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10494,14 +10494,14 @@ uint32_t AccumuloProxy_getActiveCompactions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size388;
-            ::apache::thrift::protocol::TType _etype391;
-            xfer += iprot->readListBegin(_etype391, _size388);
-            this->success.resize(_size388);
-            uint32_t _i392;
-            for (_i392 = 0; _i392 < _size388; ++_i392)
+            uint32_t _size390;
+            ::apache::thrift::protocol::TType _etype393;
+            xfer += iprot->readListBegin(_etype393, _size390);
+            this->success.resize(_size390);
+            uint32_t _i394;
+            for (_i394 = 0; _i394 < _size390; ++_i394)
             {
-              xfer += this->success[_i392].read(iprot);
+              xfer += this->success[_i394].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10548,10 +10548,10 @@ uint32_t AccumuloProxy_getActiveCompactions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<ActiveCompaction> ::const_iterator _iter393;
-      for (_iter393 = this->success.begin(); _iter393 != this->success.end(); ++_iter393)
+      std::vector<ActiveCompaction> ::const_iterator _iter395;
+      for (_iter395 = this->success.begin(); _iter395 != this->success.end(); ++_iter395)
       {
-        xfer += (*_iter393).write(oprot);
+        xfer += (*_iter395).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10594,14 +10594,14 @@ uint32_t AccumuloProxy_getActiveCompactions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size394;
-            ::apache::thrift::protocol::TType _etype397;
-            xfer += iprot->readListBegin(_etype397, _size394);
-            (*(this->success)).resize(_size394);
-            uint32_t _i398;
-            for (_i398 = 0; _i398 < _size394; ++_i398)
+            uint32_t _size396;
+            ::apache::thrift::protocol::TType _etype399;
+            xfer += iprot->readListBegin(_etype399, _size396);
+            (*(this->success)).resize(_size396);
+            uint32_t _i400;
+            for (_i400 = 0; _i400 < _size396; ++_i400)
             {
-              xfer += (*(this->success))[_i398].read(iprot);
+              xfer += (*(this->success))[_i400].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10728,17 +10728,17 @@ uint32_t AccumuloProxy_getSiteConfiguration_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size399;
-            ::apache::thrift::protocol::TType _ktype400;
-            ::apache::thrift::protocol::TType _vtype401;
-            xfer += iprot->readMapBegin(_ktype400, _vtype401, _size399);
-            uint32_t _i403;
-            for (_i403 = 0; _i403 < _size399; ++_i403)
+            uint32_t _size401;
+            ::apache::thrift::protocol::TType _ktype402;
+            ::apache::thrift::protocol::TType _vtype403;
+            xfer += iprot->readMapBegin(_ktype402, _vtype403, _size401);
+            uint32_t _i405;
+            for (_i405 = 0; _i405 < _size401; ++_i405)
             {
-              std::string _key404;
-              xfer += iprot->readString(_key404);
-              std::string& _val405 = this->success[_key404];
-              xfer += iprot->readString(_val405);
+              std::string _key406;
+              xfer += iprot->readString(_key406);
+              std::string& _val407 = this->success[_key406];
+              xfer += iprot->readString(_val407);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10785,11 +10785,11 @@ uint32_t AccumuloProxy_getSiteConfiguration_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter406;
-      for (_iter406 = this->success.begin(); _iter406 != this->success.end(); ++_iter406)
+      std::map<std::string, std::string> ::const_iterator _iter408;
+      for (_iter408 = this->success.begin(); _iter408 != this->success.end(); ++_iter408)
       {
-        xfer += oprot->writeString(_iter406->first);
-        xfer += oprot->writeString(_iter406->second);
+        xfer += oprot->writeString(_iter408->first);
+        xfer += oprot->writeString(_iter408->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -10832,17 +10832,17 @@ uint32_t AccumuloProxy_getSiteConfiguration_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size407;
-            ::apache::thrift::protocol::TType _ktype408;
-            ::apache::thrift::protocol::TType _vtype409;
-            xfer += iprot->readMapBegin(_ktype408, _vtype409, _size407);
-            uint32_t _i411;
-            for (_i411 = 0; _i411 < _size407; ++_i411)
+            uint32_t _size409;
+            ::apache::thrift::protocol::TType _ktype410;
+            ::apache::thrift::protocol::TType _vtype411;
+            xfer += iprot->readMapBegin(_ktype410, _vtype411, _size409);
+            uint32_t _i413;
+            for (_i413 = 0; _i413 < _size409; ++_i413)
             {
-              std::string _key412;
-              xfer += iprot->readString(_key412);
-              std::string& _val413 = (*(this->success))[_key412];
-              xfer += iprot->readString(_val413);
+              std::string _key414;
+              xfer += iprot->readString(_key414);
+              std::string& _val415 = (*(this->success))[_key414];
+              xfer += iprot->readString(_val415);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10969,17 +10969,17 @@ uint32_t AccumuloProxy_getSystemConfiguration_result::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size414;
-            ::apache::thrift::protocol::TType _ktype415;
-            ::apache::thrift::protocol::TType _vtype416;
-            xfer += iprot->readMapBegin(_ktype415, _vtype416, _size414);
-            uint32_t _i418;
-            for (_i418 = 0; _i418 < _size414; ++_i418)
+            uint32_t _size416;
+            ::apache::thrift::protocol::TType _ktype417;
+            ::apache::thrift::protocol::TType _vtype418;
+            xfer += iprot->readMapBegin(_ktype417, _vtype418, _size416);
+            uint32_t _i420;
+            for (_i420 = 0; _i420 < _size416; ++_i420)
             {
-              std::string _key419;
-              xfer += iprot->readString(_key419);
-              std::string& _val420 = this->success[_key419];
-              xfer += iprot->readString(_val420);
+              std::string _key421;
+              xfer += iprot->readString(_key421);
+              std::string& _val422 = this->success[_key421];
+              xfer += iprot->readString(_val422);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11026,11 +11026,11 @@ uint32_t AccumuloProxy_getSystemConfiguration_result::write(::apache::thrift::pr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter421;
-      for (_iter421 = this->success.begin(); _iter421 != this->success.end(); ++_iter421)
+      std::map<std::string, std::string> ::const_iterator _iter423;
+      for (_iter423 = this->success.begin(); _iter423 != this->success.end(); ++_iter423)
       {
-        xfer += oprot->writeString(_iter421->first);
-        xfer += oprot->writeString(_iter421->second);
+        xfer += oprot->writeString(_iter423->first);
+        xfer += oprot->writeString(_iter423->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -11073,17 +11073,17 @@ uint32_t AccumuloProxy_getSystemConfiguration_presult::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size422;
-            ::apache::thrift::protocol::TType _ktype423;
-            ::apache::thrift::protocol::TType _vtype424;
-            xfer += iprot->readMapBegin(_ktype423, _vtype424, _size422);
-            uint32_t _i426;
-            for (_i426 = 0; _i426 < _size422; ++_i426)
+            uint32_t _size424;
+            ::apache::thrift::protocol::TType _ktype425;
+            ::apache::thrift::protocol::TType _vtype426;
+            xfer += iprot->readMapBegin(_ktype425, _vtype426, _size424);
+            uint32_t _i428;
+            for (_i428 = 0; _i428 < _size424; ++_i428)
             {
-              std::string _key427;
-              xfer += iprot->readString(_key427);
-              std::string& _val428 = (*(this->success))[_key427];
-              xfer += iprot->readString(_val428);
+              std::string _key429;
+              xfer += iprot->readString(_key429);
+              std::string& _val430 = (*(this->success))[_key429];
+              xfer += iprot->readString(_val430);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11210,14 +11210,14 @@ uint32_t AccumuloProxy_getTabletServers_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size429;
-            ::apache::thrift::protocol::TType _etype432;
-            xfer += iprot->readListBegin(_etype432, _size429);
-            this->success.resize(_size429);
-            uint32_t _i433;
-            for (_i433 = 0; _i433 < _size429; ++_i433)
+            uint32_t _size431;
+            ::apache::thrift::protocol::TType _etype434;
+            xfer += iprot->readListBegin(_etype434, _size431);
+            this->success.resize(_size431);
+            uint32_t _i435;
+            for (_i435 = 0; _i435 < _size431; ++_i435)
             {
-              xfer += iprot->readString(this->success[_i433]);
+              xfer += iprot->readString(this->success[_i435]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11248,10 +11248,10 @@ uint32_t AccumuloProxy_getTabletServers_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter434;
-      for (_iter434 = this->success.begin(); _iter434 != this->success.end(); ++_iter434)
+      std::vector<std::string> ::const_iterator _iter436;
+      for (_iter436 = this->success.begin(); _iter436 != this->success.end(); ++_iter436)
       {
-        xfer += oprot->writeString((*_iter434));
+        xfer += oprot->writeString((*_iter436));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11286,14 +11286,14 @@ uint32_t AccumuloProxy_getTabletServers_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size435;
-            ::apache::thrift::protocol::TType _etype438;
-            xfer += iprot->readListBegin(_etype438, _size435);
-            (*(this->success)).resize(_size435);
-            uint32_t _i439;
-            for (_i439 = 0; _i439 < _size435; ++_i439)
+            uint32_t _size437;
+            ::apache::thrift::protocol::TType _etype440;
+            xfer += iprot->readListBegin(_etype440, _size437);
+            (*(this->success)).resize(_size437);
+            uint32_t _i441;
+            for (_i441 = 0; _i441 < _size437; ++_i441)
             {
-              xfer += iprot->readString((*(this->success))[_i439]);
+              xfer += iprot->readString((*(this->success))[_i441]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12000,17 +12000,17 @@ uint32_t AccumuloProxy_authenticateUser_args::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size440;
-            ::apache::thrift::protocol::TType _ktype441;
-            ::apache::thrift::protocol::TType _vtype442;
-            xfer += iprot->readMapBegin(_ktype441, _vtype442, _size440);
-            uint32_t _i444;
-            for (_i444 = 0; _i444 < _size440; ++_i444)
+            uint32_t _size442;
+            ::apache::thrift::protocol::TType _ktype443;
+            ::apache::thrift::protocol::TType _vtype444;
+            xfer += iprot->readMapBegin(_ktype443, _vtype444, _size442);
+            uint32_t _i446;
+            for (_i446 = 0; _i446 < _size442; ++_i446)
             {
-              std::string _key445;
-              xfer += iprot->readString(_key445);
-              std::string& _val446 = this->properties[_key445];
-              xfer += iprot->readString(_val446);
+              std::string _key447;
+              xfer += iprot->readString(_key447);
+              std::string& _val448 = this->properties[_key447];
+              xfer += iprot->readString(_val448);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12046,11 +12046,11 @@ uint32_t AccumuloProxy_authenticateUser_args::write(::apache::thrift::protocol::
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter447;
-    for (_iter447 = this->properties.begin(); _iter447 != this->properties.end(); ++_iter447)
+    std::map<std::string, std::string> ::const_iterator _iter449;
+    for (_iter449 = this->properties.begin(); _iter449 != this->properties.end(); ++_iter449)
     {
-      xfer += oprot->writeString(_iter447->first);
-      xfer += oprot->writeString(_iter447->second);
+      xfer += oprot->writeString(_iter449->first);
+      xfer += oprot->writeString(_iter449->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12076,11 +12076,11 @@ uint32_t AccumuloProxy_authenticateUser_pargs::write(::apache::thrift::protocol:
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->properties)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter448;
-    for (_iter448 = (*(this->properties)).begin(); _iter448 != (*(this->properties)).end(); ++_iter448)
+    std::map<std::string, std::string> ::const_iterator _iter450;
+    for (_iter450 = (*(this->properties)).begin(); _iter450 != (*(this->properties)).end(); ++_iter450)
     {
-      xfer += oprot->writeString(_iter448->first);
-      xfer += oprot->writeString(_iter448->second);
+      xfer += oprot->writeString(_iter450->first);
+      xfer += oprot->writeString(_iter450->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12267,15 +12267,15 @@ uint32_t AccumuloProxy_changeUserAuthorizations_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->authorizations.clear();
-            uint32_t _size449;
-            ::apache::thrift::protocol::TType _etype452;
-            xfer += iprot->readSetBegin(_etype452, _size449);
-            uint32_t _i453;
-            for (_i453 = 0; _i453 < _size449; ++_i453)
+            uint32_t _size451;
+            ::apache::thrift::protocol::TType _etype454;
+            xfer += iprot->readSetBegin(_etype454, _size451);
+            uint32_t _i455;
+            for (_i455 = 0; _i455 < _size451; ++_i455)
             {
-              std::string _elem454;
-              xfer += iprot->readBinary(_elem454);
-              this->authorizations.insert(_elem454);
+              std::string _elem456;
+              xfer += iprot->readBinary(_elem456);
+              this->authorizations.insert(_elem456);
             }
             xfer += iprot->readSetEnd();
           }
@@ -12311,10 +12311,10 @@ uint32_t AccumuloProxy_changeUserAuthorizations_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("authorizations", ::apache::thrift::protocol::T_SET, 3);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->authorizations.size()));
-    std::set<std::string> ::const_iterator _iter455;
-    for (_iter455 = this->authorizations.begin(); _iter455 != this->authorizations.end(); ++_iter455)
+    std::set<std::string> ::const_iterator _iter457;
+    for (_iter457 = this->authorizations.begin(); _iter457 != this->authorizations.end(); ++_iter457)
     {
-      xfer += oprot->writeBinary((*_iter455));
+      xfer += oprot->writeBinary((*_iter457));
     }
     xfer += oprot->writeSetEnd()

<TRUNCATED>

[08/18] git commit: ACCUMULO-1957 test cleanup

Posted by ec...@apache.org.
ACCUMULO-1957 test cleanup


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c56e3009
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c56e3009
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c56e3009

Branch: refs/heads/master
Commit: c56e300940b694bef845d8847d64e8a7f3aeb4f4
Parents: 549d1c2
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Sep 3 09:27:54 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/test/functional/DurabilityIT.java  | 22 +++++---------
 .../test/functional/SessionDurabilityIT.java    | 32 +++++++++-----------
 2 files changed, 22 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c56e3009/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 74df598..11f61c7 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -27,9 +27,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -99,7 +97,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     // sync table should lose nothing
     writeSome(tableNames[0], N);
     restartTServer();
-    assertEquals(N, readSome(tableNames[0], N));
+    assertEquals(N, readSome(tableNames[0]));
     cleanup(tableNames);
   }
 
@@ -109,7 +107,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     // flush table won't lose anything since we're not losing power/dfs
     writeSome(tableNames[1], N);
     restartTServer();
-    assertEquals(N, readSome(tableNames[1], N));
+    assertEquals(N, readSome(tableNames[1]));
     cleanup(tableNames);
   }
 
@@ -119,7 +117,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     // we're probably going to lose something the the log setting
     writeSome(tableNames[2], N);
     restartTServer();
-    assertTrue(N >= readSome(tableNames[2], N));
+    assertTrue(N >= readSome(tableNames[2]));
     cleanup(tableNames);
   }
 
@@ -129,7 +127,7 @@ public class DurabilityIT extends ConfigurableMacIT {
     // probably won't get any data back without logging
     writeSome(tableNames[3], N);
     restartTServer();
-    assertTrue(N > readSome(tableNames[3], N));
+    assertTrue(N > readSome(tableNames[3]));
     cleanup(tableNames);
   }
   
@@ -142,11 +140,11 @@ public class DurabilityIT extends ConfigurableMacIT {
     UtilWaitThread.sleep(1000);
     writeSome(tableName, N);
     restartTServer();
-    assertTrue(N > readSome(tableName, N));
+    assertTrue(N > readSome(tableName));
     c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "sync");
     writeSome(tableName, N);
     restartTServer();
-    assertTrue(N == readSome(tableName, N));
+    assertTrue(N == readSome(tableName));
   }
   
   private static Map<String, String> map(Iterable<Entry<String, String>> entries) {
@@ -170,12 +168,8 @@ public class DurabilityIT extends ConfigurableMacIT {
     
   }
 
-  private long readSome(String table, long n) throws Exception {
-    long count = 0;
-    for (@SuppressWarnings("unused") Entry<Key,Value> entry : getConnector().createScanner(table, Authorizations.EMPTY)) {
-      count++;
-    }
-    return count;
+  private long readSome(String table) throws Exception {
+    return FunctionalTestUtils.count(getConnector().createScanner(table, Authorizations.EMPTY));
   }
 
   private void restartTServer() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c56e3009/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
index 91041a9..d777ee4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -1,18 +1,14 @@
 package org.apache.accumulo.test.functional;
 
-import static org.junit.Assert.*;
-
-import java.util.Map.Entry;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Durability;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -30,44 +26,44 @@ public class SessionDurabilityIT extends ConfigurableMacIT {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
   }
   
-  @Test
+  @Test(timeout = 3 * 60 * 1000)
   public void nondurableTableHasDurableWrites() throws Exception {
     Connector c = getConnector();
     String tableName = getUniqueNames(1)[0];
+    // table default has no durability
     c.tableOperations().create(tableName);
     c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "none");
+    // send durable writes
     BatchWriterConfig cfg = new BatchWriterConfig();
     cfg.setDurability(Durability.SYNC);
-    write(tableName, 10, cfg);
+    writeSome(tableName, 10, cfg);
     assertEquals(10, count(tableName));
+    // verify writes servive restart
     restartTServer();
     assertEquals(10, count(tableName));
   }
   
-  @Test
+  @Test(timeout = 3 * 60 * 1000)
   public void durableTableLosesNonDurableWrites() throws Exception {
     Connector c = getConnector();
     String tableName = getUniqueNames(1)[0];
+    // table default is durable writes
     c.tableOperations().create(tableName);
     c.tableOperations().setProperty(tableName, Property.TABLE_DURABILITY.getKey(), "sync");
+    // write with no durability
     BatchWriterConfig cfg = new BatchWriterConfig();
     cfg.setDurability(Durability.NONE);
-    write(tableName, 10, cfg);
+    writeSome(tableName, 10, cfg);
+    // verify writes are lost on restart
     restartTServer();
     assertTrue(10 > count(tableName));
   }
   
   private int count(String tableName) throws Exception {
-    Connector c = getConnector();
-    Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
-    int result = 0;
-    for (@SuppressWarnings("unused") Entry<Key,Value> entry :scanner) {
-      result++;
-    }
-    return result;
+    return FunctionalTestUtils.count(getConnector().createScanner(tableName, Authorizations.EMPTY));
   }
 
-  private void write(String tableName, int n, BatchWriterConfig cfg) throws Exception {
+  private void writeSome(String tableName, int n, BatchWriterConfig cfg) throws Exception {
     Connector c = getConnector();
     BatchWriter bw = c.createBatchWriter(tableName, cfg);
     for (int i = 0; i < 10; i++) {


[12/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
deleted file mode 100644
index e0b17ac..0000000
--- a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.proxy.thrift.SystemPermission;
-import org.apache.accumulo.proxy.thrift.TablePermission;
-import org.apache.accumulo.proxy.thrift.TimeType;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.server.TServer;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestProxySecurityOperations {
-  protected static TServer proxy;
-  protected static Thread thread;
-  protected static TestProxyClient tpc;
-  protected static ByteBuffer userpass;
-  protected static final int port = 10196;
-  protected static final String testtable = "testtable";
-  protected static final String testuser = "VonJines";
-  protected static final ByteBuffer testpw = ByteBuffer.wrap("fiveones".getBytes());
-  
-  @BeforeClass
-  public static void setup() throws Exception {
-    Properties prop = new Properties();
-    prop.setProperty("useMockInstance", "true");
-    prop.put("tokenClass", PasswordToken.class.getName());
-    
-    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
-        port, TCompactProtocol.Factory.class, prop);
-    thread = new Thread() {
-      @Override
-      public void run() {
-        proxy.serve();
-      }
-    };
-    thread.start();
-    
-    tpc = new TestProxyClient("localhost", port);
-    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
-      private static final long serialVersionUID = 1L;
-      
-      {
-        put("password", "");
-      }
-    });
-  }
-  
-  @AfterClass
-  public static void tearDown() throws InterruptedException {
-    proxy.stop();
-    thread.join();
-  }
-  
-  @Before
-  public void makeTestTableAndUser() throws Exception {
-    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
-    tpc.proxy().createLocalUser(userpass, testuser, testpw);
-  }
-  
-  @After
-  public void deleteTestTable() throws Exception {
-    tpc.proxy().deleteTable(userpass, testtable);
-    tpc.proxy().dropLocalUser(userpass, testuser);
-  }
-  
-  @Test
-  public void create() throws TException {
-    tpc.proxy().createLocalUser(userpass, testuser + "2", testpw);
-    assertTrue(tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
-    tpc.proxy().dropLocalUser(userpass, testuser + "2");
-    assertTrue(!tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
-  }
-  
-  @Test
-  public void authenticate() throws TException {
-    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
-    assertFalse(tpc.proxy().authenticateUser(userpass, "EvilUser", bb2pp(testpw)));
-    
-    tpc.proxy().changeLocalUserPassword(userpass, testuser, ByteBuffer.wrap("newpass".getBytes()));
-    assertFalse(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
-    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(ByteBuffer.wrap("newpass".getBytes()))));
-    
-  }
-  
-  @Test
-  public void tablePermissions() throws TException {
-    tpc.proxy().grantTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
-    assertTrue(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
-    
-    tpc.proxy().revokeTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
-    assertFalse(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
-    
-  }
-  
-  @Test
-  public void systemPermissions() throws TException {
-    tpc.proxy().grantSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
-    assertTrue(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
-    
-    tpc.proxy().revokeSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
-    assertFalse(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
-    
-  }
-  
-  @Test
-  public void auths() throws TException {
-    HashSet<ByteBuffer> newauths = new HashSet<ByteBuffer>();
-    newauths.add(ByteBuffer.wrap("BBR".getBytes()));
-    newauths.add(ByteBuffer.wrap("Barney".getBytes()));
-    tpc.proxy().changeUserAuthorizations(userpass, testuser, newauths);
-    List<ByteBuffer> actualauths = tpc.proxy().getUserAuthorizations(userpass, testuser);
-    assertEquals(actualauths.size(), newauths.size());
-    
-    for (ByteBuffer auth : actualauths) {
-      assertTrue(newauths.contains(auth));
-    }
-  }
-  
-  private Map<String,String> bb2pp(ByteBuffer cf) {
-    Map<String,String> toRet = new TreeMap<String,String>();
-    toRet.put("password", ByteBufferUtil.toString(cf));
-    return toRet;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java b/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
deleted file mode 100644
index 87d3454..0000000
--- a/proxy/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.proxy.thrift.ColumnUpdate;
-import org.apache.accumulo.proxy.thrift.TimeType;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.server.TServer;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestProxyTableOperations {
-  
-  protected static TServer proxy;
-  protected static Thread thread;
-  protected static TestProxyClient tpc;
-  protected static ByteBuffer userpass;
-  protected static final int port = 10195;
-  protected static final String testtable = "testtable";
-  
-  @SuppressWarnings("serial")
-  @BeforeClass
-  public static void setup() throws Exception {
-    Properties prop = new Properties();
-    prop.setProperty("useMockInstance", "true");
-    prop.put("tokenClass", PasswordToken.class.getName());
-    
-    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
-        port, TCompactProtocol.Factory.class, prop);
-    thread = new Thread() {
-      @Override
-      public void run() {
-        proxy.serve();
-      }
-    };
-    thread.start();
-    tpc = new TestProxyClient("localhost", port);
-    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
-      {
-        put("password", "");
-      }
-    });
-  }
-  
-  @AfterClass
-  public static void tearDown() throws InterruptedException {
-    proxy.stop();
-    thread.join();
-  }
-  
-  @Before
-  public void makeTestTable() throws Exception {
-    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
-  }
-  
-  @After
-  public void deleteTestTable() throws Exception {
-    tpc.proxy().deleteTable(userpass, testtable);
-  }
-  
-  @Test
-  public void createExistsDelete() throws TException {
-    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
-    tpc.proxy().createTable(userpass, "testtable2", true, TimeType.MILLIS);
-    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
-    tpc.proxy().deleteTable(userpass, "testtable2");
-    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
-  }
-  
-  @Test
-  public void listRename() throws TException {
-    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
-    tpc.proxy().renameTable(userpass, testtable, "testtable2");
-    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
-    tpc.proxy().renameTable(userpass, "testtable2", testtable);
-    assertTrue(tpc.proxy().listTables(userpass).contains("testtable"));
-    
-  }
-  
-  // This test does not yet function because the backing Mock instance does not yet support merging
-  @Test
-  public void merge() throws TException {
-    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
-    splits.add(ByteBuffer.wrap("a".getBytes()));
-    splits.add(ByteBuffer.wrap("c".getBytes()));
-    splits.add(ByteBuffer.wrap("z".getBytes()));
-    tpc.proxy().addSplits(userpass, testtable, splits);
-    
-    tpc.proxy().mergeTablets(userpass, testtable, ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap("d".getBytes()));
-    
-    splits.remove(ByteBuffer.wrap("c".getBytes()));
-    
-    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
-    
-    for (ByteBuffer split : tableSplits)
-      assertTrue(splits.contains(split));
-    assertTrue(tableSplits.size() == splits.size());
-    
-  }
-  
-  @Test
-  public void splits() throws TException {
-    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
-    splits.add(ByteBuffer.wrap("a".getBytes()));
-    splits.add(ByteBuffer.wrap("b".getBytes()));
-    splits.add(ByteBuffer.wrap("z".getBytes()));
-    tpc.proxy().addSplits(userpass, testtable, splits);
-    
-    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
-    
-    for (ByteBuffer split : tableSplits)
-      assertTrue(splits.contains(split));
-    assertTrue(tableSplits.size() == splits.size());
-  }
-  
-  @Test
-  public void constraints() throws TException {
-    int cid = tpc.proxy().addConstraint(userpass, testtable, "org.apache.accumulo.TestConstraint");
-    Map<String,Integer> constraints = tpc.proxy().listConstraints(userpass, testtable);
-    assertEquals((int) constraints.get("org.apache.accumulo.TestConstraint"), cid);
-    tpc.proxy().removeConstraint(userpass, testtable, cid);
-    constraints = tpc.proxy().listConstraints(userpass, testtable);
-    assertNull(constraints.get("org.apache.accumulo.TestConstraint"));
-  }
-  
-  @Test
-  public void localityGroups() throws TException {
-    Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
-    Set<String> group1 = new HashSet<String>();
-    group1.add("cf1");
-    groups.put("group1", group1);
-    Set<String> group2 = new HashSet<String>();
-    group2.add("cf2");
-    group2.add("cf3");
-    groups.put("group2", group2);
-    tpc.proxy().setLocalityGroups(userpass, testtable, groups);
-    
-    Map<String,Set<String>> actualGroups = tpc.proxy().getLocalityGroups(userpass, testtable);
-    
-    assertEquals(groups.size(), actualGroups.size());
-    for (String groupName : groups.keySet()) {
-      assertTrue(actualGroups.containsKey(groupName));
-      assertEquals(groups.get(groupName).size(), actualGroups.get(groupName).size());
-      for (String cf : groups.get(groupName)) {
-        assertTrue(actualGroups.get(groupName).contains(cf));
-      }
-    }
-  }
-  
-  @Test
-  public void tableProperties() throws TException {
-    tpc.proxy().setTableProperty(userpass, testtable, "test.property1", "wharrrgarbl");
-    assertEquals(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"), "wharrrgarbl");
-    tpc.proxy().removeTableProperty(userpass, testtable, "test.property1");
-    assertNull(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"));
-  }
-  
-  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String value) {
-    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
-    update.setValue(value.getBytes());
-    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
-  }
-  
-  @Test
-  public void tableOperationsRowMethods() throws TException {
-    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
-    for (int i = 0; i < 10; i++) {
-      addMutation(mutations, "" + i, "cf", "cq", "");
-    }
-    tpc.proxy().updateAndFlush(userpass, testtable, mutations);
-    
-    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("9".getBytes()));
-    
-    tpc.proxy().deleteRows(userpass, testtable, ByteBuffer.wrap("51".getBytes()), ByteBuffer.wrap("99".getBytes()));
-    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("5".getBytes()));
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index a8c6afa..0db1e5d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -315,6 +315,41 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             log.debug("Prepping table " + id + " for compaction cancellations.");
             zoo.putPersistentData(zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, zero, NodeExistsPolicy.SKIP);
           }
+
+          @SuppressWarnings("deprecation")
+          String zpath = zooRoot + Constants.ZCONFIG + "/" + Property.TSERV_WAL_SYNC_METHOD.getKey();
+          boolean flushDefault = false;
+          try {
+            byte data[] = zoo.getData(zpath, null);
+            if (new String(data, StandardCharsets.UTF_8).endsWith("flush")) {
+              flushDefault = true;
+            }
+          } catch (KeeperException.NoNodeException ex) {
+            // skip
+          } 
+          for (String id : zoo.getChildren(zooRoot + Constants.ZTABLES)) {
+            log.debug("Converting table " + id + " WALog setting to Durability");
+            try {
+              @SuppressWarnings("deprecation")
+              String path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_WALOG_ENABLED.getKey();
+              byte[] data = zoo.getData(path, null);
+              boolean useWAL = Boolean.parseBoolean(new String(data, StandardCharsets.UTF_8));
+              zoo.recursiveDelete(path, NodeMissingPolicy.FAIL);
+              path = zooRoot + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + Property.TABLE_DURABILITY.getKey();
+              if (useWAL) {
+                if (flushDefault) {
+                  zoo.putPersistentData(path, "flush".getBytes(), NodeExistsPolicy.SKIP);
+                } else {
+                  zoo.putPersistentData(path, "sync".getBytes(), NodeExistsPolicy.SKIP);
+                }
+              } else {
+                zoo.putPersistentData(path, "none".getBytes(), NodeExistsPolicy.SKIP);
+              }
+            } catch (KeeperException.NoNodeException ex) {
+              // skip it
+            }
+          }
+        
         }
 
         // create initial namespaces

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
index f9c443a..9cc07dc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
@@ -22,8 +22,8 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.client.Durability;
 
 public class TabletMutations {
-  private final int tid; 
-  private final int seq; 
+  private final int tid;
+  private final int seq;
   private final List<Mutation> mutations;
   private final Durability durability;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 8a9c510..f164621 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -60,6 +60,7 @@ import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.CompressedIterators;
 import org.apache.accumulo.core.client.impl.CompressedIterators.IterConfig;
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletType;
@@ -349,6 +350,8 @@ public class TabletServer implements Runnable {
   private final WriteTracker writeTracker = new WriteTracker();
 
   private final RowLocks rowLocks = new RowLocks();
+  
+  private final AtomicLong totalQueuedMutationSize = new AtomicLong(0);
 
   private class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
 
@@ -655,7 +658,7 @@ public class TabletServer implements Runnable {
     @Override
     public long startUpdate(TInfo tinfo, TCredentials credentials, TDurability tdurabilty) throws ThriftSecurityException {
       // Make sure user is real
-      Durability durability = Durability.fromThrift(tdurabilty);
+      Durability durability = DurabilityImpl.fromThrift(tdurabilty);
       security.authenticateUser(credentials, credentials);
       if (updateMetrics.isEnabled())
         updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
@@ -726,14 +729,18 @@ public class TabletServer implements Runnable {
         setUpdateTablet(us, keyExtent);
 
         if (us.currentTablet != null) {
+          long additionalMutationSize = 0;
           List<Mutation> mutations = us.queuedMutations.get(us.currentTablet);
           for (TMutation tmutation : tmutations) {
             Mutation mutation = new ServerMutation(tmutation);
             mutations.add(mutation);
-            us.queuedMutationSize += mutation.numBytes();
+            additionalMutationSize += mutation.numBytes();
           }
-          if (us.queuedMutationSize > TabletServer.this.getConfiguration().getMemoryInBytes(Property.TSERV_MUTATION_QUEUE_MAX))
+          us.queuedMutationSize += additionalMutationSize;
+          long totalQueued = TabletServer.this.updateTotalQueuedMutationSize(additionalMutationSize);
+          if (totalQueued > TabletServer.this.getConfiguration().getMemoryInBytes(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX)) {
             flush(us);
+          }
         }
       } finally {
         sessionManager.unreserveSession(us);
@@ -775,8 +782,8 @@ public class TabletServer implements Runnable {
                 }
                 us.failures.put(tablet.getExtent(), us.successfulCommits.get(tablet));
               } else {
-                log.debug("Durablity for " + tablet.getExtent() + " durability " + us.durability + " table durability " + tabletDurability + " using " + us.durability.resolveDurability(tabletDurability));
-                sendables.put(commitSession, new Mutations(us.durability.resolveDurability(tabletDurability), mutations));
+                log.debug("Durablity for " + tablet.getExtent() + " durability " + us.durability + " table durability " + tabletDurability + " using " + DurabilityImpl.resolveDurabilty(us.durability, tabletDurability));
+                sendables.put(commitSession, new Mutations(DurabilityImpl.resolveDurabilty(us.durability, tabletDurability), mutations));
                 mutationCount += mutations.size();
               }
 
@@ -791,7 +798,7 @@ public class TabletServer implements Runnable {
                 // violate constraints... this is what
                 // prepareMutationsForCommit()
                 // expects
-                sendables.put(e.getCommitSession(), new Mutations(us.durability.resolveDurability(tabletDurability), e.getNonViolators()));
+                sendables.put(e.getCommitSession(), new Mutations(DurabilityImpl.resolveDurabilty(us.durability, tabletDurability), e.getNonViolators()));
               }
 
               mutationCount += mutations.size();
@@ -880,6 +887,7 @@ public class TabletServer implements Runnable {
         if (us.currentTablet != null) {
           us.queuedMutations.put(us.currentTablet, new ArrayList<Mutation>());
         }
+        TabletServer.this.updateTotalQueuedMutationSize(-us.queuedMutationSize);
         us.queuedMutationSize = 0;
       }
       us.totalUpdates += mutationCount;
@@ -977,7 +985,7 @@ public class TabletServer implements Runnable {
           try {
             final Span wal = Trace.start("wal");
             try {
-              logger.log(cs, cs.getWALogSeq(), mutation, Durability.fromThrift(tdurability).resolveDurability(tabletDurability));
+              logger.log(cs, cs.getWALogSeq(), mutation, DurabilityImpl.resolveDurabilty(DurabilityImpl.fromThrift(tdurability), tabletDurability));
             } finally {
               wal.stop();
             }
@@ -1108,12 +1116,12 @@ public class TabletServer implements Runnable {
                 } else {
                   for (ServerConditionalMutation scm : entry.getValue())
                     results.add(new TCMResult(scm.getID(), TCMStatus.ACCEPTED));
-                  sendables.put(cs, new Mutations(sess.durability.resolveDurability(tabletDurability), mutations));
+                  sendables.put(cs, new Mutations(DurabilityImpl.resolveDurabilty(sess.durability, tabletDurability), mutations));
                 }
               }
             } catch (TConstraintViolationException e) {
               if (e.getNonViolators().size() > 0) {
-                sendables.put(e.getCommitSession(), new Mutations(sess.durability.resolveDurability(tabletDurability), e.getNonViolators()));
+                sendables.put(e.getCommitSession(), new Mutations(DurabilityImpl.resolveDurabilty(sess.durability, tabletDurability), e.getNonViolators()));
                 for (Mutation m : e.getNonViolators())
                   results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.ACCEPTED));
               }
@@ -1214,7 +1222,7 @@ public class TabletServer implements Runnable {
         if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
 
-      ConditionalSession cs = new ConditionalSession(credentials, new Authorizations(authorizations), tableId, Durability.fromThrift(tdurabilty));
+      ConditionalSession cs = new ConditionalSession(credentials, new Authorizations(authorizations), tableId, DurabilityImpl.fromThrift(tdurabilty));
 
       long sid = sessionManager.createSession(cs, false);
       return new TConditionalSession(sid, lockID, sessionManager.getMaxIdleTime());
@@ -1756,6 +1764,10 @@ public class TabletServer implements Runnable {
     return majorCompactorDisabled;
   }
 
+  public long updateTotalQueuedMutationSize(long additionalMutationSize) {
+    return totalQueuedMutationSize .addAndGet(additionalMutationSize);
+  }
+
   public Tablet getOnlineTablet(KeyExtent extent) {
     return onlineTablets.get(extent);
   }
@@ -2905,7 +2917,7 @@ public class TabletServer implements Runnable {
 
   public int createLogId(KeyExtent tablet) {
     AccumuloConfiguration acuTableConf = getTableConfiguration(tablet);
-    if (Durability.fromString(acuTableConf.get(Property.TABLE_DURABILITY)) != Durability.NONE) {
+    if (DurabilityImpl.fromString(acuTableConf.get(Property.TABLE_DURABILITY)) != Durability.NONE) {
       return logIdGenerator.incrementAndGet();
     }
     return -1;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 2aa6380..25c0ee8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -156,15 +156,16 @@ public class TabletServerResourceManager {
     long blockSize = acuConf.getMemoryInBytes(Property.TSERV_DEFAULT_BLOCKSIZE);
     long dCacheSize = acuConf.getMemoryInBytes(Property.TSERV_DATACACHE_SIZE);
     long iCacheSize = acuConf.getMemoryInBytes(Property.TSERV_INDEXCACHE_SIZE);
+    long totalQueueSize = acuConf.getMemoryInBytes(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX);
 
     _iCache = new LruBlockCache(iCacheSize, blockSize);
     _dCache = new LruBlockCache(dCacheSize, blockSize);
 
     Runtime runtime = Runtime.getRuntime();
-    if (!usingNativeMap && maxMemory + dCacheSize + iCacheSize > runtime.maxMemory()) {
+    if (!usingNativeMap && maxMemory + dCacheSize + iCacheSize + totalQueueSize > runtime.maxMemory()) {
       throw new IllegalArgumentException(String.format(
-          "Maximum tablet server map memory %,d and block cache sizes %,d is too large for this JVM configuration %,d", maxMemory, dCacheSize + iCacheSize,
-          runtime.maxMemory()));
+          "Maximum tablet server map memory %,d block cache sizes %,d and mutation queue size %,d is too large for this JVM configuration %,d", maxMemory, dCacheSize + iCacheSize,
+          totalQueueSize, runtime.maxMemory()));
     }
     runtime.gc();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index d374492..50475c2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -152,12 +152,9 @@ public class DfsLogger {
           switch (logWork.durability) {
             case DEFAULT:
             case NONE:
-              // shouldn't make it to the work queue
-              log.warn("unexpected durability " + logWork.durability, new Throwable());
-              break;
             case LOG:
-              // do nothing
-              break;
+              // shouldn't make it to the work queue
+              throw new IllegalArgumentException("unexpected durability " + logWork.durability);
             case SYNC:
               durabilityMethod = sync;
               break loop;
@@ -556,7 +553,17 @@ public class DfsLogger {
         durability = tabletMutations.getDurability();
       }
     }
-    return logFileData(data, durability);
+    return logFileData(data, chooseDurabilityForGroupCommit(mutations));
+  }
+
+  static Durability chooseDurabilityForGroupCommit(List<TabletMutations> mutations) {
+    Durability result = Durability.NONE;
+    for (TabletMutations tabletMutations : mutations) {
+      if (tabletMutations.getDurability().ordinal() > result.ordinal()) {
+        result = tabletMutations.getDurability();
+      }
+    }
+    return result;
   }
 
   public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn) throws IOException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index cb476c9..2a540e5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -361,10 +361,12 @@ public class TabletServerLogger {
   }
 
   public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m, final Durability durability) throws IOException {
-    if (durability == Durability.NONE)
+    if (durability == Durability.NONE) {
       return -1;
-    if (durability == Durability.DEFAULT)
-      log.warn("Unexpected durability " + durability, new Throwable());
+    }
+    if (durability == Durability.DEFAULT) {
+      throw new IllegalArgumentException("Unexpected durability " + durability);
+    }
     int seq = write(commitSession, false, new Writer() {
       @Override
       public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 965324a..c3380bf 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -44,6 +44,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Durability;
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -2384,6 +2385,7 @@ public class Tablet implements TabletCommitter {
 
   // this lock is basically used to synchronize writing of log info to metadata
   private final ReentrantLock logLock = new ReentrantLock();
+  private AtomicLong totalQueuedMutationSize = new AtomicLong(0);
 
   public synchronized int getLogCount() {
     return currentLogs.size();
@@ -2512,7 +2514,7 @@ public class Tablet implements TabletCommitter {
 
   @Override
   public Durability getDurability() {
-    return Durability.fromString(getTableConfiguration().get(Property.TABLE_DURABILITY));
+    return DurabilityImpl.fromString(getTableConfiguration().get(Property.TABLE_DURABILITY));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
new file mode 100644
index 0000000..2ae37ed
--- /dev/null
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.tserver.log;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.core.client.Durability;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.tserver.TabletMutations;
+import org.junit.Test;
+
+public class DfsLoggerTest {
+  
+  @Test
+  public void testDurabilityForGroupCommit() {
+    List<TabletMutations> lst = new ArrayList<TabletMutations>();
+    assertEquals(Durability.NONE, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m1 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.NONE);
+    lst.add(m1);
+    assertEquals(Durability.NONE, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m2 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.LOG);
+    lst.add(m2);
+    assertEquals(Durability.LOG, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m3 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.NONE);
+    lst.add(m3);
+    assertEquals(Durability.LOG, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m4 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.FLUSH);
+    lst.add(m4);
+    assertEquals(Durability.FLUSH, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m5 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.LOG);
+    lst.add(m5);
+    assertEquals(Durability.FLUSH, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m6 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.SYNC);
+    lst.add(m6);
+    assertEquals(Durability.SYNC, DfsLogger.chooseDurabilityForGroupCommit(lst));
+    TabletMutations m7 = new TabletMutations(0, 1, Collections.<Mutation>emptyList(), Durability.FLUSH);
+    lst.add(m7);
+    assertEquals(Durability.SYNC, DfsLogger.chooseDurabilityForGroupCommit(lst));
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
index 19ae5b8..5fae7ec 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
@@ -46,6 +47,7 @@ import org.apache.hadoop.io.Text;
 public class InsertCommand extends Command {
   private Option insertOptAuths, timestampOpt;
   private Option timeoutOption;
+  private Option durabilityOption;
   
   protected long getTimeout(final CommandLine cl) {
     if (cl.hasOption(timeoutOption.getLongOpt())) {
@@ -78,8 +80,27 @@ public class InsertCommand extends Command {
     else
       m.put(colf, colq, val);
     
-    final BatchWriter bw = shellState.getConnector().createBatchWriter(shellState.getTableName(),
-        new BatchWriterConfig().setMaxMemory(Math.max(m.estimatedMemoryUsed(), 1024)).setMaxWriteThreads(1).setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS));
+    final BatchWriterConfig cfg = new BatchWriterConfig().setMaxMemory(Math.max(m.estimatedMemoryUsed(), 1024)).setMaxWriteThreads(1).setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS);
+    if (cl.hasOption(durabilityOption.getOpt())) {
+      String userDurability = cl.getOptionValue(durabilityOption.getOpt());
+      switch (userDurability) {
+        case "sync":
+          cfg.setDurability(Durability.SYNC); 
+          break;
+        case "flush":
+          cfg.setDurability(Durability.FLUSH);
+          break;
+        case "none":
+          cfg.setDurability(Durability.NONE);
+          break;
+        case "log":
+          cfg.setDurability(Durability.NONE);
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown durability: " + userDurability);
+      }
+    }
+    final BatchWriter bw = shellState.getConnector().createBatchWriter(shellState.getTableName(), cfg);
     bw.addMutation(m);
     try {
       bw.close();
@@ -138,6 +159,8 @@ public class InsertCommand extends Command {
     timeoutOption.setArgName("timeout");
     o.addOption(timeoutOption);
     
+    durabilityOption = new Option("d", "durability", true, "durability to use for insert, should be one of \"none\" \"log\" \"flush\" or \"sync\"");
+    
     return o;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 80e8bee..fd6988c 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -100,6 +100,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-proxy</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-server-base</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
index 51880b5..b563ed9 100644
--- a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
@@ -30,20 +30,20 @@ import org.apache.hadoop.io.Text;
 import com.beust.jcommander.Parameter;
 
 public class WrongTabletTest {
-  
+
   static class Opts extends ClientOpts {
     @Parameter(names = "--location", required = true)
     String location;
   }
-  
+
   public static void main(String[] args) {
     Opts opts = new Opts();
     opts.parseArgs(WrongTabletTest.class.getName(), args);
-    
+
     ServerConfigurationFactory conf = new ServerConfigurationFactory(opts.getInstance());
     try {
       TabletClientService.Iface client = ThriftUtil.getTServerClient(opts.location, conf.getConfiguration());
-      
+
       Mutation mutation = new Mutation(new Text("row_0003750001"));
       mutation.putDelete(new Text("colf"), new Text("colq"));
       client.update(Tracer.traceInfo(), new Credentials(opts.principal, opts.getToken()).toThrift(opts.getInstance()), new KeyExtent(new Text("!!"), null,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index ee8a80d..8ab1e98 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -80,152 +80,152 @@ import com.google.common.net.HostAndPort;
 /**
  * The purpose of this class is to server as fake tserver that is a data sink like /dev/null. NullTserver modifies the metadata location entries for a table to
  * point to it. This allows thrift performance to be measured by running any client code that writes to a table.
- * 
+ *
  */
 
 public class NullTserver {
-  
+
   public static class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
-    
+
     private long updateSession = 1;
-    
+
     public ThriftClientHandler(Instance instance, TransactionWatcher watcher) {
       super(instance, watcher, null);
     }
-    
+
     @Override
     public long startUpdate(TInfo tinfo, TCredentials credentials, TDurability durability) {
       return updateSession++;
     }
-    
+
     @Override
     public void applyUpdates(TInfo tinfo, long updateID, TKeyExtent keyExtent, List<TMutation> mutation) {}
-    
+
     @Override
     public UpdateErrors closeUpdate(TInfo tinfo, long updateID) {
       return new UpdateErrors(new HashMap<TKeyExtent,Long>(), new ArrayList<TConstraintViolationSummary>(), new HashMap<TKeyExtent,SecurityErrorCode>());
     }
-    
+
     @Override
     public List<TKeyExtent> bulkImport(TInfo tinfo, TCredentials credentials, long tid, Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime) {
       return null;
     }
-    
+
     @Override
     public void closeMultiScan(TInfo tinfo, long scanID) {}
-    
+
     @Override
     public void closeScan(TInfo tinfo, long scanID) {}
-    
+
     @Override
     public MultiScanResult continueMultiScan(TInfo tinfo, long scanID) {
       return null;
     }
-    
+
     @Override
     public ScanResult continueScan(TInfo tinfo, long scanID) {
       return null;
     }
-    
+
     @Override
     public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent extent, ByteBuffer splitPoint) {
-      
+
     }
-    
+
     @Override
     public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials, Map<TKeyExtent,List<TRange>> batch, List<TColumn> columns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) {
       return null;
     }
-    
+
     @Override
     public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent extent, TRange range, List<TColumn> columns, int batchSize,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold) {
       return null;
     }
-    
+
     @Override
     public void update(TInfo tinfo, TCredentials credentials, TKeyExtent keyExtent, TMutation mutation, TDurability durability) {
-      
+
     }
-    
+
     @Override
     public TabletServerStatus getTabletServerStatus(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return null;
     }
-    
+
     @Override
     public List<TabletStats> getTabletStats(TInfo tinfo, TCredentials credentials, String tableId) throws ThriftSecurityException, TException {
       return null;
     }
-    
+
     @Override
     public TabletStats getHistoricalStats(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return null;
     }
-    
+
     @Override
     public void halt(TInfo tinfo, TCredentials credentials, String lock) throws ThriftSecurityException, TException {}
-    
+
     @Override
     public void fastHalt(TInfo tinfo, TCredentials credentials, String lock) {}
-    
+
     @Override
     public void loadTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent) throws TException {}
-    
+
     @Override
     public void unloadTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent, boolean save) throws TException {}
-    
+
     @Override
     public List<ActiveScan> getActiveScans(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveScan>();
     }
-    
+
     @Override
     public void chop(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent) throws TException {}
-    
+
     @Override
     public void flushTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent) throws TException {
-      
+
     }
-    
+
     @Override
     public void compact(TInfo tinfo, TCredentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
-      
+
     }
-    
+
     @Override
     public void flush(TInfo tinfo, TCredentials credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException {
-      
+
     }
-    
+
     @Override
     public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {}
-    
+
     @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveCompaction>();
     }
-    
+
     @Override
     public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability)
         throws ThriftSecurityException, TException {
       return null;
     }
-    
+
     @Override
     public List<TCMResult> conditionalUpdate(TInfo tinfo, long sessID, Map<TKeyExtent,List<TConditionalMutation>> mutations, List<String> symbols)
         throws NoSuchScanIDException, TException {
       return null;
     }
-    
+
     @Override
     public void invalidateConditionalUpdate(TInfo tinfo, long sessID) throws TException {}
-    
+
     @Override
     public void closeConditionalUpdate(TInfo tinfo, long sessID) throws TException {}
   }
-  
+
   static class Opts extends Help {
     @Parameter(names = {"-i", "--instance"}, description = "instance name", required = true)
     String iname = null;
@@ -236,22 +236,22 @@ public class NullTserver {
     @Parameter(names = "--port", description = "port number to use")
     int port = DefaultConfiguration.getInstance().getPort(Property.TSERV_CLIENTPORT);
   }
-  
+
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(NullTserver.class.getName(), args);
-    
+
     TransactionWatcher watcher = new TransactionWatcher();
     ThriftClientHandler tch = new ThriftClientHandler(HdfsZooInstance.getInstance(), watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
     TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1, 1000, 10 * 1024 * 1024, null, -1);
-    
+
     HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port);
-    
+
     // modify metadata
     ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers));
     String tableId = Tables.getTableId(zki, opts.tableName);
-    
+
     // read the locations for the table
     Range tableRange = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
     MetaDataTableScanner s = new MetaDataTableScanner(zki, SystemCredentials.get(), tableRange);
@@ -266,7 +266,7 @@ public class NullTserver {
     // point them to this server
     MetaDataStateStore store = new MetaDataStateStore();
     store.setLocations(assignments);
-    
+
     while (true) {
       UtilWaitThread.sleep(10000);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index 4af85a7..8d14574 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -72,7 +72,7 @@ public class Config extends Test {
       s(Property.TSERV_MAXMEM, 1000000, 3 * 1024 * 1024 * 1024L),
       s(Property.TSERV_READ_AHEAD_MAXCONCURRENT, 1, 25),
       s(Property.TSERV_MIGRATE_MAXCONCURRENT, 1, 10),
-      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
+      s(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
       s(Property.TSERV_RECOVERY_MAX_CONCURRENT, 1, 100),
       s(Property.TSERV_SCAN_MAX_OPENFILES, 10, 1000),
       s(Property.TSERV_THREADCHECK, 100, 10000),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
new file mode 100644
index 0000000..f8bcbfb
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
+import org.apache.accumulo.proxy.thrift.Column;
+import org.apache.accumulo.proxy.thrift.ColumnUpdate;
+import org.apache.accumulo.proxy.thrift.Condition;
+import org.apache.accumulo.proxy.thrift.ConditionalStatus;
+import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
+import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
+import org.apache.accumulo.proxy.thrift.Durability;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.accumulo.proxy.thrift.WriterOptions;
+import org.apache.accumulo.server.util.PortUtils;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.accumulo.test.functional.FunctionalTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.thrift.protocol.TJSONProtocol.Factory;
+import org.apache.thrift.server.TServer;
+import org.junit.Test;
+
+public class ProxyDurabilityIT extends ConfigurableMacIT {
+  
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+    cfg.setNumTservers(1);
+  }
+  
+  private static ByteBuffer bytes(String value) {
+    return ByteBuffer.wrap(value.getBytes());
+  }
+  
+  @Test
+  public void testDurability() throws Exception {
+    Connector c = getConnector();
+    Properties props = new Properties();
+    props.put("instance", c.getInstance().getInstanceName());
+    props.put("zookeepers", c.getInstance().getZooKeepers());
+    props.put("tokenClass", PasswordToken.class.getName());
+
+    Class<Factory> protocolClass = org.apache.thrift.protocol.TJSONProtocol.Factory.class;
+
+    int proxyPort = PortUtils.getRandomFreePort();
+    final TServer proxyServer = Proxy.createProxyServer(org.apache.accumulo.proxy.thrift.AccumuloProxy.class, org.apache.accumulo.proxy.ProxyServer.class, proxyPort,
+        protocolClass, props);
+    Thread thread = new Thread() {
+      @Override
+      public void run() {
+        proxyServer.serve();
+      }
+    };
+    thread.start();
+    while (!proxyServer.isServing())
+      UtilWaitThread.sleep(100);
+    Client client = new TestProxyClient("localhost", proxyPort, protocolClass.newInstance()).proxy();
+    Map<String,String> properties = new TreeMap<String,String>();
+    properties.put("password", ROOT_PASSWORD);
+    ByteBuffer login = client.login("root", properties);
+    
+    String tableName = getUniqueNames(1)[0];
+    client.createTable(login, tableName, true, TimeType.MILLIS);
+    assertTrue(c.tableOperations().exists(tableName));
+    
+    WriterOptions options = new WriterOptions();
+    options.setDurability(Durability.NONE);
+    String writer = client.createWriter(login, tableName, options);
+    Map<ByteBuffer,List<ColumnUpdate>> cells = new TreeMap<ByteBuffer, List<ColumnUpdate>>();
+    ColumnUpdate column = new ColumnUpdate(bytes("cf"), bytes("cq"));
+    column.setValue("value".getBytes());
+    cells.put(bytes("row"), Collections.singletonList(column));
+    client.update(writer, cells);
+    client.closeWriter(writer);
+    assertEquals(1, count(tableName));
+    restartTServer();
+    assertEquals(0, count(tableName));
+    
+    ConditionalWriterOptions cfg = new ConditionalWriterOptions();
+    cfg.setDurability(Durability.LOG);
+    String cwriter = client.createConditionalWriter(login, tableName, cfg);
+    ConditionalUpdates updates = new ConditionalUpdates();
+    updates.addToConditions(new Condition(new Column(bytes("cf"), bytes("cq"), bytes(""))));
+    updates.addToUpdates(column);
+    Map<ByteBuffer,ConditionalStatus> status = client.updateRowsConditionally(cwriter, Collections.singletonMap(bytes("row"), updates));
+    assertEquals(ConditionalStatus.ACCEPTED, status.get(bytes("row")));
+    assertEquals(1, count(tableName));
+    restartTServer();
+    assertEquals(0, count(tableName));
+    
+    proxyServer.stop();
+    thread.join();
+  }
+
+  private void restartTServer() throws Exception {
+    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+      cluster.killProcess(ServerType.TABLET_SERVER, proc);
+    }
+    cluster.start();
+  }
+
+  private int count(String tableName) throws Exception {
+    Connector c = getConnector();
+    return FunctionalTestUtils.count(c.createScanner(tableName, Authorizations.EMPTY));
+  }
+  
+}


[03/18] git commit: ACCUMULO-1957 updated with [~elserj]s review

Posted by ec...@apache.org.
ACCUMULO-1957 updated with [~elserj]s review


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7d608509
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7d608509
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7d608509

Branch: refs/heads/master
Commit: 7d608509e2f79c5f9d6093efb72aca71a32921bd
Parents: a25796c
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Aug 27 17:07:13 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:58 2014 -0400

----------------------------------------------------------------------
 core/src/main/java/org/apache/accumulo/core/conf/Property.java     | 1 +
 core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7d608509/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 018e83f..6ac6fa9 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -392,6 +392,7 @@ public enum Property {
   TABLE_BLOOM_HASHTYPE("table.bloom.hash.type", "murmur", PropertyType.STRING, "The bloom filter hash type"),
   TABLE_DURABILITY("table.durability", "sync", PropertyType.DURABILITY, "The durability used to write to the write-ahead log." +
       " Legal values are: none, which skips the write-ahead log; " +
+      "log, which sends the data to the write-ahead log, but does nothing to make it durable; " +
       "flush, which pushes data to the file system; and " +
       "sync, which ensures the data is written to disk."),
   TABLE_FAILURES_IGNORE("table.failures.ignore", "false", PropertyType.BOOLEAN,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7d608509/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index 00202cf..fc20535 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -68,7 +68,7 @@ public enum PropertyType {
   CLASSNAME("java class", "[\\w$.]*", "A fully qualified java class name representing a class on the classpath.\n"
       + "An example is 'java.lang.String', rather than 'String'"),
 
-  DURABILITY("durability", "(?:none|log|flush|sync)", "One of 'none', 'flush' or 'sync'."),
+  DURABILITY("durability", "(?:none|log|flush|sync)", "One of 'none', 'log', 'flush' or 'sync'."),
 
   STRING("string", ".*",
       "An arbitrary string of characters whose format is unspecified and interpreted based on the context of the property to which it applies."), BOOLEAN(


[18/18] git commit: ACCUMULO-1957 backed out changes I didn't mean to commit, added a test of the durability option for InsertCommand

Posted by ec...@apache.org.
ACCUMULO-1957 backed out changes I didn't mean to commit, added a test of the durability option for InsertCommand


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/6e02b3c7
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/6e02b3c7
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/6e02b3c7

Branch: refs/heads/master
Commit: 6e02b3c7bd96c7ab3840f311bfc62f5727549b75
Parents: 3a0beab
Author: Eric C. Newton <er...@gmail.com>
Authored: Thu Sep 4 15:07:36 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/core/conf/Property.java  |  6 +-----
 .../examples/simple/client/ReadWriteExample.java      |  2 +-
 .../org/apache/accumulo/tserver/TabletServer.java     | 14 ++------------
 .../accumulo/tserver/TabletServerResourceManager.java |  2 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java    |  1 -
 .../apache/accumulo/shell/commands/InsertCommand.java |  3 ++-
 .../accumulo/test/randomwalk/concurrent/Config.java   |  2 +-
 .../java/org/apache/accumulo/proxy/SimpleProxyIT.java | 12 ++++++------
 .../java/org/apache/accumulo/test/ShellServerIT.java  | 10 ++++++++++
 .../accumulo/test/functional/BloomFilterIT.java       |  2 +-
 10 files changed, 25 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 35cd0a6..5401c7c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -197,15 +197,11 @@ public enum Property {
   TSERV_INDEXCACHE_SIZE("tserver.cache.index.size", "512M", PropertyType.MEMORY, "Specifies the size of the cache for file indices."),
   TSERV_PORTSEARCH("tserver.port.search", "false", PropertyType.BOOLEAN, "if the ports above are in use, search higher ports until one is available"),
   TSERV_CLIENTPORT("tserver.port.client", "9997", PropertyType.PORT, "The port used for handling client connections on the tablet servers"),
-  @Deprecated
   TSERV_MUTATION_QUEUE_MAX("tserver.mutation.queue.max", "1M", PropertyType.MEMORY,
-      "This setting is deprecated. See tserver.total.mutation.queue.max. " 
-          + "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
+          "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
           + " max number of concurrent writer when configuring. When using Hadoop 2, Accumulo will call hsync() on the WAL . For a small number of "
           + "concurrent writers, increasing this buffer size decreases the frequncy of hsync calls. For a large number of concurrent writers a small buffers "
           + "size is ok because of group commit."),
-  TSERV_TOTAL_MUTATION_QUEUE_MAX("tserver.total.mutation.queue.max", "50M", PropertyType.MEMORY, 
-      "The amount of memory used to store write-ahead-log mutations before flushing them."),
   TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max", "30", PropertyType.COUNT,
       "To find a tablets split points, all index files are opened. This setting determines how many index "
           + "files can be opened at once. When there are more index files than this setting multiple passes "

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
index 2a42d29..ccc924b 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
@@ -64,7 +64,7 @@ public class ReadWriteExample {
     boolean readEntries = false;
     @Parameter(names = {"-d", "--delete"}, description = "delete entries after any creates")
     boolean deleteEntries = false;
-    @Parameter(names = {"--durability"}, description = "durabilty used for writes (none, log, flush or sync)", converter=DurabilityConverter.class)
+    @Parameter(names = {"--durability"}, description = "durability used for writes (none, log, flush or sync)", converter=DurabilityConverter.class)
     Durability durability = Durability.DEFAULT;
 
     public Opts() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index f164621..dc9f27f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -351,8 +351,6 @@ public class TabletServer implements Runnable {
 
   private final RowLocks rowLocks = new RowLocks();
   
-  private final AtomicLong totalQueuedMutationSize = new AtomicLong(0);
-
   private class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
 
     ThriftClientHandler() {
@@ -729,16 +727,13 @@ public class TabletServer implements Runnable {
         setUpdateTablet(us, keyExtent);
 
         if (us.currentTablet != null) {
-          long additionalMutationSize = 0;
           List<Mutation> mutations = us.queuedMutations.get(us.currentTablet);
           for (TMutation tmutation : tmutations) {
             Mutation mutation = new ServerMutation(tmutation);
             mutations.add(mutation);
-            additionalMutationSize += mutation.numBytes();
+            us.queuedMutationSize += mutation.numBytes();
           }
-          us.queuedMutationSize += additionalMutationSize;
-          long totalQueued = TabletServer.this.updateTotalQueuedMutationSize(additionalMutationSize);
-          if (totalQueued > TabletServer.this.getConfiguration().getMemoryInBytes(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX)) {
+          if (us.queuedMutationSize > TabletServer.this.getConfiguration().getMemoryInBytes(Property.TSERV_MUTATION_QUEUE_MAX)) {
             flush(us);
           }
         }
@@ -887,7 +882,6 @@ public class TabletServer implements Runnable {
         if (us.currentTablet != null) {
           us.queuedMutations.put(us.currentTablet, new ArrayList<Mutation>());
         }
-        TabletServer.this.updateTotalQueuedMutationSize(-us.queuedMutationSize);
         us.queuedMutationSize = 0;
       }
       us.totalUpdates += mutationCount;
@@ -1764,10 +1758,6 @@ public class TabletServer implements Runnable {
     return majorCompactorDisabled;
   }
 
-  public long updateTotalQueuedMutationSize(long additionalMutationSize) {
-    return totalQueuedMutationSize .addAndGet(additionalMutationSize);
-  }
-
   public Tablet getOnlineTablet(KeyExtent extent) {
     return onlineTablets.get(extent);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 25c0ee8..3d42c7c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -156,7 +156,7 @@ public class TabletServerResourceManager {
     long blockSize = acuConf.getMemoryInBytes(Property.TSERV_DEFAULT_BLOCKSIZE);
     long dCacheSize = acuConf.getMemoryInBytes(Property.TSERV_DATACACHE_SIZE);
     long iCacheSize = acuConf.getMemoryInBytes(Property.TSERV_INDEXCACHE_SIZE);
-    long totalQueueSize = acuConf.getMemoryInBytes(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX);
+    long totalQueueSize = acuConf.getMemoryInBytes(Property.TSERV_MUTATION_QUEUE_MAX);
 
     _iCache = new LruBlockCache(iCacheSize, blockSize);
     _dCache = new LruBlockCache(dCacheSize, blockSize);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index c3380bf..1151390 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -2385,7 +2385,6 @@ public class Tablet implements TabletCommitter {
 
   // this lock is basically used to synchronize writing of log info to metadata
   private final ReentrantLock logLock = new ReentrantLock();
-  private AtomicLong totalQueuedMutationSize = new AtomicLong(0);
 
   public synchronized int getLogCount() {
     return currentLogs.size();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
index 5fae7ec..16afc9e 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
@@ -85,7 +85,7 @@ public class InsertCommand extends Command {
       String userDurability = cl.getOptionValue(durabilityOption.getOpt());
       switch (userDurability) {
         case "sync":
-          cfg.setDurability(Durability.SYNC); 
+          cfg.setDurability(Durability.SYNC);
           break;
         case "flush":
           cfg.setDurability(Durability.FLUSH);
@@ -160,6 +160,7 @@ public class InsertCommand extends Command {
     o.addOption(timeoutOption);
     
     durabilityOption = new Option("d", "durability", true, "durability to use for insert, should be one of \"none\" \"log\" \"flush\" or \"sync\"");
+    o.addOption(durabilityOption);
     
     return o;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index 8d14574..4af85a7 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -72,7 +72,7 @@ public class Config extends Test {
       s(Property.TSERV_MAXMEM, 1000000, 3 * 1024 * 1024 * 1024L),
       s(Property.TSERV_READ_AHEAD_MAXCONCURRENT, 1, 25),
       s(Property.TSERV_MIGRATE_MAXCONCURRENT, 1, 10),
-      s(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
+      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
       s(Property.TSERV_RECOVERY_MAX_CONCURRENT, 1, 100),
       s(Property.TSERV_SCAN_MAX_OPENFILES, 10, 1000),
       s(Property.TSERV_THREADCHECK, 100, 10000),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java b/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
index 62e5b64..6b4bcfb 100644
--- a/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
+++ b/test/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
@@ -632,7 +632,7 @@ public class SimpleProxyIT {
       client.createConditionalWriter(creds, doesNotExist, new ConditionalWriterOptions());
     } catch (TableNotFoundException ex) {}
   }
-  
+
   @Test(timeout = 10 * 1000)
   public void testExists() throws Exception {
     client.createTable(creds, "ett1", false, TimeType.MILLIS);
@@ -806,7 +806,7 @@ public class SimpleProxyIT {
     };
     t.start();
 
-    // look for the scan many times 
+    // look for the scan many times
     List<ActiveScan> scans = new ArrayList<ActiveScan>();
     for (int i = 0; i < 100 && scans.isEmpty(); i++) {
       for (String tserver : client.getTabletServers(creds)) {
@@ -1012,7 +1012,7 @@ public class SimpleProxyIT {
     assertScan(new String[][] {}, TABLE_TEST);
 
     UtilWaitThread.sleep(2000);
-    
+
     writerOptions = new WriterOptions();
     writerOptions.setLatencyMs(10000);
     writerOptions.setMaxMemory(3000);
@@ -1551,11 +1551,11 @@ public class SimpleProxyIT {
   static private ByteBuffer t2bb(Text t) {
     return ByteBuffer.wrap(t.getBytes());
   }
-  
+
   @Test
   public void testGetRowRange() throws Exception {
     Range range = client.getRowRange(s2bb("xyzzy"));
-    org.apache.accumulo.core.data.Range range2 = new org.apache.accumulo.core.data.Range(new Text("xyzzy")); 
+    org.apache.accumulo.core.data.Range range2 = new org.apache.accumulo.core.data.Range(new Text("xyzzy"));
     assertEquals(0, range.start.row.compareTo(t2bb(range2.getStartKey().getRow())));
     assertEquals(0, range.stop.row.compareTo(t2bb(range2.getEndKey().getRow())));
     assertEquals(range.startInclusive, range2.isStartKeyInclusive());
@@ -1567,7 +1567,7 @@ public class SimpleProxyIT {
     assertEquals(range.start.timestamp, range.start.timestamp);
     assertEquals(range.stop.timestamp, range.stop.timestamp);
   }
-  
+
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 04fdd1c..b1ad3d8 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -421,6 +421,16 @@ public class ShellServerIT extends SimpleMacIT {
     ts.exec("deleteuser -f xyzzy", true);
     ts.exec("users", true, "xyzzy", false);
   }
+  
+  @Test(timeout = 60 * 1000)
+  public void durability() throws Exception {
+    final String table = name.getMethodName();
+    ts.exec("createtable " + table);
+    ts.exec("insert -d none a cf cq randomGunkaASDFWEAQRd");
+    ts.exec("insert -d foo a cf cq2 2", false, "foo", true);
+    ts.exec("scan -r a", true, "randomGunkaASDFWEAQRd", true);
+    ts.exec("scan -r a", true, "foo", false);
+  }  
 
   @Test(timeout = 45000)
   public void iter() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e02b3c7/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 6ee671e..8f6b830 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -57,7 +57,7 @@ public class BloomFilterIT extends ConfigurableMacIT {
     siteConfig.put(Property.TABLE_BLOOM_SIZE.getKey(), "2000000");
     siteConfig.put(Property.TABLE_BLOOM_ERRORRATE.getKey(), "1%");
     siteConfig.put(Property.TABLE_BLOOM_LOAD_THRESHOLD.getKey(), "0");
-    siteConfig.put(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "10M");
+    siteConfig.put(Property.TSERV_MUTATION_QUEUE_MAX.getKey(), "10M");
     siteConfig.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "64K");
     cfg.setSiteConfig(siteConfig );
   }


[09/18] git commit: ACCUMULO-1957 add durabilty to the simple Read/Write example

Posted by ec...@apache.org.
ACCUMULO-1957 add durabilty to the simple Read/Write example


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/1a2c8d5d
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/1a2c8d5d
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/1a2c8d5d

Branch: refs/heads/master
Commit: 1a2c8d5d47fa0c5d44cc01a26c3ff4dc13105d5f
Parents: e0fe2ae
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Sep 3 10:44:57 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../examples/simple/client/ReadWriteExample.java     | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1a2c8d5d/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
index a7b288d..7bc44e8 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -34,6 +35,7 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.ByteArraySet;
 import org.apache.hadoop.io.Text;
 
+import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
 
 public class ReadWriteExample {
@@ -43,6 +45,13 @@ public class ReadWriteExample {
   
   private Connector conn;
   
+  static class DurabilityConverter implements IStringConverter<Durability> {
+    @Override
+    public Durability convert(String value) {
+      return Durability.fromString(value);
+    }    
+  }
+  
   static class Opts extends ClientOnDefaultTable {
     @Parameter(names = {"-C", "--createtable"}, description = "create table before doing anything")
     boolean createtable = false;
@@ -54,6 +63,8 @@ public class ReadWriteExample {
     boolean readEntries = false;
     @Parameter(names = {"-d", "--delete"}, description = "delete entries after any creates")
     boolean deleteEntries = false;
+    @Parameter(names = {"--durability"}, description = "durabilty used for writes (none, log, flush or sync)", converter=DurabilityConverter.class)
+    Durability durability = Durability.DEFAULT;
     
     public Opts() {
       super(DEFAULT_TABLE_NAME);
@@ -103,7 +114,9 @@ public class ReadWriteExample {
   
   private void createEntries(Opts opts) throws Exception {
     if (opts.createEntries || opts.deleteEntries) {
-      BatchWriter writer = conn.createBatchWriter(opts.getTableName(), new BatchWriterConfig());
+      BatchWriterConfig cfg = new BatchWriterConfig();
+      cfg.setDurability(opts.durability);
+      BatchWriter writer = conn.createBatchWriter(opts.getTableName(), cfg);
       ColumnVisibility cv = new ColumnVisibility(opts.auths.toString().replace(',', '|'));
       
       Text cf = new Text("datatypes");


[02/18] git commit: ACCUMULO-1957 implement per-session Durability

Posted by ec...@apache.org.
ACCUMULO-1957 implement per-session Durability


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c2d95a1d
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c2d95a1d
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c2d95a1d

Branch: refs/heads/master
Commit: c2d95a1dcad3086279593279a8882ce347e03d4b
Parents: 7d60850
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Aug 29 17:12:00 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:58 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/BatchWriterConfig.java |  33 +-
 .../core/client/ConditionalWriterConfig.java    |  20 +
 .../apache/accumulo/core/client/Durability.java |  83 ++++
 .../core/client/impl/ConditionalWriterImpl.java |   5 +-
 .../client/impl/TabletServerBatchWriter.java    |  11 +-
 .../accumulo/core/client/impl/Writer.java       |   3 +-
 .../core/master/thrift/MasterClientService.java | 135 +-----
 .../core/tabletserver/thrift/TDurability.java   |  70 +++
 .../thrift/TabletClientService.java             | 444 +++++++++++++++++--
 core/src/main/thrift/tabletserver.thrift        |  14 +-
 .../org/apache/accumulo/tserver/Mutations.java  |  22 +
 .../accumulo/tserver/TabletMutations.java       |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |  49 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |  31 +-
 .../tserver/log/TabletServerLogger.java         |  48 +-
 .../tserver/session/ConditionalSession.java     |   5 +-
 .../accumulo/tserver/session/UpdateSession.java |   6 +-
 .../accumulo/tserver/tablet/CommitSession.java  |   4 -
 .../accumulo/tserver/tablet/Durability.java     |  33 --
 .../apache/accumulo/tserver/tablet/Tablet.java  |   1 +
 .../tserver/tablet/TabletCommitter.java         |   1 +
 .../apache/accumulo/test/WrongTabletTest.java   |   3 +-
 .../test/performance/thrift/NullTserver.java    |   7 +-
 .../accumulo/test/functional/DurabilityIT.java  |  92 ++--
 .../test/functional/SessionDurabilityIT.java    |  75 ++++
 25 files changed, 856 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index e2ec22e..1897552 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -46,6 +46,8 @@ public class BatchWriterConfig implements Writable {
 
   private static final Integer DEFAULT_MAX_WRITE_THREADS = 3;
   private Integer maxWriteThreads = null;
+  
+  private Durability durability = Durability.DEFAULT;
 
   /**
    * Sets the maximum memory to batch before writing. The smaller this value, the more frequently the {@link BatchWriter} will write.<br />
@@ -166,7 +168,24 @@ public class BatchWriterConfig implements Writable {
   public int getMaxWriteThreads() {
     return maxWriteThreads != null ? maxWriteThreads : DEFAULT_MAX_WRITE_THREADS;
   }
-
+  
+  /**
+   * @since 1.7.0
+   * @return the durability to be used by the BatchWriter
+   */
+  public Durability getDurability() {
+    return durability;
+  }
+ 
+  /**
+   * @param durability the Durability to be used by the BatchWriter
+   * @since 1.7.0
+   * 
+   */
+  public void setDurability(Durability durability) {
+    this.durability = durability;
+  }
+ 
   @Override
   public void write(DataOutput out) throws IOException {
     // write this out in a human-readable way
@@ -179,6 +198,8 @@ public class BatchWriterConfig implements Writable {
       addField(fields, "maxWriteThreads", maxWriteThreads);
     if (timeout != null)
       addField(fields, "timeout", timeout);
+    if (durability != Durability.DEFAULT)
+      addField(fields, "durability", durability);
     String output = StringUtils.join(",", fields);
 
     byte[] bytes = output.getBytes(StandardCharsets.UTF_8);
@@ -219,6 +240,8 @@ public class BatchWriterConfig implements Writable {
         maxWriteThreads = Integer.valueOf(value);
       } else if ("timeout".equals(key)) {
         timeout = Long.valueOf(value);
+      } else if ("durability".equals(key)) {
+        durability = Durability.fromString(value);
       } else {
         /* ignore any other properties */
       }
@@ -269,6 +292,9 @@ public class BatchWriterConfig implements Writable {
           return false;
         }
       }
+      if (durability != other.durability) {
+        return false;
+      }
 
       return true;
     }
@@ -279,7 +305,7 @@ public class BatchWriterConfig implements Writable {
   @Override
   public int hashCode() {
     HashCodeBuilder hcb = new HashCodeBuilder();
-    hcb.append(maxMemory).append(maxLatency).append(maxWriteThreads).append(timeout);
+    hcb.append(maxMemory).append(maxLatency).append(maxWriteThreads).append(timeout).append(durability);
     return hcb.toHashCode();
   }
 
@@ -287,7 +313,8 @@ public class BatchWriterConfig implements Writable {
   public String toString() {
     StringBuilder sb = new StringBuilder(32);
     sb.append("[maxMemory=").append(getMaxMemory()).append(", maxLatency=").append(getMaxLatency(TimeUnit.MILLISECONDS)).append(", maxWriteThreads=")
-        .append(getMaxWriteThreads()).append(", timeout=").append(getTimeout(TimeUnit.MILLISECONDS)).append("]");
+        .append(getMaxWriteThreads()).append(", timeout=").append(getTimeout(TimeUnit.MILLISECONDS))
+        .append(", durability=").append(durability).append("]");
     return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
index 7671c35..7bf568f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.client;
 
 import static com.google.common.base.Preconditions.checkArgument;
+
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.security.Authorizations;
@@ -35,6 +36,8 @@ public class ConditionalWriterConfig {
   
   private Authorizations auths = Authorizations.EMPTY;
   
+  private Durability durability = Durability.DEFAULT;
+  
   /**
    * A set of authorization labels that will be checked against the column visibility of each key in order to filter data. The authorizations passed in must be
    * a subset of the accumulo user's set of authorizations. If the accumulo user has authorizations (A1, A2) and authorizations (A2, A3) are passed, then an
@@ -102,6 +105,19 @@ public class ConditionalWriterConfig {
     return this;
   }
   
+  /**
+   * Sets the Durability for the mutation, if applied.
+   * <p>
+   * <b>Default:</b> Durability.DEFAULT: use the table's durability configuration.
+   * @param durability
+   * @return {@code this} to allow chaining of set methods
+   * @since 1.7.0
+   */
+  public ConditionalWriterConfig setDurability(Durability durability) {
+    this.durability = durability;
+    return this;
+  }
+  
   public Authorizations getAuthorizations() {
     return auths;
   }
@@ -113,4 +129,8 @@ public class ConditionalWriterConfig {
   public int getMaxWriteThreads() {
     return maxWriteThreads != null ? maxWriteThreads : DEFAULT_MAX_WRITE_THREADS;
   }
+  
+  public Durability getDurability() {
+    return durability;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/Durability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Durability.java b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
new file mode 100644
index 0000000..8f85aa2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
@@ -0,0 +1,83 @@
+package org.apache.accumulo.core.client;
+
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+
+/**
+ * The value for the durability of a BatchWriter or ConditionalWriter.
+ * @since 1.7.0 
+ */
+public enum Durability {
+  // Note, the order of these is important; the "highest" Durability is used in group commits.
+  /**
+   * Use the durability as specified by the table or system configuration.
+   */
+  DEFAULT,
+  /**
+   * Don't bother writing mutations to the write-ahead log.
+   */
+  NONE,
+  /**
+   * Write mutations the the write-ahead log. Data may be sitting the the servers output buffers, and not replicated anywhere.
+   */
+  LOG,
+  /**
+   * Write mutations to the write-ahead log, and ensure the data is stored on remote servers, but perhaps not on persistent storage.
+   */
+  FLUSH,
+  /**
+   * Write mutations to the write-ahead log, and ensure the data is saved to persistent storage.
+   */
+  SYNC;
+
+  // for internal use only
+  public TDurability toThrift() {
+    switch (this) {
+      case DEFAULT:
+        return TDurability.DEFAULT;
+      case SYNC:
+        return TDurability.SYNC;
+      case FLUSH:
+        return TDurability.FLUSH;
+      case LOG:
+        return TDurability.LOG;
+      default:
+        return TDurability.NONE;
+    }
+  }
+  
+  // for internal use only
+  static public Durability fromString(String value) {
+    try {
+      return Durability.valueOf(value.toUpperCase());
+    } catch (IllegalArgumentException ex) {
+      return Durability.SYNC;
+    }
+  }
+
+  // for internal use only
+  public static Durability fromThrift(TDurability tdurabilty) {
+    if (tdurabilty == null) {
+      return Durability.DEFAULT;
+    }
+    switch (tdurabilty) {
+      case DEFAULT:
+        return Durability.DEFAULT;
+      case SYNC:
+        return Durability.SYNC;
+      case FLUSH:
+        return Durability.FLUSH;
+      case LOG:
+        return Durability.LOG;
+      default:
+        return Durability.NONE;
+    }
+  }
+
+  // for internal use only
+  public Durability resolveDurability(Durability tabletDurability) {
+    if (this == Durability.DEFAULT) {
+      return tabletDurability;
+    }
+    return this;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index f5e6dd2..02954fa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriterConfig;
+import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -109,6 +110,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
   private TabletLocator locator;
   private String tableId;
   private long timeout;
+  private final Durability durability;
   
   private static class ServerQueue {
     BlockingQueue<TabletServerMutations<QCMutation>> queue = new LinkedBlockingQueue<TabletServerMutations<QCMutation>>();
@@ -380,6 +382,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     this.serverQueues = new HashMap<String,ServerQueue>();
     this.tableId = tableId;
     this.timeout = config.getTimeout(TimeUnit.MILLISECONDS);
+    this.durability = config.getDurability();
     
     Runnable failureHandler = new Runnable() {
       
@@ -497,7 +500,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
     
     TConditionalSession tcs = client.startConditionalUpdate(tinfo, credentials.toThrift(instance), ByteBufferUtil.toByteBuffers(auths.getAuthorizations()),
-        tableId);
+        tableId, durability.toThrift());
     
     synchronized (cachedSessionIDs) {
       SessionID sid = new SessionID();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index f2dd980..c68f842 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicLong;
 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.Durability;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableDeletedException;
@@ -106,6 +107,7 @@ public class TabletServerBatchWriter {
   private final long maxMem;
   private final long maxLatency;
   private final long timeout;
+  private final Durability durability;
   
   // state
   private boolean flushing;
@@ -149,9 +151,9 @@ public class TabletServerBatchWriter {
   private int unknownErrors = 0;
   private boolean somethingFailed = false;
   private Throwable lastUnknownError = null;
-  
+
   private static class TimeoutTracker {
-    
+
     final String server;
     final long timeOut;
     long activityTime;
@@ -196,6 +198,7 @@ public class TabletServerBatchWriter {
     this.timeout = config.getTimeout(TimeUnit.MILLISECONDS);
     this.mutations = new MutationSet();
     this.lastProcessingStartTime = System.currentTimeMillis();
+    this.durability = config.getDurability();
     
     this.writer = new MutationWriter(config.getMaxWriteThreads());
     
@@ -857,7 +860,7 @@ public class TabletServerBatchWriter {
             Entry<KeyExtent,List<Mutation>> entry = tabMuts.entrySet().iterator().next();
             
             try {
-              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift());
+              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift(), durability.toThrift());
             } catch (NotServingTabletException e) {
               allFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
               TabletLocator.getLocator(instance, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
@@ -867,7 +870,7 @@ public class TabletServerBatchWriter {
             timeoutTracker.madeProgress();
           } else {
             
-            long usid = client.startUpdate(tinfo, credentials.toThrift(instance));
+            long usid = client.startUpdate(tinfo, credentials.toThrift(instance), durability.toThrift());
             
             List<TMutation> updates = new ArrayList<TMutation>();
             for (Entry<KeyExtent,List<Mutation>> entry : tabMuts.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
index 44314c7..0358a88 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -72,7 +73,7 @@ public class Writer {
     TabletClientService.Iface client = null;
     try {
       client = ThriftUtil.getTServerClient(server, configuration);
-      client.update(Tracer.traceInfo(), ai.toThrift(instance), extent.toThrift(), m.toThrift());
+      client.update(Tracer.traceInfo(), ai.toThrift(instance), extent.toThrift(), m.toThrift(), TDurability.DEFAULT);
       return;
     } catch (ThriftSecurityException e) {
       throw new AccumuloSecurityException(e.user, e.code);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
index 9c850a8..a135e6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
@@ -76,7 +76,7 @@ import org.slf4j.LoggerFactory;
 
     public MasterMonitorInfo getMasterStats(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public void waitForBalance(org.apache.accumulo.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public void waitForBalance(org.apache.accumulo.trace.thrift.TInfo tinfo) throws org.apache.thrift.TException;
 
     public void reportSplitExtent(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String serverName, TabletSplit split) throws org.apache.thrift.TException;
 
@@ -473,7 +473,7 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getMasterStats failed: unknown result");
     }
 
-    public void waitForBalance(org.apache.accumulo.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public void waitForBalance(org.apache.accumulo.trace.thrift.TInfo tinfo) throws org.apache.thrift.TException
     {
       send_waitForBalance(tinfo);
       recv_waitForBalance();
@@ -486,13 +486,10 @@ import org.slf4j.LoggerFactory;
       sendBase("waitForBalance", args);
     }
 
-    public void recv_waitForBalance() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public void recv_waitForBalance() throws org.apache.thrift.TException
     {
       waitForBalance_result result = new waitForBalance_result();
       receiveBase(result, "waitForBalance");
-      if (result.sec != null) {
-        throw result.sec;
-      }
       return;
     }
 
@@ -1056,7 +1053,7 @@ import org.slf4j.LoggerFactory;
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+      public void getResult() throws org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -1496,11 +1493,7 @@ import org.slf4j.LoggerFactory;
 
       public waitForBalance_result getResult(I iface, waitForBalance_args args) throws org.apache.thrift.TException {
         waitForBalance_result result = new waitForBalance_result();
-        try {
-          iface.waitForBalance(args.tinfo);
-        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
+        iface.waitForBalance(args.tinfo);
         return result;
       }
     }
@@ -2305,12 +2298,6 @@ import org.slf4j.LoggerFactory;
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
             waitForBalance_result result = new waitForBalance_result();
-            if (e instanceof org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) {
-                        result.sec = (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException) e;
-                        result.setSecIsSet(true);
-                        msg = result;
-            }
-             else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
               msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
@@ -15738,7 +15725,6 @@ import org.slf4j.LoggerFactory;
   public static class waitForBalance_result implements org.apache.thrift.TBase<waitForBalance_result, waitForBalance_result._Fields>, java.io.Serializable, Cloneable, Comparable<waitForBalance_result>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("waitForBalance_result");
 
-    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -15746,11 +15732,10 @@ import org.slf4j.LoggerFactory;
       schemes.put(TupleScheme.class, new waitForBalance_resultTupleSchemeFactory());
     }
 
-    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // required
 
     /** 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 {
-      SEC((short)1, "sec");
+;
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -15765,8 +15750,6 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // SEC
-            return SEC;
           default:
             return null;
         }
@@ -15805,13 +15788,9 @@ import org.slf4j.LoggerFactory;
         return _fieldName;
       }
     }
-
-    // isset id assignments
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", 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(waitForBalance_result.class, metaDataMap);
     }
@@ -15819,20 +15798,10 @@ import org.slf4j.LoggerFactory;
     public waitForBalance_result() {
     }
 
-    public waitForBalance_result(
-      org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec)
-    {
-      this();
-      this.sec = sec;
-    }
-
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public waitForBalance_result(waitForBalance_result other) {
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
-      }
     }
 
     public waitForBalance_result deepCopy() {
@@ -15841,51 +15810,15 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public void clear() {
-      this.sec = null;
-    }
-
-    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public waitForBalance_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
-      this.sec = sec;
-      return this;
-    }
-
-    public void unsetSec() {
-      this.sec = null;
-    }
-
-    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
-    public boolean isSetSec() {
-      return this.sec != null;
-    }
-
-    public void setSecIsSet(boolean value) {
-      if (!value) {
-        this.sec = null;
-      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case SEC:
-        if (value == null) {
-          unsetSec();
-        } else {
-          setSec((org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException)value);
-        }
-        break;
-
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case SEC:
-        return getSec();
-
       }
       throw new IllegalStateException();
     }
@@ -15897,8 +15830,6 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case SEC:
-        return isSetSec();
       }
       throw new IllegalStateException();
     }
@@ -15916,15 +15847,6 @@ import org.slf4j.LoggerFactory;
       if (that == null)
         return false;
 
-      boolean this_present_sec = true && this.isSetSec();
-      boolean that_present_sec = true && that.isSetSec();
-      if (this_present_sec || that_present_sec) {
-        if (!(this_present_sec && that_present_sec))
-          return false;
-        if (!this.sec.equals(that.sec))
-          return false;
-      }
-
       return true;
     }
 
@@ -15941,16 +15863,6 @@ import org.slf4j.LoggerFactory;
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetSec()).compareTo(other.isSetSec());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSec()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -15971,13 +15883,6 @@ import org.slf4j.LoggerFactory;
       StringBuilder sb = new StringBuilder("waitForBalance_result(");
       boolean first = true;
 
-      sb.append("sec:");
-      if (this.sec == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.sec);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -16021,15 +15926,6 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 1: // SEC
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
-                struct.sec.read(iprot);
-                struct.setSecIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -16045,11 +15941,6 @@ import org.slf4j.LoggerFactory;
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -16067,25 +15958,11 @@ import org.slf4j.LoggerFactory;
       @Override
       public void write(org.apache.thrift.protocol.TProtocol prot, waitForBalance_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetSec()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, waitForBalance_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
-          struct.sec.read(iprot);
-          struct.setSecIsSet(true);
-        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
new file mode 100644
index 0000000..549798e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.tabletserver.thrift;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+@SuppressWarnings("all") public enum TDurability implements org.apache.thrift.TEnum {
+  DEFAULT(0),
+  SYNC(1),
+  FLUSH(2),
+  LOG(3),
+  NONE(4);
+
+  private final int value;
+
+  private TDurability(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TDurability findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return DEFAULT;
+      case 1:
+        return SYNC;
+      case 2:
+        return FLUSH;
+      case 3:
+        return LOG;
+      case 4:
+        return NONE;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 2ba7674..d583cd4 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -64,15 +64,15 @@ import org.slf4j.LoggerFactory;
 
     public void closeMultiScan(org.apache.accumulo.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, org.apache.thrift.TException;
 
-    public long startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public long startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
     public void applyUpdates(org.apache.accumulo.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, List<org.apache.accumulo.core.data.thrift.TMutation> mutations) throws org.apache.thrift.TException;
 
     public org.apache.accumulo.core.data.thrift.UpdateErrors closeUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long updateID) throws NoSuchScanIDException, org.apache.thrift.TException;
 
-    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException;
+    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException;
 
-    public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
     public List<org.apache.accumulo.core.data.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols) throws NoSuchScanIDException, org.apache.thrift.TException;
 
@@ -128,15 +128,15 @@ import org.slf4j.LoggerFactory;
 
     public void closeMultiScan(org.apache.accumulo.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void applyUpdates(org.apache.accumulo.trace.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, List<org.apache.accumulo.core.data.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void closeUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -371,17 +371,18 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public long startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public long startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      send_startUpdate(tinfo, credentials);
+      send_startUpdate(tinfo, credentials, durability);
       return recv_startUpdate();
     }
 
-    public void send_startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability) throws org.apache.thrift.TException
     {
       startUpdate_args args = new startUpdate_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
+      args.setDurability(durability);
       sendBase("startUpdate", args);
     }
 
@@ -440,19 +441,20 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "closeUpdate failed: unknown result");
     }
 
-    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException
+    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException
     {
-      send_update(tinfo, credentials, keyExtent, mutation);
+      send_update(tinfo, credentials, keyExtent, mutation, durability);
       recv_update();
     }
 
-    public void send_update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation) throws org.apache.thrift.TException
+    public void send_update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability) throws org.apache.thrift.TException
     {
       update_args args = new update_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setKeyExtent(keyExtent);
       args.setMutation(mutation);
+      args.setDurability(durability);
       sendBase("update", args);
     }
 
@@ -472,19 +474,20 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      send_startConditionalUpdate(tinfo, credentials, authorizations, tableID);
+      send_startConditionalUpdate(tinfo, credentials, authorizations, tableID, durability);
       return recv_startConditionalUpdate();
     }
 
-    public void send_startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.thrift.TException
+    public void send_startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability) throws org.apache.thrift.TException
     {
       startConditionalUpdate_args args = new startConditionalUpdate_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setAuthorizations(authorizations);
       args.setTableID(tableID);
+      args.setDurability(durability);
       sendBase("startConditionalUpdate", args);
     }
 
@@ -1182,9 +1185,9 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void startUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      startUpdate_call method_call = new startUpdate_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      startUpdate_call method_call = new startUpdate_call(tinfo, credentials, durability, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1192,10 +1195,12 @@ import org.slf4j.LoggerFactory;
     public static class startUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      public startUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TDurability durability;
+      public startUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
+        this.durability = durability;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1203,6 +1208,7 @@ import org.slf4j.LoggerFactory;
         startUpdate_args args = new startUpdate_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
+        args.setDurability(durability);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -1292,9 +1298,9 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      update_call method_call = new update_call(tinfo, credentials, keyExtent, mutation, resultHandler, this, ___protocolFactory, ___transport);
+      update_call method_call = new update_call(tinfo, credentials, keyExtent, mutation, durability, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1304,12 +1310,14 @@ import org.slf4j.LoggerFactory;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent;
       private org.apache.accumulo.core.data.thrift.TMutation mutation;
-      public update_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TDurability durability;
+      public update_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
         this.keyExtent = keyExtent;
         this.mutation = mutation;
+        this.durability = durability;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1319,6 +1327,7 @@ import org.slf4j.LoggerFactory;
         args.setCredentials(credentials);
         args.setKeyExtent(keyExtent);
         args.setMutation(mutation);
+        args.setDurability(durability);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -1333,9 +1342,9 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, resultHandler, this, ___protocolFactory, ___transport);
+      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, durability, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1345,12 +1354,14 @@ import org.slf4j.LoggerFactory;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private List<ByteBuffer> authorizations;
       private String tableID;
-      public startConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      private TDurability durability;
+      public startConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, TDurability durability, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
         this.authorizations = authorizations;
         this.tableID = tableID;
+        this.durability = durability;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1360,6 +1371,7 @@ import org.slf4j.LoggerFactory;
         args.setCredentials(credentials);
         args.setAuthorizations(authorizations);
         args.setTableID(tableID);
+        args.setDurability(durability);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -2323,7 +2335,7 @@ import org.slf4j.LoggerFactory;
       public startUpdate_result getResult(I iface, startUpdate_args args) throws org.apache.thrift.TException {
         startUpdate_result result = new startUpdate_result();
         try {
-          result.success = iface.startUpdate(args.tinfo, args.credentials);
+          result.success = iface.startUpdate(args.tinfo, args.credentials, args.durability);
           result.setSuccessIsSet(true);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
@@ -2391,7 +2403,7 @@ import org.slf4j.LoggerFactory;
       public update_result getResult(I iface, update_args args) throws org.apache.thrift.TException {
         update_result result = new update_result();
         try {
-          iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation);
+          iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (NotServingTabletException nste) {
@@ -2419,7 +2431,7 @@ import org.slf4j.LoggerFactory;
       public startConditionalUpdate_result getResult(I iface, startConditionalUpdate_args args) throws org.apache.thrift.TException {
         startConditionalUpdate_result result = new startConditionalUpdate_result();
         try {
-          result.success = iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID);
+          result.success = iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         }
@@ -3268,7 +3280,7 @@ import org.slf4j.LoggerFactory;
       }
 
       public void start(I iface, startUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Long> resultHandler) throws TException {
-        iface.startUpdate(args.tinfo, args.credentials,resultHandler);
+        iface.startUpdate(args.tinfo, args.credentials, args.durability,resultHandler);
       }
     }
 
@@ -3419,7 +3431,7 @@ import org.slf4j.LoggerFactory;
       }
 
       public void start(I iface, update_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation,resultHandler);
+        iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability,resultHandler);
       }
     }
 
@@ -3476,7 +3488,7 @@ import org.slf4j.LoggerFactory;
       }
 
       public void start(I iface, startConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.data.thrift.TConditionalSession> resultHandler) throws TException {
-        iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID,resultHandler);
+        iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability,resultHandler);
       }
     }
 
@@ -11862,6 +11874,7 @@ import org.slf4j.LoggerFactory;
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)3);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -11871,11 +11884,21 @@ import org.slf4j.LoggerFactory;
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability durability; // required
 
     /** 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 {
       TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)1, "credentials");
+      CREDENTIALS((short)1, "credentials"),
+      /**
+       * 
+       * @see TDurability
+       */
+      DURABILITY((short)3, "durability");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -11894,6 +11917,8 @@ import org.slf4j.LoggerFactory;
             return TINFO;
           case 1: // CREDENTIALS
             return CREDENTIALS;
+          case 3: // DURABILITY
+            return DURABILITY;
           default:
             return null;
         }
@@ -11941,6 +11966,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
       tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startUpdate_args.class, metaDataMap);
     }
@@ -11950,11 +11977,13 @@ import org.slf4j.LoggerFactory;
 
     public startUpdate_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.security.thrift.TCredentials credentials)
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      TDurability durability)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
+      this.durability = durability;
     }
 
     /**
@@ -11967,6 +11996,9 @@ import org.slf4j.LoggerFactory;
       if (other.isSetCredentials()) {
         this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
       }
+      if (other.isSetDurability()) {
+        this.durability = other.durability;
+      }
     }
 
     public startUpdate_args deepCopy() {
@@ -11977,6 +12009,7 @@ import org.slf4j.LoggerFactory;
     public void clear() {
       this.tinfo = null;
       this.credentials = null;
+      this.durability = null;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
@@ -12027,6 +12060,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability getDurability() {
+      return this.durability;
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    public startUpdate_args setDurability(TDurability durability) {
+      this.durability = durability;
+      return this;
+    }
+
+    public void unsetDurability() {
+      this.durability = null;
+    }
+
+    /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+    public boolean isSetDurability() {
+      return this.durability != null;
+    }
+
+    public void setDurabilityIsSet(boolean value) {
+      if (!value) {
+        this.durability = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case TINFO:
@@ -12045,6 +12110,14 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
+      case DURABILITY:
+        if (value == null) {
+          unsetDurability();
+        } else {
+          setDurability((TDurability)value);
+        }
+        break;
+
       }
     }
 
@@ -12056,6 +12129,9 @@ import org.slf4j.LoggerFactory;
       case CREDENTIALS:
         return getCredentials();
 
+      case DURABILITY:
+        return getDurability();
+
       }
       throw new IllegalStateException();
     }
@@ -12071,6 +12147,8 @@ import org.slf4j.LoggerFactory;
         return isSetTinfo();
       case CREDENTIALS:
         return isSetCredentials();
+      case DURABILITY:
+        return isSetDurability();
       }
       throw new IllegalStateException();
     }
@@ -12106,6 +12184,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_durability = true && this.isSetDurability();
+      boolean that_present_durability = true && that.isSetDurability();
+      if (this_present_durability || that_present_durability) {
+        if (!(this_present_durability && that_present_durability))
+          return false;
+        if (!this.durability.equals(that.durability))
+          return false;
+      }
+
       return true;
     }
 
@@ -12142,6 +12229,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDurability()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -12177,6 +12274,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.credentials);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -12244,6 +12349,14 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 3: // DURABILITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.durability = TDurability.findByValue(iprot.readI32());
+                struct.setDurabilityIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -12269,6 +12382,11 @@ import org.slf4j.LoggerFactory;
           struct.tinfo.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.durability != null) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -12293,19 +12411,25 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetCredentials()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetDurability()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
         if (struct.isSetCredentials()) {
           struct.credentials.write(oprot);
         }
+        if (struct.isSetDurability()) {
+          oprot.writeI32(struct.durability.getValue());
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, startUpdate_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
           struct.tinfo.read(iprot);
@@ -12316,6 +12440,10 @@ import org.slf4j.LoggerFactory;
           struct.credentials.read(iprot);
           struct.setCredentialsIsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.durability = TDurability.findByValue(iprot.readI32());
+          struct.setDurabilityIsSet(true);
+        }
       }
     }
 
@@ -14418,6 +14546,7 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField KEY_EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("keyExtent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField MUTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("mutation", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -14429,13 +14558,23 @@ import org.slf4j.LoggerFactory;
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
     public org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent; // required
     public org.apache.accumulo.core.data.thrift.TMutation mutation; // required
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability durability; // required
 
     /** 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 {
       TINFO((short)4, "tinfo"),
       CREDENTIALS((short)1, "credentials"),
       KEY_EXTENT((short)2, "keyExtent"),
-      MUTATION((short)3, "mutation");
+      MUTATION((short)3, "mutation"),
+      /**
+       * 
+       * @see TDurability
+       */
+      DURABILITY((short)5, "durability");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -14458,6 +14597,8 @@ import org.slf4j.LoggerFactory;
             return KEY_EXTENT;
           case 3: // MUTATION
             return MUTATION;
+          case 5: // DURABILITY
+            return DURABILITY;
           default:
             return null;
         }
@@ -14509,6 +14650,8 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TKeyExtent.class)));
       tmpMap.put(_Fields.MUTATION, new org.apache.thrift.meta_data.FieldMetaData("mutation", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TMutation.class)));
+      tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_args.class, metaDataMap);
     }
@@ -14520,13 +14663,15 @@ import org.slf4j.LoggerFactory;
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent,
-      org.apache.accumulo.core.data.thrift.TMutation mutation)
+      org.apache.accumulo.core.data.thrift.TMutation mutation,
+      TDurability durability)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.keyExtent = keyExtent;
       this.mutation = mutation;
+      this.durability = durability;
     }
 
     /**
@@ -14545,6 +14690,9 @@ import org.slf4j.LoggerFactory;
       if (other.isSetMutation()) {
         this.mutation = new org.apache.accumulo.core.data.thrift.TMutation(other.mutation);
       }
+      if (other.isSetDurability()) {
+        this.durability = other.durability;
+      }
     }
 
     public update_args deepCopy() {
@@ -14557,6 +14705,7 @@ import org.slf4j.LoggerFactory;
       this.credentials = null;
       this.keyExtent = null;
       this.mutation = null;
+      this.durability = null;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
@@ -14655,6 +14804,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability getDurability() {
+      return this.durability;
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    public update_args setDurability(TDurability durability) {
+      this.durability = durability;
+      return this;
+    }
+
+    public void unsetDurability() {
+      this.durability = null;
+    }
+
+    /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+    public boolean isSetDurability() {
+      return this.durability != null;
+    }
+
+    public void setDurabilityIsSet(boolean value) {
+      if (!value) {
+        this.durability = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case TINFO:
@@ -14689,6 +14870,14 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
+      case DURABILITY:
+        if (value == null) {
+          unsetDurability();
+        } else {
+          setDurability((TDurability)value);
+        }
+        break;
+
       }
     }
 
@@ -14706,6 +14895,9 @@ import org.slf4j.LoggerFactory;
       case MUTATION:
         return getMutation();
 
+      case DURABILITY:
+        return getDurability();
+
       }
       throw new IllegalStateException();
     }
@@ -14725,6 +14917,8 @@ import org.slf4j.LoggerFactory;
         return isSetKeyExtent();
       case MUTATION:
         return isSetMutation();
+      case DURABILITY:
+        return isSetDurability();
       }
       throw new IllegalStateException();
     }
@@ -14778,6 +14972,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_durability = true && this.isSetDurability();
+      boolean that_present_durability = true && that.isSetDurability();
+      if (this_present_durability || that_present_durability) {
+        if (!(this_present_durability && that_present_durability))
+          return false;
+        if (!this.durability.equals(that.durability))
+          return false;
+      }
+
       return true;
     }
 
@@ -14834,6 +15037,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDurability()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -14885,6 +15098,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.mutation);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -14976,6 +15197,14 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 5: // DURABILITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.durability = TDurability.findByValue(iprot.readI32());
+                struct.setDurabilityIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -15011,6 +15240,11 @@ import org.slf4j.LoggerFactory;
           struct.tinfo.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.durability != null) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -15041,7 +15275,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetMutation()) {
           optionals.set(3);
         }
-        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetDurability()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
@@ -15054,12 +15291,15 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetMutation()) {
           struct.mutation.write(oprot);
         }
+        if (struct.isSetDurability()) {
+          oprot.writeI32(struct.durability.getValue());
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, update_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(4);
+        BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
           struct.tinfo.read(iprot);
@@ -15080,6 +15320,10 @@ import org.slf4j.LoggerFactory;
           struct.mutation.read(iprot);
           struct.setMutationIsSet(true);
         }
+        if (incoming.get(4)) {
+          struct.durability = TDurability.findByValue(iprot.readI32());
+          struct.setDurabilityIsSet(true);
+        }
       }
     }
 
@@ -15652,6 +15896,7 @@ import org.slf4j.LoggerFactory;
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)3);
     private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableID", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -15663,13 +15908,23 @@ import org.slf4j.LoggerFactory;
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
     public List<ByteBuffer> authorizations; // required
     public String tableID; // required
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability durability; // required
 
     /** 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 {
       TINFO((short)1, "tinfo"),
       CREDENTIALS((short)2, "credentials"),
       AUTHORIZATIONS((short)3, "authorizations"),
-      TABLE_ID((short)4, "tableID");
+      TABLE_ID((short)4, "tableID"),
+      /**
+       * 
+       * @see TDurability
+       */
+      DURABILITY((short)5, "durability");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -15692,6 +15947,8 @@ import org.slf4j.LoggerFactory;
             return AUTHORIZATIONS;
           case 4: // TABLE_ID
             return TABLE_ID;
+          case 5: // DURABILITY
+            return DURABILITY;
           default:
             return null;
         }
@@ -15744,6 +16001,8 @@ import org.slf4j.LoggerFactory;
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
       tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_args.class, metaDataMap);
     }
@@ -15755,13 +16014,15 @@ import org.slf4j.LoggerFactory;
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       List<ByteBuffer> authorizations,
-      String tableID)
+      String tableID,
+      TDurability durability)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.authorizations = authorizations;
       this.tableID = tableID;
+      this.durability = durability;
     }
 
     /**
@@ -15781,6 +16042,9 @@ import org.slf4j.LoggerFactory;
       if (other.isSetTableID()) {
         this.tableID = other.tableID;
       }
+      if (other.isSetDurability()) {
+        this.durability = other.durability;
+      }
     }
 
     public startConditionalUpdate_args deepCopy() {
@@ -15793,6 +16057,7 @@ import org.slf4j.LoggerFactory;
       this.credentials = null;
       this.authorizations = null;
       this.tableID = null;
+      this.durability = null;
     }
 
     public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
@@ -15906,6 +16171,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    /**
+     * 
+     * @see TDurability
+     */
+    public TDurability getDurability() {
+      return this.durability;
+    }
+
+    /**
+     * 
+     * @see TDurability
+     */
+    public startConditionalUpdate_args setDurability(TDurability durability) {
+      this.durability = durability;
+      return this;
+    }
+
+    public void unsetDurability() {
+      this.durability = null;
+    }
+
+    /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+    public boolean isSetDurability() {
+      return this.durability != null;
+    }
+
+    public void setDurabilityIsSet(boolean value) {
+      if (!value) {
+        this.durability = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case TINFO:
@@ -15940,6 +16237,14 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
+      case DURABILITY:
+        if (value == null) {
+          unsetDurability();
+        } else {
+          setDurability((TDurability)value);
+        }
+        break;
+
       }
     }
 
@@ -15957,6 +16262,9 @@ import org.slf4j.LoggerFactory;
       case TABLE_ID:
         return getTableID();
 
+      case DURABILITY:
+        return getDurability();
+
       }
       throw new IllegalStateException();
     }
@@ -15976,6 +16284,8 @@ import org.slf4j.LoggerFactory;
         return isSetAuthorizations();
       case TABLE_ID:
         return isSetTableID();
+      case DURABILITY:
+        return isSetDurability();
       }
       throw new IllegalStateException();
     }
@@ -16029,6 +16339,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_durability = true && this.isSetDurability();
+      boolean that_present_durability = true && that.isSetDurability();
+      if (this_present_durability || that_present_durability) {
+        if (!(this_present_durability && that_present_durability))
+          return false;
+        if (!this.durability.equals(that.durability))
+          return false;
+      }
+
       return true;
     }
 
@@ -16085,6 +16404,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDurability()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -16136,6 +16465,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.tableID);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -16229,6 +16566,14 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 5: // DURABILITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.durability = TDurability.findByValue(iprot.readI32());
+                struct.setDurabilityIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -16271,6 +16616,11 @@ import org.slf4j.LoggerFactory;
           oprot.writeString(struct.tableID);
           oprot.writeFieldEnd();
         }
+        if (struct.durability != null) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -16301,7 +16651,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTableID()) {
           optionals.set(3);
         }
-        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetDurability()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
         if (struct.isSetTinfo()) {
           struct.tinfo.write(oprot);
         }
@@ -16320,12 +16673,15 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTableID()) {
           oprot.writeString(struct.tableID);
         }
+        if (struct.isSetDurability()) {
+          oprot.writeI32(struct.durability.getValue());
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(4);
+        BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
           struct.tinfo.read(iprot);
@@ -16353,6 +16709,10 @@ import org.slf4j.LoggerFactory;
           struct.tableID = iprot.readString();
           struct.setTableIDIsSet(true);
         }
+        if (incoming.get(4)) {
+          struct.durability = TDurability.findByValue(iprot.readI32());
+          struct.setDurabilityIsSet(true);
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 25e0b10..c6e620d 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -102,6 +102,14 @@ enum CompactionReason {
    CLOSE
 }
 
+enum TDurability {
+  DEFAULT = 0,
+  SYNC = 1,
+  FLUSH = 2,
+  LOG = 3,
+  NONE = 4
+}
+
 struct ActiveCompaction {
     1:data.TKeyExtent extent
     2:i64 age
@@ -158,17 +166,17 @@ service TabletClientService extends client.ClientService {
   void closeMultiScan(2:trace.TInfo tinfo, 1:data.ScanID scanID) throws (1:NoSuchScanIDException nssi),
   
   //the following calls support a batch update to multiple tablets on a tablet server
-  data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec),
+  data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.TCredentials credentials, 3:TDurability durability) throws (1:client.ThriftSecurityException sec),
   oneway void applyUpdates(1:trace.TInfo tinfo, 2:data.UpdateID updateID, 3:data.TKeyExtent keyExtent, 4:list<data.TMutation> mutations),
   data.UpdateErrors closeUpdate(2:trace.TInfo tinfo, 1:data.UpdateID updateID) throws (1:NoSuchScanIDException nssi),
 
   //the following call supports making a single update to a tablet
-  void update(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation)
+  void update(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation, 5:TDurability durability)
     throws (1:client.ThriftSecurityException sec, 
             2:NotServingTabletException nste, 
             3:ConstraintViolationException cve),
 
-  data.TConditionalSession startConditionalUpdate(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<binary> authorizations, 4:string tableID)
+  data.TConditionalSession startConditionalUpdate(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<binary> authorizations, 4:string tableID, 5:TDurability durability)
      throws (1:client.ThriftSecurityException sec);
   
   list<data.TCMResult> conditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID, 3:data.CMBatch mutations, 4:list<string> symbols)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2d95a1d/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
new file mode 100644
index 0000000..aa4ec81
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Mutations.java
@@ -0,0 +1,22 @@
+package org.apache.accumulo.tserver;
+
+import java.util.List;
+
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.client.Durability;
+
+public class Mutations {
+  private final Durability durability;
+  private final List<Mutation> mutations;
+
+  Mutations(Durability durability, List<Mutation> mutations) {
+    this.durability = durability;
+    this.mutations = mutations;
+  }
+  public Durability getDurability() {
+    return durability;
+  }
+  public List<Mutation> getMutations() {
+    return mutations;
+  }
+}
\ No newline at end of file