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:18 UTC

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

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