You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/07/24 21:49:20 UTC

[1/6] ACCUMULO-1000 added conditional mutations to Accumulo

Updated Branches:
  refs/heads/master 77cac56ec -> 9dc244484


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
new file mode 100644
index 0000000..a42a08b
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
@@ -0,0 +1,1236 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+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.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriter.Result;
+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.IsolatedScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.LongCombiner.Type;
+import org.apache.accumulo.core.iterators.user.SummingCombiner;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.FastFormat;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.test.functional.BadIterator;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+
+/**
+ * 
+ */
+public class ConditionalWriterTest {
+  
+  private static String secret = "superSecret";
+  public static TemporaryFolder folder = new TemporaryFolder();
+  public static MiniAccumuloCluster cluster;
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    folder.create();
+    MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
+    cluster = new MiniAccumuloCluster(cfg);
+    cluster.start();
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create("foo");
+
+    ConditionalWriter cw = conn.createConditionalWriter("foo", new ConditionalWriterConfig());
+    
+    // mutation conditional on column tx:seq not exiting
+    ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq"));
+    cm0.put("name", "last", "doe");
+    cm0.put("name", "first", "john");
+    cm0.put("tx", "seq", "1");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+    Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+
+    // mutation conditional on column tx:seq being 1
+    ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"));
+    cm1.put("name", "last", "Doe");
+    cm1.put("tx", "seq", "2");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+
+    // test condition where value differs
+    ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"));
+    cm2.put("name", "last", "DOE");
+    cm2.put("tx", "seq", "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm2).getStatus());
+    
+    // test condition where column does not exists
+    ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("txtypo", "seq").setValue("1"));
+    cm3.put("name", "last", "deo");
+    cm3.put("tx", "seq", "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm3).getStatus());
+    
+    // test two conditions, where one should fail
+    ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("2"), new Condition("name", "last").setValue("doe"));
+    cm4.put("name", "last", "deo");
+    cm4.put("tx", "seq", "3");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm4).getStatus());
+
+    // test two conditions, where one should fail
+    ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"), new Condition("name", "last").setValue("Doe"));
+    cm5.put("name", "last", "deo");
+    cm5.put("tx", "seq", "3");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm5).getStatus());
+
+    // ensure rejected mutations did not write
+    Scanner scanner = conn.createScanner("foo", Authorizations.EMPTY);
+    scanner.fetchColumn(new Text("name"), new Text("last"));
+    scanner.setRange(new Range("99006"));
+    Assert.assertEquals("Doe", scanner.iterator().next().getValue().toString());
+
+    // test w/ two conditions that are met
+    ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("2"), new Condition("name", "last").setValue("Doe"));
+    cm6.put("name", "last", "DOE");
+    cm6.put("tx", "seq", "3");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm6).getStatus());
+    
+    Assert.assertEquals("DOE", scanner.iterator().next().getValue().toString());
+    
+    // test a conditional mutation that deletes
+    ConditionalMutation cm7 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("3"));
+    cm7.putDelete("name", "last");
+    cm7.putDelete("name", "first");
+    cm7.putDelete("tx", "seq");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm7).getStatus());
+    
+    Assert.assertFalse(scanner.iterator().hasNext());
+
+    // add the row back
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+    Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+    
+    Assert.assertEquals("doe", scanner.iterator().next().getValue().toString());
+  }
+  
+  @Test
+  public void testFields() throws Exception {
+    String table = "foo2";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    Authorizations auths = new Authorizations("A", "B");
+    
+    conn.securityOperations().changeUserAuthorizations("root", auths);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(auths));
+    
+    ColumnVisibility cva = new ColumnVisibility("A");
+    ColumnVisibility cvb = new ColumnVisibility("B");
+    
+    ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva));
+    cm0.put("name", "last", cva, "doe");
+    cm0.put("name", "first", cva, "john");
+    cm0.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+    
+    Scanner scanner = conn.createScanner(table, auths);
+    scanner.setRange(new Range("99006"));
+    // TODO verify all columns
+    scanner.fetchColumn(new Text("tx"), new Text("seq"));
+    Entry<Key,Value> entry = scanner.iterator().next();
+    Assert.assertEquals("1", entry.getValue().toString());
+    long ts = entry.getKey().getTimestamp();
+    
+    // test wrong colf
+    ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("txA", "seq").setVisibility(cva).setValue("1"));
+    cm1.put("name", "last", cva, "Doe");
+    cm1.put("name", "first", cva, "John");
+    cm1.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm1).getStatus());
+    
+    // test wrong colq
+    ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seqA").setVisibility(cva).setValue("1"));
+    cm2.put("name", "last", cva, "Doe");
+    cm2.put("name", "first", cva, "John");
+    cm2.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm2).getStatus());
+    
+    // test wrong colv
+    ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"));
+    cm3.put("name", "last", cva, "Doe");
+    cm3.put("name", "first", cva, "John");
+    cm3.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm3).getStatus());
+
+    // test wrong timestamp
+    ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts + 1).setValue("1"));
+    cm4.put("name", "last", cva, "Doe");
+    cm4.put("name", "first", cva, "John");
+    cm4.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm4).getStatus());
+    
+    // test wrong timestamp
+    ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts - 1).setValue("1"));
+    cm5.put("name", "last", cva, "Doe");
+    cm5.put("name", "first", cva, "John");
+    cm5.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm5).getStatus());
+
+    // ensure no updates were made
+    entry = scanner.iterator().next();
+    Assert.assertEquals("1", entry.getValue().toString());
+
+    // set all columns correctly
+    ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts).setValue("1"));
+    cm6.put("name", "last", cva, "Doe");
+    cm6.put("name", "first", cva, "John");
+    cm6.put("tx", "seq", cva, "2");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm6).getStatus());
+
+    entry = scanner.iterator().next();
+    Assert.assertEquals("2", entry.getValue().toString());
+    
+    // TODO test each field w/ absence
+
+  }
+
+  @Test
+  public void testBadColVis() throws Exception {
+    // test when a user sets a col vis in a condition that can never be seen
+    String table = "foo3";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    Authorizations auths = new Authorizations("A", "B");
+    
+    conn.securityOperations().changeUserAuthorizations("root", auths);
+
+    Authorizations filteredAuths = new Authorizations("A");
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(filteredAuths));
+    
+    ColumnVisibility cva = new ColumnVisibility("A");
+    ColumnVisibility cvb = new ColumnVisibility("B");
+    ColumnVisibility cvc = new ColumnVisibility("C");
+    
+    // User has authorization, but didn't include it in the writer
+    ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb));
+    cm0.put("name", "last", cva, "doe");
+    cm0.put("name", "first", cva, "john");
+    cm0.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm0).getStatus());
+    
+    ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"));
+    cm1.put("name", "last", cva, "doe");
+    cm1.put("name", "first", cva, "john");
+    cm1.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm1).getStatus());
+
+    // User does not have the authorization
+    ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvc));
+    cm2.put("name", "last", cva, "doe");
+    cm2.put("name", "first", cva, "john");
+    cm2.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm2).getStatus());
+    
+    ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvc).setValue("1"));
+    cm3.put("name", "last", cva, "doe");
+    cm3.put("name", "first", cva, "john");
+    cm3.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm3).getStatus());
+
+    // if any visibility is bad, good visibilities don't override
+    ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva));
+
+    cm4.put("name", "last", cva, "doe");
+    cm4.put("name", "first", cva, "john");
+    cm4.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm4).getStatus());
+    
+    ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"), new Condition("tx", "seq")
+        .setVisibility(cva).setValue("1"));
+    cm5.put("name", "last", cva, "doe");
+    cm5.put("name", "first", cva, "john");
+    cm5.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm5).getStatus());
+
+    ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"),
+        new Condition("tx", "seq").setVisibility(cva));
+    cm6.put("name", "last", cva, "doe");
+    cm6.put("name", "first", cva, "john");
+    cm6.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm6).getStatus());
+
+    ConditionalMutation cm7 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva)
+        .setValue("1"));
+    cm7.put("name", "last", cva, "doe");
+    cm7.put("name", "first", cva, "john");
+    cm7.put("tx", "seq", cva, "1");
+    Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm7).getStatus());
+    
+    cw.close();
+
+    // test passing auths that exceed users configured auths
+    
+    Authorizations exceedingAuths = new Authorizations("A", "B", "D");
+    ConditionalWriter cw2 = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(exceedingAuths));
+    
+    ConditionalMutation cm8 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva)
+        .setValue("1"));
+    cm8.put("name", "last", cva, "doe");
+    cm8.put("name", "first", cva, "john");
+    cm8.put("tx", "seq", cva, "1");
+
+    try {
+      cw2.write(cm8).getStatus();
+      Assert.assertTrue(false);
+    } catch (AccumuloSecurityException ase) {}
+    
+
+    cw2.close();
+  }
+  
+  @Test
+  public void testConstraints() throws Exception {
+    // ensure constraint violations are properly reported
+    String table = "foo5";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    conn.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
+    conn.tableOperations().clone(table, table + "_clone", true, new HashMap<String,String>(), new HashSet<String>());
+    
+    Scanner scanner = conn.createScanner(table + "_clone", new Authorizations());
+
+    ConditionalWriter cw = conn.createConditionalWriter(table + "_clone", new ConditionalWriterConfig());
+
+    ConditionalMutation cm0 = new ConditionalMutation("99006+", new Condition("tx", "seq"));
+    cm0.put("tx", "seq", "1");
+    
+    Assert.assertEquals(Status.VIOLATED, cw.write(cm0).getStatus());
+    Assert.assertFalse(scanner.iterator().hasNext());
+    
+    ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+    Assert.assertTrue(scanner.iterator().hasNext());
+    
+    cw.close();
+
+  }
+
+  @Test
+  public void testIterators() throws Exception {
+    String table = "foo4";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table, false);
+    
+    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    
+    Mutation m = new Mutation("ACCUMULO-1000");
+    m.put("count", "comments", "1");
+    bw.addMutation(m);
+    bw.addMutation(m);
+    bw.addMutation(m);
+    
+    m = new Mutation("ACCUMULO-1001");
+    m.put("count2", "comments", "1");
+    bw.addMutation(m);
+    bw.addMutation(m);
+
+    m = new Mutation("ACCUMULO-1002");
+    m.put("count2", "comments", "1");
+    bw.addMutation(m);
+    bw.addMutation(m);
+
+    bw.close();
+    
+    IteratorSetting iterConfig = new IteratorSetting(10, SummingCombiner.class);
+    SummingCombiner.setEncodingType(iterConfig, Type.STRING);
+    SummingCombiner.setColumns(iterConfig, Collections.singletonList(new IteratorSetting.Column("count")));
+    
+    IteratorSetting iterConfig2 = new IteratorSetting(10, SummingCombiner.class);
+    SummingCombiner.setEncodingType(iterConfig2, Type.STRING);
+    SummingCombiner.setColumns(iterConfig2, Collections.singletonList(new IteratorSetting.Column("count2", "comments")));
+    
+    IteratorSetting iterConfig3 = new IteratorSetting(5, VersioningIterator.class);
+    VersioningIterator.setMaxVersions(iterConfig3, 1);
+
+    Scanner scanner = conn.createScanner(table, new Authorizations());
+    scanner.addScanIterator(iterConfig);
+    scanner.setRange(new Range("ACCUMULO-1000"));
+    scanner.fetchColumn(new Text("count"), new Text("comments"));
+    
+    Assert.assertEquals("3", scanner.iterator().next().getValue().toString());
+
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    ConditionalMutation cm0 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setValue("3"));
+    cm0.put("count", "comments", "1");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+    Assert.assertEquals("3", scanner.iterator().next().getValue().toString());
+    
+    ConditionalMutation cm1 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setIterators(iterConfig).setValue("3"));
+    cm1.put("count", "comments", "1");
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+    Assert.assertEquals("4", scanner.iterator().next().getValue().toString());
+    
+    ConditionalMutation cm2 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setValue("4"));
+    cm2.put("count", "comments", "1");
+    Assert.assertEquals(Status.REJECTED, cw.write(cm1).getStatus());
+    Assert.assertEquals("4", scanner.iterator().next().getValue().toString());
+    
+    // run test with multiple iterators passed in same batch and condition with two iterators
+
+    ConditionalMutation cm3 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setIterators(iterConfig).setValue("4"));
+    cm3.put("count", "comments", "1");
+    
+    ConditionalMutation cm4 = new ConditionalMutation("ACCUMULO-1001", new Condition("count2", "comments").setIterators(iterConfig2).setValue("2"));
+    cm4.put("count2", "comments", "1");
+    
+    ConditionalMutation cm5 = new ConditionalMutation("ACCUMULO-1002", new Condition("count2", "comments").setIterators(iterConfig2, iterConfig3).setValue("2"));
+    cm5.put("count2", "comments", "1");
+    
+    Iterator<Result> results = cw.write(Arrays.asList(cm3, cm4, cm5).iterator());
+    Map<String,Status> actual = new HashMap<String,Status>();
+    
+    while (results.hasNext()) {
+      Result result = results.next();
+      String k = new String(result.getMutation().getRow());
+      Assert.assertFalse(actual.containsKey(k));
+      actual.put(k, result.getStatus());
+    }
+
+    Map<String,Status> expected = new HashMap<String,Status>();
+    expected.put("ACCUMULO-1000", Status.ACCEPTED);
+    expected.put("ACCUMULO-1001", Status.ACCEPTED);
+    expected.put("ACCUMULO-1002", Status.REJECTED);
+    
+    Assert.assertEquals(expected, actual);
+
+    // TODO test w/ table that has iterators configured
+    
+    cw.close();
+  }
+
+  @Test
+  public void testBatch() throws Exception {
+    String table = "foo6";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    conn.securityOperations().changeUserAuthorizations("root", new Authorizations("A", "B"));
+    
+    ColumnVisibility cvab = new ColumnVisibility("A|B");
+    
+    ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+    
+    ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvab));
+    cm0.put("name", "last", cvab, "doe");
+    cm0.put("name", "first", cvab, "john");
+    cm0.put("tx", "seq", cvab, "1");
+    mutations.add(cm0);
+    
+    ConditionalMutation cm1 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvab));
+    cm1.put("name", "last", cvab, "doe");
+    cm1.put("name", "first", cvab, "jane");
+    cm1.put("tx", "seq", cvab, "1");
+    mutations.add(cm1);
+    
+    ConditionalMutation cm2 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvab));
+    cm2.put("name", "last", cvab, "doe");
+    cm2.put("name", "first", cvab, "jack");
+    cm2.put("tx", "seq", cvab, "1");
+    mutations.add(cm2);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(new Authorizations("A")));
+    Iterator<Result> results = cw.write(mutations.iterator());
+    int count = 0;
+    while (results.hasNext()) {
+      Result result = results.next();
+      Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+      count++;
+    }
+    
+    Assert.assertEquals(3, count);
+
+    Scanner scanner = conn.createScanner(table, new Authorizations("A"));
+    scanner.fetchColumn(new Text("tx"), new Text("seq"));
+    
+    for (String row : new String[] {"99006", "59056", "19059"}) {
+      scanner.setRange(new Range(row));
+      Assert.assertEquals("1", scanner.iterator().next().getValue().toString());
+    }
+
+    TreeSet<Text> splits = new TreeSet<Text>();
+    splits.add(new Text("7"));
+    splits.add(new Text("3"));
+    conn.tableOperations().addSplits(table, splits);
+
+    mutations.clear();
+
+    ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvab).setValue("1"));
+    cm3.put("name", "last", cvab, "Doe");
+    cm3.put("tx", "seq", cvab, "2");
+    mutations.add(cm3);
+    
+    ConditionalMutation cm4 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvab));
+    cm4.put("name", "last", cvab, "Doe");
+    cm4.put("tx", "seq", cvab, "1");
+    mutations.add(cm4);
+    
+    ConditionalMutation cm5 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvab).setValue("2"));
+    cm5.put("name", "last", cvab, "Doe");
+    cm5.put("tx", "seq", cvab, "3");
+    mutations.add(cm5);
+
+    results = cw.write(mutations.iterator());
+    int accepted = 0;
+    int rejected = 0;
+    while (results.hasNext()) {
+      Result result = results.next();
+      if (new String(result.getMutation().getRow()).equals("99006")) {
+        Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+        accepted++;
+      } else {
+        Assert.assertEquals(Status.REJECTED, result.getStatus());
+        rejected++;
+      }
+    }
+    
+    Assert.assertEquals(1, accepted);
+    Assert.assertEquals(2, rejected);
+
+    for (String row : new String[] {"59056", "19059"}) {
+      scanner.setRange(new Range(row));
+      Assert.assertEquals("1", scanner.iterator().next().getValue().toString());
+    }
+    
+    scanner.setRange(new Range("99006"));
+    Assert.assertEquals("2", scanner.iterator().next().getValue().toString());
+
+    scanner.clearColumns();
+    scanner.fetchColumn(new Text("name"), new Text("last"));
+    Assert.assertEquals("Doe", scanner.iterator().next().getValue().toString());
+    
+    cw.close();
+  }
+  
+  @Test
+  public void testBigBatch() throws Exception {
+    
+    String table = "foo100";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    conn.tableOperations().addSplits(table, nss("2", "4", "6"));
+    
+    UtilWaitThread.sleep(2000);
+
+    int num = 100;
+    
+    ArrayList<byte[]> rows = new ArrayList<byte[]>(num);
+    ArrayList<ConditionalMutation> cml = new ArrayList<ConditionalMutation>(num);
+    
+    Random r = new Random();
+    byte[] e = new byte[0];
+    
+    for (int i = 0; i < num; i++) {
+      rows.add(FastFormat.toZeroPaddedString(Math.abs(r.nextLong()), 16, 16, e));
+    }
+    
+    for (int i = 0; i < num; i++) {
+      ConditionalMutation cm = new ConditionalMutation(rows.get(i), new Condition("meta", "seq"));
+      
+      cm.put("meta", "seq", "1");
+      cm.put("meta", "tx", UUID.randomUUID().toString());
+      
+      cml.add(cm);
+    }
+
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+    Iterator<Result> results = cw.write(cml.iterator());
+
+    int count = 0;
+    
+    // TODO check got each row back
+    while (results.hasNext()) {
+      Result result = results.next();
+      Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+      count++;
+    }
+    
+    Assert.assertEquals(num, count);
+
+    ArrayList<ConditionalMutation> cml2 = new ArrayList<ConditionalMutation>(num);
+    
+    for (int i = 0; i < num; i++) {
+      ConditionalMutation cm = new ConditionalMutation(rows.get(i), new Condition("meta", "seq").setValue("1"));
+      
+      cm.put("meta", "seq", "2");
+      cm.put("meta", "tx", UUID.randomUUID().toString());
+      
+      cml2.add(cm);
+    }
+    
+    count = 0;
+
+    results = cw.write(cml2.iterator());
+    
+    while (results.hasNext()) {
+      Result result = results.next();
+      Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+      count++;
+    }
+    
+    Assert.assertEquals(num, count);
+    
+    cw.close();
+  }
+  
+  @Test
+  public void testBatchErrors() throws Exception {
+    
+    String table = "foo7";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    conn.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
+    conn.tableOperations().clone(table, table + "_clone", true, new HashMap<String,String>(), new HashSet<String>());
+
+    conn.securityOperations().changeUserAuthorizations("root", new Authorizations("A", "B"));
+    
+    ColumnVisibility cvaob = new ColumnVisibility("A|B");
+    ColumnVisibility cvaab = new ColumnVisibility("A&B");
+    
+    switch ((new Random()).nextInt(3)) {
+      case 1:
+        conn.tableOperations().addSplits(table, nss("6"));
+        break;
+      case 2:
+        conn.tableOperations().addSplits(table, nss("2", "95"));
+        break;
+    }
+    
+    ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+    
+    ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvaob));
+    cm0.put("name+", "last", cvaob, "doe");
+    cm0.put("name", "first", cvaob, "john");
+    cm0.put("tx", "seq", cvaob, "1");
+    mutations.add(cm0);
+    
+    ConditionalMutation cm1 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvaab));
+    cm1.put("name", "last", cvaab, "doe");
+    cm1.put("name", "first", cvaab, "jane");
+    cm1.put("tx", "seq", cvaab, "1");
+    mutations.add(cm1);
+    
+    ConditionalMutation cm2 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvaob));
+    cm2.put("name", "last", cvaob, "doe");
+    cm2.put("name", "first", cvaob, "jack");
+    cm2.put("tx", "seq", cvaob, "1");
+    mutations.add(cm2);
+    
+    ConditionalMutation cm3 = new ConditionalMutation("90909", new Condition("tx", "seq").setVisibility(cvaob).setValue("1"));
+    cm3.put("name", "last", cvaob, "doe");
+    cm3.put("name", "first", cvaob, "john");
+    cm3.put("tx", "seq", cvaob, "2");
+    mutations.add(cm3);
+
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(new Authorizations("A")));
+    Iterator<Result> results = cw.write(mutations.iterator());
+    HashSet<String> rows = new HashSet<String>();
+    while (results.hasNext()) {
+      Result result = results.next();
+      String row = new String(result.getMutation().getRow());
+      if (row.equals("19059")) {
+        Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+      } else if (row.equals("59056")) {
+        Assert.assertEquals(Status.INVISIBLE_VISIBILITY, result.getStatus());
+      } else if (row.equals("99006")) {
+        Assert.assertEquals(Status.VIOLATED, result.getStatus());
+      } else if (row.equals("90909")) {
+        Assert.assertEquals(Status.REJECTED, result.getStatus());
+      }
+      rows.add(row);
+    }
+    
+    Assert.assertEquals(4, rows.size());
+
+    Scanner scanner = conn.createScanner(table, new Authorizations("A"));
+    scanner.fetchColumn(new Text("tx"), new Text("seq"));
+    
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    Assert.assertEquals("1", iter.next().getValue().toString());
+    Assert.assertFalse(iter.hasNext());
+
+    cw.close();
+  }
+  
+  @Test
+  public void testSameRow() throws Exception {
+    // test multiple mutations for same row in same batch
+    
+    String table = "foo8";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+    
+    ConditionalMutation cm2 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+    cm2.put("tx", "seq", "2");
+    cm2.put("data", "x", "b");
+    
+    ConditionalMutation cm3 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+    cm3.put("tx", "seq", "2");
+    cm3.put("data", "x", "c");
+    
+    ConditionalMutation cm4 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+    cm4.put("tx", "seq", "2");
+    cm4.put("data", "x", "d");
+    
+    Iterator<Result> results = cw.write(Arrays.asList(cm2, cm3, cm4).iterator());
+    
+    int accepted = 0;
+    int rejected = 0;
+    int total = 0;
+    
+    while (results.hasNext()) {
+      Status status = results.next().getStatus();
+      if (status == Status.ACCEPTED)
+        accepted++;
+      if (status == Status.REJECTED)
+        rejected++;
+      total++;
+    }
+    
+    Assert.assertEquals(1, accepted);
+    Assert.assertEquals(2, rejected);
+    Assert.assertEquals(3, total);
+    
+    cw.close();
+  }
+
+  private static class Stats {
+    
+    ByteSequence row = null;
+    int seq;
+    long sum;
+    int data[] = new int[10];
+    
+    public Stats(Iterator<Entry<Key,Value>> iterator) {
+      while (iterator.hasNext()) {
+        Entry<Key,Value> entry = iterator.next();
+        
+        if (row == null)
+          row = entry.getKey().getRowData();
+
+        String cf = entry.getKey().getColumnFamilyData().toString();
+        String cq = entry.getKey().getColumnQualifierData().toString();
+        
+        if (cf.equals("data")) {
+          data[Integer.parseInt(cq)] = Integer.parseInt(entry.getValue().toString());
+        } else if (cf.equals("meta")) {
+          if (cq.equals("sum")) {
+            sum = Long.parseLong(entry.getValue().toString());
+          } else if (cq.equals("seq")) {
+            seq = Integer.parseInt(entry.getValue().toString());
+          }
+        }
+      }
+      
+      long sum2 = 0;
+      
+      for (int datum : data) {
+        sum2 += datum;
+      }
+      
+      Assert.assertEquals(sum2, sum);
+    }
+    
+    public Stats(ByteSequence row) {
+      this.row = row;
+      for (int i = 0; i < data.length; i++) {
+        this.data[i] = 0;
+      }
+      this.seq = -1;
+      this.sum = 0;
+    }
+
+    void set(int index, int value) {
+      sum -= data[index];
+      sum += value;
+      data[index] = value;
+    }
+    
+    ConditionalMutation toMutation() {
+      Condition cond = new Condition("meta", "seq");
+      if (seq >= 0)
+        cond.setValue(seq + "");
+      
+      ConditionalMutation cm = new ConditionalMutation(row, cond);
+      
+      cm.put("meta", "seq", (seq + 1) + "");
+      cm.put("meta", "sum", (sum) + "");
+      
+      for (int i = 0; i < data.length; i++) {
+        cm.put("data", i + "", data[i] + "");
+      }
+      
+      return cm;
+    }
+
+    public String toString() {
+      return row + " " + seq + " " + sum;
+    }
+  }
+
+  private static class MutatorTask implements Runnable {
+    String table;
+    ArrayList<ByteSequence> rows;
+    ConditionalWriter cw;
+    Connector conn;
+    AtomicBoolean failed;
+    
+    public MutatorTask(String table, Connector conn, ArrayList<ByteSequence> rows, ConditionalWriter cw, AtomicBoolean failed) {
+      this.table = table;
+      this.rows = rows;
+      this.conn = conn;
+      this.cw = cw;
+      this.failed = failed;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Random rand = new Random();
+        
+        Scanner scanner = new IsolatedScanner(conn.createScanner(table, Authorizations.EMPTY));
+        
+        for (int i = 0; i < 20; i++) {
+          int numRows = rand.nextInt(10) + 1;
+          
+          ArrayList<ByteSequence> changes = new ArrayList<ByteSequence>(numRows);
+          ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+          
+          for (int j = 0; j < numRows; j++)
+            changes.add(rows.get(rand.nextInt(rows.size())));
+          
+          for (ByteSequence row : changes) {
+            scanner.setRange(new Range(row.toString()));
+            Stats stats = new Stats(scanner.iterator());
+            stats.set(rand.nextInt(10), Math.abs(rand.nextInt()));
+            mutations.add(stats.toMutation());
+          }
+          
+          ArrayList<ByteSequence> changed = new ArrayList<ByteSequence>(numRows);
+          Iterator<Result> results = cw.write(mutations.iterator());
+          while (results.hasNext()) {
+            Result result = results.next();
+            changed.add(new ArrayByteSequence(result.getMutation().getRow()));
+          }
+          
+          Collections.sort(changes);
+          Collections.sort(changed);
+          
+          Assert.assertEquals(changes, changed);
+
+        }
+      
+      } catch (Exception e) {
+        e.printStackTrace();
+        failed.set(true);
+      }
+    }
+  }
+
+  @Test
+  public void testThreads() throws Exception {
+    // test multiple threads using a single conditional writer
+    
+    String table = "foo9";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    Random rand = new Random();
+    
+    switch (rand.nextInt(3)) {
+      case 1:
+        conn.tableOperations().addSplits(table, nss("4"));
+        break;
+      case 2:
+        conn.tableOperations().addSplits(table, nss("3", "5"));
+        break;
+    }
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    ArrayList<ByteSequence> rows = new ArrayList<ByteSequence>();
+
+    
+    for (int i = 0; i < 1000; i++) {
+      rows.add(new ArrayByteSequence(FastFormat.toZeroPaddedString(Math.abs(rand.nextLong()), 16, 16, new byte[0])));
+    }
+    
+    ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+    
+    for (ByteSequence row : rows)
+      mutations.add(new Stats(row).toMutation());
+    
+    ArrayList<ByteSequence> rows2 = new ArrayList<ByteSequence>();
+    Iterator<Result> results = cw.write(mutations.iterator());
+    while (results.hasNext()) {
+      Result result = results.next();
+      Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+      rows2.add(new ArrayByteSequence(result.getMutation().getRow()));
+    }
+    
+    Collections.sort(rows);
+    Collections.sort(rows2);
+    
+    Assert.assertEquals(rows, rows2);
+    
+    AtomicBoolean failed = new AtomicBoolean(false);
+
+    ExecutorService tp = Executors.newFixedThreadPool(5);
+    for (int i = 0; i < 5; i++) {
+      tp.submit(new MutatorTask(table, conn, rows, cw, failed));
+    }
+
+    tp.shutdown();
+
+    while (!tp.isTerminated()) {
+      tp.awaitTermination(1, TimeUnit.MINUTES);
+    }
+
+    Assert.assertFalse(failed.get());
+
+    Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
+    
+    RowIterator rowIter = new RowIterator(scanner);
+
+    while (rowIter.hasNext()) {
+      Iterator<Entry<Key,Value>> row = rowIter.next();
+      new Stats(row);
+    }
+  }
+
+  private SortedSet<Text> nss(String... splits) {
+    TreeSet<Text> ret = new TreeSet<Text>();
+    for (String split : splits)
+      ret.add(new Text(split));
+    
+    return ret;
+  }
+
+  @Test
+  public void testSecurity() throws Exception {
+    // test against table user does not have read and/or write permissions for
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.securityOperations().createLocalUser("user1", new PasswordToken("u1p"));
+    
+    conn.tableOperations().create("sect1");
+    conn.tableOperations().create("sect2");
+    conn.tableOperations().create("sect3");
+    
+    conn.securityOperations().grantTablePermission("user1", "sect1", TablePermission.READ);
+    conn.securityOperations().grantTablePermission("user1", "sect2", TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission("user1", "sect3", TablePermission.READ);
+    conn.securityOperations().grantTablePermission("user1", "sect3", TablePermission.WRITE);
+    
+    Connector conn2 = zki.getConnector("user1", new PasswordToken("u1p"));
+    
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    ConditionalWriter cw1 = conn2.createConditionalWriter("sect1", new ConditionalWriterConfig());
+    ConditionalWriter cw2 = conn2.createConditionalWriter("sect2", new ConditionalWriterConfig());
+    ConditionalWriter cw3 = conn2.createConditionalWriter("sect3", new ConditionalWriterConfig());
+    
+    Assert.assertEquals(Status.ACCEPTED, cw3.write(cm1).getStatus());
+    
+    try {
+      cw1.write(cm1).getStatus();
+      Assert.assertFalse(true);
+    } catch (AccumuloSecurityException ase) {
+      
+    }
+    
+    try {
+      cw2.write(cm1).getStatus();
+      Assert.assertFalse(true);
+    } catch (AccumuloSecurityException ase) {
+      
+    }
+
+  }
+
+
+  @Test
+  public void testTimeout() throws Exception {
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    String table = "fooT";
+    
+    conn.tableOperations().create(table);
+
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setTimeout(1, TimeUnit.SECONDS));
+
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    Assert.assertEquals(cw.write(cm1).getStatus(), Status.ACCEPTED);
+    
+    IteratorSetting is = new IteratorSetting(5, SlowIterator.class);
+    SlowIterator.setSeekSleepTime(is, 1500);
+    
+    ConditionalMutation cm2 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1").setIterators(is));
+    cm2.put("tx", "seq", "2");
+    cm2.put("data", "x", "b");
+    
+    Assert.assertEquals(cw.write(cm2).getStatus(), Status.UNKNOWN);
+    
+    Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
+    
+    for (Entry<Key,Value> entry : scanner) {
+      String cf = entry.getKey().getColumnFamilyData().toString();
+      String cq = entry.getKey().getColumnQualifierData().toString();
+      String val = entry.getValue().toString();
+      
+      if (cf.equals("tx") && cq.equals("seq"))
+        Assert.assertEquals("1", val);
+      else if (cf.equals("data") && cq.equals("x"))
+        Assert.assertEquals("a", val);
+      else
+        Assert.assertTrue(false);
+    }
+    
+    ConditionalMutation cm3 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+    cm3.put("tx", "seq", "2");
+    cm3.put("data", "x", "b");
+    
+    Assert.assertEquals(cw.write(cm3).getStatus(), Status.ACCEPTED);
+    
+    cw.close();
+
+  }
+
+  @Test
+  public void testDeleteTable() throws Exception {
+    String table = "foo12";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    try {
+      conn.createConditionalWriter(table, new ConditionalWriterConfig());
+      Assert.assertFalse(true);
+    } catch (TableNotFoundException e) {}
+    
+    conn.tableOperations().create(table);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    conn.tableOperations().delete(table);
+    
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    Result result = cw.write(cm1);
+    
+    try {
+      result.getStatus();
+      Assert.assertFalse(true);
+    } catch (AccumuloException ae) {
+      Assert.assertEquals(TableDeletedException.class, ae.getCause().getClass());
+    }
+    
+  }
+  
+  @Test
+  public void testOffline() throws Exception {
+    String table = "foo11";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    conn.tableOperations().offline(table);
+
+    waitForSingleTabletTableToGoOffline(table, zki);
+    
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    Result result = cw.write(cm1);
+    
+    try {
+      result.getStatus();
+      Assert.assertFalse(true);
+    } catch (AccumuloException ae) {
+      Assert.assertEquals(TableOfflineException.class, ae.getCause().getClass());
+    }
+    
+    cw.close();
+    
+    try {
+      conn.createConditionalWriter(table, new ConditionalWriterConfig());
+      Assert.assertFalse(true);
+    } catch (TableOfflineException e) {}
+  }
+
+  void waitForSingleTabletTableToGoOffline(String table, ZooKeeperInstance zki) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    TabletLocator locator = TabletLocator.getLocator(zki, new Text(Tables.getNameToIdMap(zki).get(table)));
+    while (locator.locateTablet(new Text("a"), false, false, CredentialHelper.create("root", new PasswordToken(secret), zki.getInstanceID())) != null) {
+      UtilWaitThread.sleep(50);
+      locator.invalidateCache();
+    }
+  }
+
+  @Test
+  public void testError() throws Exception {
+    String table = "foo10";
+    
+    ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+    Connector conn = zki.getConnector("root", new PasswordToken(secret));
+    
+    conn.tableOperations().create(table);
+    
+    ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+    
+    IteratorSetting iterSetting = new IteratorSetting(5, BadIterator.class);
+    
+    ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq").setIterators(iterSetting));
+    cm1.put("tx", "seq", "1");
+    cm1.put("data", "x", "a");
+    
+    Result result = cw.write(cm1);
+    
+    try {
+      result.getStatus();
+      Assert.assertFalse(true);
+    } catch (AccumuloException ae) {
+      
+    }
+    
+    cw.close();
+
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    cluster.stop();
+    // folder.delete();
+  }
+}


[5/6] ACCUMULO-1000 added conditional mutations to Accumulo

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
new file mode 100644
index 0000000..fe72fd6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
@@ -0,0 +1,516 @@
+/*
+ * 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.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TCMResult implements org.apache.thrift.TBase<TCMResult, TCMResult._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCMResult");
+
+  private static final org.apache.thrift.protocol.TField CMID_FIELD_DESC = new org.apache.thrift.protocol.TField("cmid", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TCMResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TCMResultTupleSchemeFactory());
+  }
+
+  public long cmid; // required
+  /**
+   * 
+   * @see TCMStatus
+   */
+  public TCMStatus status; // 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 {
+    CMID((short)1, "cmid"),
+    /**
+     * 
+     * @see TCMStatus
+     */
+    STATUS((short)2, "status");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CMID
+          return CMID;
+        case 2: // STATUS
+          return STATUS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __CMID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  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.CMID, new org.apache.thrift.meta_data.FieldMetaData("cmid", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TCMStatus.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCMResult.class, metaDataMap);
+  }
+
+  public TCMResult() {
+  }
+
+  public TCMResult(
+    long cmid,
+    TCMStatus status)
+  {
+    this();
+    this.cmid = cmid;
+    setCmidIsSet(true);
+    this.status = status;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TCMResult(TCMResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.cmid = other.cmid;
+    if (other.isSetStatus()) {
+      this.status = other.status;
+    }
+  }
+
+  public TCMResult deepCopy() {
+    return new TCMResult(this);
+  }
+
+  @Override
+  public void clear() {
+    setCmidIsSet(false);
+    this.cmid = 0;
+    this.status = null;
+  }
+
+  public long getCmid() {
+    return this.cmid;
+  }
+
+  public TCMResult setCmid(long cmid) {
+    this.cmid = cmid;
+    setCmidIsSet(true);
+    return this;
+  }
+
+  public void unsetCmid() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CMID_ISSET_ID);
+  }
+
+  /** Returns true if field cmid is set (has been assigned a value) and false otherwise */
+  public boolean isSetCmid() {
+    return EncodingUtils.testBit(__isset_bitfield, __CMID_ISSET_ID);
+  }
+
+  public void setCmidIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CMID_ISSET_ID, value);
+  }
+
+  /**
+   * 
+   * @see TCMStatus
+   */
+  public TCMStatus getStatus() {
+    return this.status;
+  }
+
+  /**
+   * 
+   * @see TCMStatus
+   */
+  public TCMResult setStatus(TCMStatus status) {
+    this.status = status;
+    return this;
+  }
+
+  public void unsetStatus() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean isSetStatus() {
+    return this.status != null;
+  }
+
+  public void setStatusIsSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CMID:
+      if (value == null) {
+        unsetCmid();
+      } else {
+        setCmid((Long)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unsetStatus();
+      } else {
+        setStatus((TCMStatus)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CMID:
+      return Long.valueOf(getCmid());
+
+    case STATUS:
+      return getStatus();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CMID:
+      return isSetCmid();
+    case STATUS:
+      return isSetStatus();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TCMResult)
+      return this.equals((TCMResult)that);
+    return false;
+  }
+
+  public boolean equals(TCMResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_cmid = true;
+    boolean that_present_cmid = true;
+    if (this_present_cmid || that_present_cmid) {
+      if (!(this_present_cmid && that_present_cmid))
+        return false;
+      if (this.cmid != that.cmid)
+        return false;
+    }
+
+    boolean this_present_status = true && this.isSetStatus();
+    boolean that_present_status = true && that.isSetStatus();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  public int compareTo(TCMResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TCMResult typedOther = (TCMResult)other;
+
+    lastComparison = Boolean.valueOf(isSetCmid()).compareTo(typedOther.isSetCmid());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCmid()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cmid, typedOther.cmid);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStatus()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TCMResult(");
+    boolean first = true;
+
+    sb.append("cmid:");
+    sb.append(this.cmid);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TCMResultStandardSchemeFactory implements SchemeFactory {
+    public TCMResultStandardScheme getScheme() {
+      return new TCMResultStandardScheme();
+    }
+  }
+
+  private static class TCMResultStandardScheme extends StandardScheme<TCMResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TCMResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CMID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.cmid = iprot.readI64();
+              struct.setCmidIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.status = TCMStatus.findByValue(iprot.readI32());
+              struct.setStatusIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TCMResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(CMID_FIELD_DESC);
+      oprot.writeI64(struct.cmid);
+      oprot.writeFieldEnd();
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        oprot.writeI32(struct.status.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TCMResultTupleSchemeFactory implements SchemeFactory {
+    public TCMResultTupleScheme getScheme() {
+      return new TCMResultTupleScheme();
+    }
+  }
+
+  private static class TCMResultTupleScheme extends TupleScheme<TCMResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TCMResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetCmid()) {
+        optionals.set(0);
+      }
+      if (struct.isSetStatus()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetCmid()) {
+        oprot.writeI64(struct.cmid);
+      }
+      if (struct.isSetStatus()) {
+        oprot.writeI32(struct.status.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TCMResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.cmid = iprot.readI64();
+        struct.setCmidIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.status = TCMStatus.findByValue(iprot.readI32());
+        struct.setStatusIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
new file mode 100644
index 0000000..6ef94d5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
@@ -0,0 +1,67 @@
+/*
+ * 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.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+@SuppressWarnings("all") public enum TCMStatus implements org.apache.thrift.TEnum {
+  ACCEPTED(0),
+  REJECTED(1),
+  VIOLATED(2),
+  IGNORED(3);
+
+  private final int value;
+
+  private TCMStatus(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 TCMStatus findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return ACCEPTED;
+      case 1:
+        return REJECTED;
+      case 2:
+        return VIOLATED;
+      case 3:
+        return IGNORED;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
new file mode 100644
index 0000000..40a12be
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
@@ -0,0 +1,1049 @@
+/*
+ * 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.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TCondition implements org.apache.thrift.TBase<TCondition, TCondition._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCondition");
+
+  private static final org.apache.thrift.protocol.TField CF_FIELD_DESC = new org.apache.thrift.protocol.TField("cf", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField CQ_FIELD_DESC = new org.apache.thrift.protocol.TField("cq", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CV_FIELD_DESC = new org.apache.thrift.protocol.TField("cv", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TS_FIELD_DESC = new org.apache.thrift.protocol.TField("ts", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField HAS_TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("hasTimestamp", org.apache.thrift.protocol.TType.BOOL, (short)5);
+  private static final org.apache.thrift.protocol.TField VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("val", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField ITERATORS_FIELD_DESC = new org.apache.thrift.protocol.TField("iterators", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TConditionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TConditionTupleSchemeFactory());
+  }
+
+  public ByteBuffer cf; // required
+  public ByteBuffer cq; // required
+  public ByteBuffer cv; // required
+  public long ts; // required
+  public boolean hasTimestamp; // required
+  public ByteBuffer val; // required
+  public ByteBuffer iterators; // 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 {
+    CF((short)1, "cf"),
+    CQ((short)2, "cq"),
+    CV((short)3, "cv"),
+    TS((short)4, "ts"),
+    HAS_TIMESTAMP((short)5, "hasTimestamp"),
+    VAL((short)6, "val"),
+    ITERATORS((short)7, "iterators");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CF
+          return CF;
+        case 2: // CQ
+          return CQ;
+        case 3: // CV
+          return CV;
+        case 4: // TS
+          return TS;
+        case 5: // HAS_TIMESTAMP
+          return HAS_TIMESTAMP;
+        case 6: // VAL
+          return VAL;
+        case 7: // ITERATORS
+          return ITERATORS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TS_ISSET_ID = 0;
+  private static final int __HASTIMESTAMP_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  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.CF, new org.apache.thrift.meta_data.FieldMetaData("cf", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.CQ, new org.apache.thrift.meta_data.FieldMetaData("cq", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.CV, new org.apache.thrift.meta_data.FieldMetaData("cv", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.TS, new org.apache.thrift.meta_data.FieldMetaData("ts", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.HAS_TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("hasTimestamp", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VAL, new org.apache.thrift.meta_data.FieldMetaData("val", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.ITERATORS, new org.apache.thrift.meta_data.FieldMetaData("iterators", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCondition.class, metaDataMap);
+  }
+
+  public TCondition() {
+  }
+
+  public TCondition(
+    ByteBuffer cf,
+    ByteBuffer cq,
+    ByteBuffer cv,
+    long ts,
+    boolean hasTimestamp,
+    ByteBuffer val,
+    ByteBuffer iterators)
+  {
+    this();
+    this.cf = cf;
+    this.cq = cq;
+    this.cv = cv;
+    this.ts = ts;
+    setTsIsSet(true);
+    this.hasTimestamp = hasTimestamp;
+    setHasTimestampIsSet(true);
+    this.val = val;
+    this.iterators = iterators;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TCondition(TCondition other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCf()) {
+      this.cf = org.apache.thrift.TBaseHelper.copyBinary(other.cf);
+;
+    }
+    if (other.isSetCq()) {
+      this.cq = org.apache.thrift.TBaseHelper.copyBinary(other.cq);
+;
+    }
+    if (other.isSetCv()) {
+      this.cv = org.apache.thrift.TBaseHelper.copyBinary(other.cv);
+;
+    }
+    this.ts = other.ts;
+    this.hasTimestamp = other.hasTimestamp;
+    if (other.isSetVal()) {
+      this.val = org.apache.thrift.TBaseHelper.copyBinary(other.val);
+;
+    }
+    if (other.isSetIterators()) {
+      this.iterators = org.apache.thrift.TBaseHelper.copyBinary(other.iterators);
+;
+    }
+  }
+
+  public TCondition deepCopy() {
+    return new TCondition(this);
+  }
+
+  @Override
+  public void clear() {
+    this.cf = null;
+    this.cq = null;
+    this.cv = null;
+    setTsIsSet(false);
+    this.ts = 0;
+    setHasTimestampIsSet(false);
+    this.hasTimestamp = false;
+    this.val = null;
+    this.iterators = null;
+  }
+
+  public byte[] getCf() {
+    setCf(org.apache.thrift.TBaseHelper.rightSize(cf));
+    return cf == null ? null : cf.array();
+  }
+
+  public ByteBuffer bufferForCf() {
+    return cf;
+  }
+
+  public TCondition setCf(byte[] cf) {
+    setCf(cf == null ? (ByteBuffer)null : ByteBuffer.wrap(cf));
+    return this;
+  }
+
+  public TCondition setCf(ByteBuffer cf) {
+    this.cf = cf;
+    return this;
+  }
+
+  public void unsetCf() {
+    this.cf = null;
+  }
+
+  /** Returns true if field cf is set (has been assigned a value) and false otherwise */
+  public boolean isSetCf() {
+    return this.cf != null;
+  }
+
+  public void setCfIsSet(boolean value) {
+    if (!value) {
+      this.cf = null;
+    }
+  }
+
+  public byte[] getCq() {
+    setCq(org.apache.thrift.TBaseHelper.rightSize(cq));
+    return cq == null ? null : cq.array();
+  }
+
+  public ByteBuffer bufferForCq() {
+    return cq;
+  }
+
+  public TCondition setCq(byte[] cq) {
+    setCq(cq == null ? (ByteBuffer)null : ByteBuffer.wrap(cq));
+    return this;
+  }
+
+  public TCondition setCq(ByteBuffer cq) {
+    this.cq = cq;
+    return this;
+  }
+
+  public void unsetCq() {
+    this.cq = null;
+  }
+
+  /** Returns true if field cq is set (has been assigned a value) and false otherwise */
+  public boolean isSetCq() {
+    return this.cq != null;
+  }
+
+  public void setCqIsSet(boolean value) {
+    if (!value) {
+      this.cq = null;
+    }
+  }
+
+  public byte[] getCv() {
+    setCv(org.apache.thrift.TBaseHelper.rightSize(cv));
+    return cv == null ? null : cv.array();
+  }
+
+  public ByteBuffer bufferForCv() {
+    return cv;
+  }
+
+  public TCondition setCv(byte[] cv) {
+    setCv(cv == null ? (ByteBuffer)null : ByteBuffer.wrap(cv));
+    return this;
+  }
+
+  public TCondition setCv(ByteBuffer cv) {
+    this.cv = cv;
+    return this;
+  }
+
+  public void unsetCv() {
+    this.cv = null;
+  }
+
+  /** Returns true if field cv is set (has been assigned a value) and false otherwise */
+  public boolean isSetCv() {
+    return this.cv != null;
+  }
+
+  public void setCvIsSet(boolean value) {
+    if (!value) {
+      this.cv = null;
+    }
+  }
+
+  public long getTs() {
+    return this.ts;
+  }
+
+  public TCondition setTs(long ts) {
+    this.ts = ts;
+    setTsIsSet(true);
+    return this;
+  }
+
+  public void unsetTs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TS_ISSET_ID);
+  }
+
+  /** Returns true if field ts is set (has been assigned a value) and false otherwise */
+  public boolean isSetTs() {
+    return EncodingUtils.testBit(__isset_bitfield, __TS_ISSET_ID);
+  }
+
+  public void setTsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TS_ISSET_ID, value);
+  }
+
+  public boolean isHasTimestamp() {
+    return this.hasTimestamp;
+  }
+
+  public TCondition setHasTimestamp(boolean hasTimestamp) {
+    this.hasTimestamp = hasTimestamp;
+    setHasTimestampIsSet(true);
+    return this;
+  }
+
+  public void unsetHasTimestamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID);
+  }
+
+  /** Returns true if field hasTimestamp is set (has been assigned a value) and false otherwise */
+  public boolean isSetHasTimestamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID);
+  }
+
+  public void setHasTimestampIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID, value);
+  }
+
+  public byte[] getVal() {
+    setVal(org.apache.thrift.TBaseHelper.rightSize(val));
+    return val == null ? null : val.array();
+  }
+
+  public ByteBuffer bufferForVal() {
+    return val;
+  }
+
+  public TCondition setVal(byte[] val) {
+    setVal(val == null ? (ByteBuffer)null : ByteBuffer.wrap(val));
+    return this;
+  }
+
+  public TCondition setVal(ByteBuffer val) {
+    this.val = val;
+    return this;
+  }
+
+  public void unsetVal() {
+    this.val = null;
+  }
+
+  /** Returns true if field val is set (has been assigned a value) and false otherwise */
+  public boolean isSetVal() {
+    return this.val != null;
+  }
+
+  public void setValIsSet(boolean value) {
+    if (!value) {
+      this.val = null;
+    }
+  }
+
+  public byte[] getIterators() {
+    setIterators(org.apache.thrift.TBaseHelper.rightSize(iterators));
+    return iterators == null ? null : iterators.array();
+  }
+
+  public ByteBuffer bufferForIterators() {
+    return iterators;
+  }
+
+  public TCondition setIterators(byte[] iterators) {
+    setIterators(iterators == null ? (ByteBuffer)null : ByteBuffer.wrap(iterators));
+    return this;
+  }
+
+  public TCondition setIterators(ByteBuffer iterators) {
+    this.iterators = iterators;
+    return this;
+  }
+
+  public void unsetIterators() {
+    this.iterators = null;
+  }
+
+  /** Returns true if field iterators is set (has been assigned a value) and false otherwise */
+  public boolean isSetIterators() {
+    return this.iterators != null;
+  }
+
+  public void setIteratorsIsSet(boolean value) {
+    if (!value) {
+      this.iterators = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CF:
+      if (value == null) {
+        unsetCf();
+      } else {
+        setCf((ByteBuffer)value);
+      }
+      break;
+
+    case CQ:
+      if (value == null) {
+        unsetCq();
+      } else {
+        setCq((ByteBuffer)value);
+      }
+      break;
+
+    case CV:
+      if (value == null) {
+        unsetCv();
+      } else {
+        setCv((ByteBuffer)value);
+      }
+      break;
+
+    case TS:
+      if (value == null) {
+        unsetTs();
+      } else {
+        setTs((Long)value);
+      }
+      break;
+
+    case HAS_TIMESTAMP:
+      if (value == null) {
+        unsetHasTimestamp();
+      } else {
+        setHasTimestamp((Boolean)value);
+      }
+      break;
+
+    case VAL:
+      if (value == null) {
+        unsetVal();
+      } else {
+        setVal((ByteBuffer)value);
+      }
+      break;
+
+    case ITERATORS:
+      if (value == null) {
+        unsetIterators();
+      } else {
+        setIterators((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CF:
+      return getCf();
+
+    case CQ:
+      return getCq();
+
+    case CV:
+      return getCv();
+
+    case TS:
+      return Long.valueOf(getTs());
+
+    case HAS_TIMESTAMP:
+      return Boolean.valueOf(isHasTimestamp());
+
+    case VAL:
+      return getVal();
+
+    case ITERATORS:
+      return getIterators();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CF:
+      return isSetCf();
+    case CQ:
+      return isSetCq();
+    case CV:
+      return isSetCv();
+    case TS:
+      return isSetTs();
+    case HAS_TIMESTAMP:
+      return isSetHasTimestamp();
+    case VAL:
+      return isSetVal();
+    case ITERATORS:
+      return isSetIterators();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TCondition)
+      return this.equals((TCondition)that);
+    return false;
+  }
+
+  public boolean equals(TCondition that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_cf = true && this.isSetCf();
+    boolean that_present_cf = true && that.isSetCf();
+    if (this_present_cf || that_present_cf) {
+      if (!(this_present_cf && that_present_cf))
+        return false;
+      if (!this.cf.equals(that.cf))
+        return false;
+    }
+
+    boolean this_present_cq = true && this.isSetCq();
+    boolean that_present_cq = true && that.isSetCq();
+    if (this_present_cq || that_present_cq) {
+      if (!(this_present_cq && that_present_cq))
+        return false;
+      if (!this.cq.equals(that.cq))
+        return false;
+    }
+
+    boolean this_present_cv = true && this.isSetCv();
+    boolean that_present_cv = true && that.isSetCv();
+    if (this_present_cv || that_present_cv) {
+      if (!(this_present_cv && that_present_cv))
+        return false;
+      if (!this.cv.equals(that.cv))
+        return false;
+    }
+
+    boolean this_present_ts = true;
+    boolean that_present_ts = true;
+    if (this_present_ts || that_present_ts) {
+      if (!(this_present_ts && that_present_ts))
+        return false;
+      if (this.ts != that.ts)
+        return false;
+    }
+
+    boolean this_present_hasTimestamp = true;
+    boolean that_present_hasTimestamp = true;
+    if (this_present_hasTimestamp || that_present_hasTimestamp) {
+      if (!(this_present_hasTimestamp && that_present_hasTimestamp))
+        return false;
+      if (this.hasTimestamp != that.hasTimestamp)
+        return false;
+    }
+
+    boolean this_present_val = true && this.isSetVal();
+    boolean that_present_val = true && that.isSetVal();
+    if (this_present_val || that_present_val) {
+      if (!(this_present_val && that_present_val))
+        return false;
+      if (!this.val.equals(that.val))
+        return false;
+    }
+
+    boolean this_present_iterators = true && this.isSetIterators();
+    boolean that_present_iterators = true && that.isSetIterators();
+    if (this_present_iterators || that_present_iterators) {
+      if (!(this_present_iterators && that_present_iterators))
+        return false;
+      if (!this.iterators.equals(that.iterators))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  public int compareTo(TCondition other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TCondition typedOther = (TCondition)other;
+
+    lastComparison = Boolean.valueOf(isSetCf()).compareTo(typedOther.isSetCf());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCf()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cf, typedOther.cf);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCq()).compareTo(typedOther.isSetCq());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCq()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cq, typedOther.cq);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCv()).compareTo(typedOther.isSetCv());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCv()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cv, typedOther.cv);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTs()).compareTo(typedOther.isSetTs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ts, typedOther.ts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetHasTimestamp()).compareTo(typedOther.isSetHasTimestamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetHasTimestamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasTimestamp, typedOther.hasTimestamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetVal()).compareTo(typedOther.isSetVal());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetVal()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.val, typedOther.val);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIterators()).compareTo(typedOther.isSetIterators());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIterators()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.iterators, typedOther.iterators);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TCondition(");
+    boolean first = true;
+
+    sb.append("cf:");
+    if (this.cf == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.cf, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("cq:");
+    if (this.cq == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.cq, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("cv:");
+    if (this.cv == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.cv, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("ts:");
+    sb.append(this.ts);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("hasTimestamp:");
+    sb.append(this.hasTimestamp);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("val:");
+    if (this.val == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.val, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("iterators:");
+    if (this.iterators == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.iterators, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TConditionStandardSchemeFactory implements SchemeFactory {
+    public TConditionStandardScheme getScheme() {
+      return new TConditionStandardScheme();
+    }
+  }
+
+  private static class TConditionStandardScheme extends StandardScheme<TCondition> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TCondition struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CF
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.cf = iprot.readBinary();
+              struct.setCfIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // CQ
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.cq = iprot.readBinary();
+              struct.setCqIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CV
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.cv = iprot.readBinary();
+              struct.setCvIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.ts = iprot.readI64();
+              struct.setTsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // HAS_TIMESTAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.hasTimestamp = iprot.readBool();
+              struct.setHasTimestampIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // VAL
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.val = iprot.readBinary();
+              struct.setValIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // ITERATORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.iterators = iprot.readBinary();
+              struct.setIteratorsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TCondition struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.cf != null) {
+        oprot.writeFieldBegin(CF_FIELD_DESC);
+        oprot.writeBinary(struct.cf);
+        oprot.writeFieldEnd();
+      }
+      if (struct.cq != null) {
+        oprot.writeFieldBegin(CQ_FIELD_DESC);
+        oprot.writeBinary(struct.cq);
+        oprot.writeFieldEnd();
+      }
+      if (struct.cv != null) {
+        oprot.writeFieldBegin(CV_FIELD_DESC);
+        oprot.writeBinary(struct.cv);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TS_FIELD_DESC);
+      oprot.writeI64(struct.ts);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(HAS_TIMESTAMP_FIELD_DESC);
+      oprot.writeBool(struct.hasTimestamp);
+      oprot.writeFieldEnd();
+      if (struct.val != null) {
+        oprot.writeFieldBegin(VAL_FIELD_DESC);
+        oprot.writeBinary(struct.val);
+        oprot.writeFieldEnd();
+      }
+      if (struct.iterators != null) {
+        oprot.writeFieldBegin(ITERATORS_FIELD_DESC);
+        oprot.writeBinary(struct.iterators);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TConditionTupleSchemeFactory implements SchemeFactory {
+    public TConditionTupleScheme getScheme() {
+      return new TConditionTupleScheme();
+    }
+  }
+
+  private static class TConditionTupleScheme extends TupleScheme<TCondition> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TCondition struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetCf()) {
+        optionals.set(0);
+      }
+      if (struct.isSetCq()) {
+        optionals.set(1);
+      }
+      if (struct.isSetCv()) {
+        optionals.set(2);
+      }
+      if (struct.isSetTs()) {
+        optionals.set(3);
+      }
+      if (struct.isSetHasTimestamp()) {
+        optionals.set(4);
+      }
+      if (struct.isSetVal()) {
+        optionals.set(5);
+      }
+      if (struct.isSetIterators()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
+      if (struct.isSetCf()) {
+        oprot.writeBinary(struct.cf);
+      }
+      if (struct.isSetCq()) {
+        oprot.writeBinary(struct.cq);
+      }
+      if (struct.isSetCv()) {
+        oprot.writeBinary(struct.cv);
+      }
+      if (struct.isSetTs()) {
+        oprot.writeI64(struct.ts);
+      }
+      if (struct.isSetHasTimestamp()) {
+        oprot.writeBool(struct.hasTimestamp);
+      }
+      if (struct.isSetVal()) {
+        oprot.writeBinary(struct.val);
+      }
+      if (struct.isSetIterators()) {
+        oprot.writeBinary(struct.iterators);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TCondition struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(7);
+      if (incoming.get(0)) {
+        struct.cf = iprot.readBinary();
+        struct.setCfIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.cq = iprot.readBinary();
+        struct.setCqIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.cv = iprot.readBinary();
+        struct.setCvIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.ts = iprot.readI64();
+        struct.setTsIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.hasTimestamp = iprot.readBool();
+        struct.setHasTimestampIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.val = iprot.readBinary();
+        struct.setValIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.iterators = iprot.readBinary();
+        struct.setIteratorsIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
new file mode 100644
index 0000000..1928803
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
@@ -0,0 +1,659 @@
+/*
+ * 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.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TConditionalMutation implements org.apache.thrift.TBase<TConditionalMutation, TConditionalMutation._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TConditionalMutation");
+
+  private static final org.apache.thrift.protocol.TField CONDITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("conditions", org.apache.thrift.protocol.TType.LIST, (short)1);
+  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)2);
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TConditionalMutationStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TConditionalMutationTupleSchemeFactory());
+  }
+
+  public List<TCondition> conditions; // required
+  public TMutation mutation; // required
+  public long id; // 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 {
+    CONDITIONS((short)1, "conditions"),
+    MUTATION((short)2, "mutation"),
+    ID((short)3, "id");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CONDITIONS
+          return CONDITIONS;
+        case 2: // MUTATION
+          return MUTATION;
+        case 3: // ID
+          return ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  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.CONDITIONS, new org.apache.thrift.meta_data.FieldMetaData("conditions", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCondition.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, TMutation.class)));
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TConditionalMutation.class, metaDataMap);
+  }
+
+  public TConditionalMutation() {
+  }
+
+  public TConditionalMutation(
+    List<TCondition> conditions,
+    TMutation mutation,
+    long id)
+  {
+    this();
+    this.conditions = conditions;
+    this.mutation = mutation;
+    this.id = id;
+    setIdIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TConditionalMutation(TConditionalMutation other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetConditions()) {
+      List<TCondition> __this__conditions = new ArrayList<TCondition>();
+      for (TCondition other_element : other.conditions) {
+        __this__conditions.add(new TCondition(other_element));
+      }
+      this.conditions = __this__conditions;
+    }
+    if (other.isSetMutation()) {
+      this.mutation = new TMutation(other.mutation);
+    }
+    this.id = other.id;
+  }
+
+  public TConditionalMutation deepCopy() {
+    return new TConditionalMutation(this);
+  }
+
+  @Override
+  public void clear() {
+    this.conditions = null;
+    this.mutation = null;
+    setIdIsSet(false);
+    this.id = 0;
+  }
+
+  public int getConditionsSize() {
+    return (this.conditions == null) ? 0 : this.conditions.size();
+  }
+
+  public java.util.Iterator<TCondition> getConditionsIterator() {
+    return (this.conditions == null) ? null : this.conditions.iterator();
+  }
+
+  public void addToConditions(TCondition elem) {
+    if (this.conditions == null) {
+      this.conditions = new ArrayList<TCondition>();
+    }
+    this.conditions.add(elem);
+  }
+
+  public List<TCondition> getConditions() {
+    return this.conditions;
+  }
+
+  public TConditionalMutation setConditions(List<TCondition> conditions) {
+    this.conditions = conditions;
+    return this;
+  }
+
+  public void unsetConditions() {
+    this.conditions = null;
+  }
+
+  /** Returns true if field conditions is set (has been assigned a value) and false otherwise */
+  public boolean isSetConditions() {
+    return this.conditions != null;
+  }
+
+  public void setConditionsIsSet(boolean value) {
+    if (!value) {
+      this.conditions = null;
+    }
+  }
+
+  public TMutation getMutation() {
+    return this.mutation;
+  }
+
+  public TConditionalMutation setMutation(TMutation mutation) {
+    this.mutation = mutation;
+    return this;
+  }
+
+  public void unsetMutation() {
+    this.mutation = null;
+  }
+
+  /** Returns true if field mutation is set (has been assigned a value) and false otherwise */
+  public boolean isSetMutation() {
+    return this.mutation != null;
+  }
+
+  public void setMutationIsSet(boolean value) {
+    if (!value) {
+      this.mutation = null;
+    }
+  }
+
+  public long getId() {
+    return this.id;
+  }
+
+  public TConditionalMutation setId(long id) {
+    this.id = id;
+    setIdIsSet(true);
+    return this;
+  }
+
+  public void unsetId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean isSetId() {
+    return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
+  }
+
+  public void setIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CONDITIONS:
+      if (value == null) {
+        unsetConditions();
+      } else {
+        setConditions((List<TCondition>)value);
+      }
+      break;
+
+    case MUTATION:
+      if (value == null) {
+        unsetMutation();
+      } else {
+        setMutation((TMutation)value);
+      }
+      break;
+
+    case ID:
+      if (value == null) {
+        unsetId();
+      } else {
+        setId((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CONDITIONS:
+      return getConditions();
+
+    case MUTATION:
+      return getMutation();
+
+    case ID:
+      return Long.valueOf(getId());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CONDITIONS:
+      return isSetConditions();
+    case MUTATION:
+      return isSetMutation();
+    case ID:
+      return isSetId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TConditionalMutation)
+      return this.equals((TConditionalMutation)that);
+    return false;
+  }
+
+  public boolean equals(TConditionalMutation that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_conditions = true && this.isSetConditions();
+    boolean that_present_conditions = true && that.isSetConditions();
+    if (this_present_conditions || that_present_conditions) {
+      if (!(this_present_conditions && that_present_conditions))
+        return false;
+      if (!this.conditions.equals(that.conditions))
+        return false;
+    }
+
+    boolean this_present_mutation = true && this.isSetMutation();
+    boolean that_present_mutation = true && that.isSetMutation();
+    if (this_present_mutation || that_present_mutation) {
+      if (!(this_present_mutation && that_present_mutation))
+        return false;
+      if (!this.mutation.equals(that.mutation))
+        return false;
+    }
+
+    boolean this_present_id = true;
+    boolean that_present_id = true;
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (this.id != that.id)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  public int compareTo(TConditionalMutation other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TConditionalMutation typedOther = (TConditionalMutation)other;
+
+    lastComparison = Boolean.valueOf(isSetConditions()).compareTo(typedOther.isSetConditions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConditions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.conditions, typedOther.conditions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMutation()).compareTo(typedOther.isSetMutation());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMutation()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutation, typedOther.mutation);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TConditionalMutation(");
+    boolean first = true;
+
+    sb.append("conditions:");
+    if (this.conditions == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.conditions);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("mutation:");
+    if (this.mutation == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.mutation);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("id:");
+    sb.append(this.id);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (mutation != null) {
+      mutation.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TConditionalMutationStandardSchemeFactory implements SchemeFactory {
+    public TConditionalMutationStandardScheme getScheme() {
+      return new TConditionalMutationStandardScheme();
+    }
+  }
+
+  private static class TConditionalMutationStandardScheme extends StandardScheme<TConditionalMutation> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TConditionalMutation struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CONDITIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list106 = iprot.readListBegin();
+                struct.conditions = new ArrayList<TCondition>(_list106.size);
+                for (int _i107 = 0; _i107 < _list106.size; ++_i107)
+                {
+                  TCondition _elem108; // required
+                  _elem108 = new TCondition();
+                  _elem108.read(iprot);
+                  struct.conditions.add(_elem108);
+                }
+                iprot.readListEnd();
+              }
+              struct.setConditionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // MUTATION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.mutation = new TMutation();
+              struct.mutation.read(iprot);
+              struct.setMutationIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.id = iprot.readI64();
+              struct.setIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TConditionalMutation struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.conditions != null) {
+        oprot.writeFieldBegin(CONDITIONS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.conditions.size()));
+          for (TCondition _iter109 : struct.conditions)
+          {
+            _iter109.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.mutation != null) {
+        oprot.writeFieldBegin(MUTATION_FIELD_DESC);
+        struct.mutation.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(ID_FIELD_DESC);
+      oprot.writeI64(struct.id);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TConditionalMutationTupleSchemeFactory implements SchemeFactory {
+    public TConditionalMutationTupleScheme getScheme() {
+      return new TConditionalMutationTupleScheme();
+    }
+  }
+
+  private static class TConditionalMutationTupleScheme extends TupleScheme<TConditionalMutation> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TConditionalMutation struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetConditions()) {
+        optionals.set(0);
+      }
+      if (struct.isSetMutation()) {
+        optionals.set(1);
+      }
+      if (struct.isSetId()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetConditions()) {
+        {
+          oprot.writeI32(struct.conditions.size());
+          for (TCondition _iter110 : struct.conditions)
+          {
+            _iter110.write(oprot);
+          }
+        }
+      }
+      if (struct.isSetMutation()) {
+        struct.mutation.write(oprot);
+      }
+      if (struct.isSetId()) {
+        oprot.writeI64(struct.id);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TConditionalMutation struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.conditions = new ArrayList<TCondition>(_list111.size);
+          for (int _i112 = 0; _i112 < _list111.size; ++_i112)
+          {
+            TCondition _elem113; // required
+            _elem113 = new TCondition();
+            _elem113.read(iprot);
+            struct.conditions.add(_elem113);
+          }
+        }
+        struct.setConditionsIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.mutation = new TMutation();
+        struct.mutation.read(iprot);
+        struct.setMutationIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.id = iprot.readI64();
+        struct.setIdIsSet(true);
+      }
+    }
+  }
+
+}
+


[6/6] git commit: ACCUMULO-1000 added conditional mutations to Accumulo

Posted by kt...@apache.org.
ACCUMULO-1000 added conditional mutations to Accumulo


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

Branch: refs/heads/master
Commit: 9dc244484b4d35859d4d22b27580a47ae7da0e1a
Parents: 77cac56
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jul 24 15:48:48 2013 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jul 24 15:48:48 2013 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ConditionalWriter.java |  138 +
 .../core/client/ConditionalWriterConfig.java    |  118 +
 .../apache/accumulo/core/client/Connector.java  |   15 +
 .../core/client/impl/CompressedIterators.java   |  131 +
 .../core/client/impl/ConditionalWriterImpl.java |  794 ++++
 .../core/client/impl/ConnectorImpl.java         |    8 +-
 .../core/client/impl/RootTabletLocator.java     |    8 +-
 .../accumulo/core/client/impl/ServerClient.java |    4 +-
 .../core/client/impl/TabletLocator.java         |   16 +-
 .../core/client/impl/TabletLocatorImpl.java     |   16 +-
 .../client/impl/TabletServerBatchWriter.java    |   34 +-
 .../core/client/impl/TimeoutTabletLocator.java  |    4 +-
 .../core/client/mock/MockConnector.java         |    8 +
 .../core/client/mock/MockTabletLocator.java     |    8 +-
 .../accumulo/core/data/ArrayByteSequence.java   |   13 +
 .../apache/accumulo/core/data/Condition.java    |  148 +
 .../accumulo/core/data/ConditionalMutation.java |   83 +
 .../org/apache/accumulo/core/data/Mutation.java |  130 +-
 .../accumulo/core/data/thrift/TCMResult.java    |  516 ++
 .../accumulo/core/data/thrift/TCMStatus.java    |   67 +
 .../accumulo/core/data/thrift/TCondition.java   | 1049 +++++
 .../core/data/thrift/TConditionalMutation.java  |  659 +++
 .../core/data/thrift/TConditionalSession.java   |  578 +++
 .../accumulo/core/file/rfile/RelativeKey.java   |   33 +-
 .../iterators/system/ColumnQualifierFilter.java |    5 +-
 .../thrift/TabletClientService.java             | 4395 +++++++++++++++++-
 .../accumulo/core/util/ByteBufferUtil.java      |   13 +
 .../core/util/UnsynchronizedBuffer.java         |  195 +
 core/src/main/thrift/data.thrift                |   36 +
 core/src/main/thrift/tabletserver.thrift        |   10 +-
 .../core/client/impl/TabletLocatorImplTest.java |    6 +-
 .../core/file/rfile/RelativeKeyTest.java        |   31 +-
 .../server/data/ServerConditionalMutation.java  |   58 +
 .../server/security/SecurityOperation.java      |    7 +
 .../tabletserver/ConditionalMutationSet.java    |   91 +
 .../accumulo/server/tabletserver/RowLocks.java  |  162 +
 .../accumulo/server/tabletserver/Tablet.java    |    8 +-
 .../server/tabletserver/TabletServer.java       |  308 +-
 .../server/client/BulkImporterTest.java         |    7 +-
 .../accumulo/test/FaultyConditionalWriter.java  |   81 +
 .../accumulo/test/functional/BadIterator.java   |    5 +
 .../accumulo/test/functional/SlowIterator.java  |   24 +-
 .../test/performance/thrift/NullTserver.java    |   23 +
 .../accumulo/test/ConditionalWriterTest.java    | 1236 +++++
 44 files changed, 10840 insertions(+), 439 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
new file mode 100644
index 0000000..4ed4d31
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
@@ -0,0 +1,138 @@
+/*
+ * 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 java.util.Iterator;
+
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.data.ConditionalMutation;
+
+/**
+ * @since 1.6.0
+ */
+public interface ConditionalWriter {
+  public static class Result {
+    
+    private Status status;
+    private ConditionalMutation mutation;
+    private String server;
+    private Exception exception;
+    
+    public Result(Status s, ConditionalMutation m, String server) {
+      this.status = s;
+      this.mutation = m;
+      this.server = server;
+    }
+    
+    public Result(Exception e, ConditionalMutation cm, String server) {
+      this.exception = e;
+      this.mutation = cm;
+      this.server = server;
+    }
+
+    /**
+     * If this method throws an exception, then its possible the mutation is still being actively processed. Therefore if code chooses to continue after seeing
+     * an exception it should take this into consideration.
+     * 
+     * @return status of a conditional mutation
+     * @throws AccumuloException
+     * @throws AccumuloSecurityException
+     */
+
+    public Status getStatus() throws AccumuloException, AccumuloSecurityException {
+      if (status == null) {
+        if (exception instanceof AccumuloException)
+          throw new AccumuloException(exception);
+        if (exception instanceof AccumuloSecurityException) {
+          AccumuloSecurityException ase = (AccumuloSecurityException) exception;
+          throw new AccumuloSecurityException(ase.getUser(), SecurityErrorCode.valueOf(ase.getSecurityErrorCode().name()), ase.getTableInfo(), ase);
+        }
+        else
+          throw new AccumuloException(exception);
+      }
+
+      return status;
+    }
+    
+    /**
+     * 
+     * @return A copy of the mutation previously submitted by a user. The mutation will reference the same data, but the object may be different.
+     */
+    public ConditionalMutation getMutation() {
+      return mutation;
+    }
+    
+    /**
+     * 
+     * @return The server this mutation was sent to. Returns null if was not sent to a server.
+     */
+    public String getTabletServer() {
+      return server;
+    }
+  }
+  
+  public static enum Status {
+    /**
+     * conditions were met and mutation was written
+     */
+    ACCEPTED,
+    /**
+     * conditions were not met and mutation was not written
+     */
+    REJECTED,
+    /**
+     * mutation violated a constraint and was not written
+     */
+    VIOLATED,
+    /**
+     * error occurred after mutation was sent to server, its unknown if the mutation was written. Although the status of the mutation is unknown, Accumulo
+     * guarantees the mutation will not be written at a later point in time.
+     */
+    UNKNOWN,
+    /**
+     * A condition contained a column visibility that could never be seen
+     */
+    INVISIBLE_VISIBILITY,
+
+  }
+
+  /**
+   * This method returns one result for each mutation passed to it. This method is thread safe. Multiple threads can safely use a single conditional writer.
+   * Sharing a conditional writer between multiple threads may result in batching of request to tablet servers.
+   * 
+   * @param mutations
+   * @return Result for each mutation submitted. The mutations may still be processing in the background when this method returns, if so the iterator will
+   *         block.
+   */
+  public abstract Iterator<Result> write(Iterator<ConditionalMutation> mutations);
+  
+  /**
+   * This method has the same thread safety guarantees as @link {@link #write(Iterator)}
+   * 
+   * 
+   * @param mutation
+   * @return Result for the submitted mutation
+   */
+
+  public abstract Result write(ConditionalMutation mutation);
+
+  /**
+   * release any resources (like threads pools) used by conditional writer
+   */
+  public void close();
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/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
new file mode 100644
index 0000000..f2a91ea
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
@@ -0,0 +1,118 @@
+/*
+ * 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 java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ArgumentChecker;
+
+/**
+ * 
+ * @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;
+  
+  /**
+   * 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.
+   * 
+   * @param auths
+   */
+  public ConditionalWriterConfig setAuthorizations(Authorizations auths) {
+    ArgumentChecker.notNull(auths);
+    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
+   *          determines how {@code timeout} will be interpreted
+   * @throws IllegalArgumentException
+   *           if {@code timeout} is less than 0
+   * @return {@code this} to allow chaining of set methods
+   */
+  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
+      // make small, positive values that truncate to 0 when converted use the minimum millis instead
+      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
+   *           if {@code maxWriteThreads} is non-positive
+   * @return {@code this} to allow chaining of set methods
+   */
+  public ConditionalWriterConfig setMaxWriteThreads(int maxWriteThreads) {
+    if (maxWriteThreads <= 0)
+      throw new IllegalArgumentException("Max threads must be positive " + maxWriteThreads);
+    
+    this.maxWriteThreads = maxWriteThreads;
+    return this;
+  }
+  
+  public Authorizations getAuthorizations() {
+    return auths;
+  }
+
+  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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/Connector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index d2e7321..bbfa55f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -174,6 +174,21 @@ public abstract class Connector {
   public abstract Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException;
   
   /**
+   * Factory method to create a ConditionalWriter connected to Accumulo.
+   * 
+   * @param tableName
+   *          the name of the table to query data from
+   * @param config
+   *          configuration used to create conditional writer
+   * 
+   * @return ConditionalWriter object for writing ConditionalMutations
+   * @throws TableNotFoundException
+   *           when the specified table doesn't exist
+   * @since 1.6.0
+   */
+  public abstract ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException;
+
+  /**
    * Accessor method for internal instance object.
    * 
    * @return the internal instance object

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java b/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
new file mode 100644
index 0000000..549322e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
@@ -0,0 +1,131 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
+
+public class CompressedIterators {
+  private Map<String,Integer> symbolMap;
+  private List<String> symbolTable;
+  private Map<ByteSequence,IterConfig> cache;
+  
+  public static class IterConfig {
+    public List<IterInfo> ssiList = new ArrayList<IterInfo>();
+    public Map<String,Map<String,String>> ssio = new HashMap<String,Map<String,String>>();
+  }
+
+  public CompressedIterators() {
+    symbolMap = new HashMap<String,Integer>();
+    symbolTable = new ArrayList<String>();
+  }
+  
+  public CompressedIterators(List<String> symbols) {
+    this.symbolTable = symbols;
+    this.cache = new HashMap<ByteSequence,IterConfig>();
+  }
+
+  private int getSymbolID(String symbol) {
+    Integer id = symbolMap.get(symbol);
+    if (id == null) {
+      id = symbolTable.size();
+      symbolTable.add(symbol);
+      symbolMap.put(symbol, id);
+    }
+    
+    return id;
+  }
+  
+  public ByteBuffer compress(IteratorSetting[] iterators) {
+    
+    UnsynchronizedBuffer.Writer out = new UnsynchronizedBuffer.Writer(iterators.length * 8);
+    
+    out.writeVInt(iterators.length);
+
+    for (IteratorSetting is : iterators) {
+      out.writeVInt(getSymbolID(is.getName()));
+      out.writeVInt(getSymbolID(is.getIteratorClass()));
+      out.writeVInt(is.getPriority());
+      
+      Map<String,String> opts = is.getOptions();
+      out.writeVInt(opts.size());
+      
+      for (Entry<String,String> entry : opts.entrySet()) {
+        out.writeVInt(getSymbolID(entry.getKey()));
+        out.writeVInt(getSymbolID(entry.getValue()));
+      }
+    }
+    
+    return out.toByteBuffer();
+    
+  }
+  
+  public IterConfig decompress(ByteBuffer iterators) {
+    
+    ByteSequence iterKey = new ArrayByteSequence(iterators);
+    IterConfig config = cache.get(iterKey);
+    if (config != null) {
+      return config;
+    }
+
+    config = new IterConfig();
+
+    UnsynchronizedBuffer.Reader in = new UnsynchronizedBuffer.Reader(iterators);
+
+    int num = in.readVInt();
+    
+    for (int i = 0; i < num; i++) {
+      String name = symbolTable.get(in.readVInt());
+      String iterClass = symbolTable.get(in.readVInt());
+      int prio = in.readVInt();
+      
+      config.ssiList.add(new IterInfo(prio, iterClass, name));
+      
+      int numOpts = in.readVInt();
+      
+      HashMap<String,String> opts = new HashMap<String,String>();
+      
+      for (int j = 0; j < numOpts; j++) {
+        String key = symbolTable.get(in.readVInt());
+        String val = symbolTable.get(in.readVInt());
+        
+        opts.put(key, val);
+      }
+      
+      config.ssio.put(name, opts);
+      
+    }
+
+    cache.put(iterKey, config);
+    return config;
+  }
+
+  public List<String> getSymbolTable() {
+    return symbolTable;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/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
new file mode 100644
index 0000000..9d7e257
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -0,0 +1,794 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+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.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.TCMResult;
+import org.apache.accumulo.core.data.thrift.TCMStatus;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
+import org.apache.accumulo.core.data.thrift.TKeyExtent;
+import org.apache.accumulo.core.data.thrift.TMutation;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.VisibilityEvaluator;
+import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.util.BadArgumentException;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.LoggingRunnable;
+import org.apache.accumulo.core.util.ThriftUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooLock;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.accumulo.trace.thrift.TInfo;
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.commons.lang.mutable.MutableLong;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.transport.TTransportException;
+
+
+class ConditionalWriterImpl implements ConditionalWriter {
+  
+  private static ThreadPoolExecutor cleanupThreadPool = new ThreadPoolExecutor(1, 1, 10, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
+  
+  static {
+    cleanupThreadPool.allowCoreThreadTimeOut(true);
+  }
+
+  private static final Logger log = Logger.getLogger(ConditionalWriterImpl.class);
+
+  private static final int MAX_SLEEP = 30000;
+
+  private Authorizations auths;
+  private VisibilityEvaluator ve;
+  @SuppressWarnings("unchecked")
+  private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap(1000));
+  private Instance instance;
+  private TCredentials credentials;
+  private TabletLocator locator;
+  private String tableId;
+  private long timeout;
+
+  private static class ServerQueue {
+    BlockingQueue<TabletServerMutations<QCMutation>> queue = new LinkedBlockingQueue<TabletServerMutations<QCMutation>>();
+    boolean taskQueued = false;
+  }
+  
+  private Map<String,ServerQueue> serverQueues;
+  private DelayQueue<QCMutation> failedMutations = new DelayQueue<QCMutation>();
+  private ScheduledThreadPoolExecutor threadPool;
+  
+  private class RQIterator implements Iterator<Result> {
+    
+    private BlockingQueue<Result> rq;
+    private int count;
+    
+    public RQIterator(BlockingQueue<Result> resultQueue, int count) {
+      this.rq = resultQueue;
+      this.count = count;
+    }
+    
+    @Override
+    public boolean hasNext() {
+      return count > 0;
+    }
+    
+    @Override
+    public Result next() {
+      if (count <= 0)
+        throw new NoSuchElementException();
+
+      try {
+        Result result = rq.poll(1, TimeUnit.SECONDS);
+        while (result == null) {
+          
+          if (threadPool.isShutdown()) {
+            throw new NoSuchElementException("ConditionalWriter closed");
+          }
+          
+          result = rq.poll(1, TimeUnit.SECONDS);
+        }
+        count--;
+        return result;
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+
+  private static class QCMutation extends ConditionalMutation implements Delayed {
+    private BlockingQueue<Result> resultQueue;
+    private long resetTime;
+    private long delay = 50;
+    private long entryTime;
+    
+    QCMutation(ConditionalMutation cm, BlockingQueue<Result> resultQueue, long entryTime) {
+      super(cm);
+      this.resultQueue = resultQueue;
+      this.entryTime = entryTime;
+    }
+
+    @Override
+    public int compareTo(Delayed o) {
+      QCMutation oqcm = (QCMutation) o;
+      return Long.valueOf(resetTime).compareTo(Long.valueOf(oqcm.resetTime));
+    }
+    
+    @Override
+    public long getDelay(TimeUnit unit) {
+      return unit.convert(delay - (System.currentTimeMillis() - resetTime), TimeUnit.MILLISECONDS);
+    }
+    
+    void resetDelay() {
+      delay = Math.min(delay * 2, MAX_SLEEP);
+      resetTime = System.currentTimeMillis();
+    }
+    
+    void queueResult(Result result) {
+      resultQueue.add(result);
+    }
+  }
+  
+  private ServerQueue getServerQueue(String location) {
+    ServerQueue serverQueue;
+    synchronized (serverQueues) {
+       serverQueue = serverQueues.get(location);
+      if (serverQueue == null) {
+        
+        serverQueue = new ServerQueue();
+        serverQueues.put(location, serverQueue);
+      }
+    }
+    return serverQueue;
+  }
+  
+  private class CleanupTask implements Runnable {
+    private List<SessionID> sessions;
+    
+    CleanupTask(List<SessionID> activeSessions) {
+      this.sessions = activeSessions;
+    }
+    
+    @Override
+    public void run() {
+      TabletClientService.Iface client = null;
+      
+      for (SessionID sid : sessions) {
+        if (!sid.isActive())
+          continue;
+        
+        TInfo tinfo = Tracer.traceInfo();
+        try {
+          client = getClient(sid.location);
+          client.closeConditionalUpdate(tinfo, sid.sessionID);
+        } catch (Exception e) {
+        } finally {
+          ThriftUtil.returnClient((TServiceClient) client);
+        }
+
+      }
+    }
+  }
+
+  private void queueRetry(List<QCMutation> mutations, String server) {
+    
+    if (timeout < Long.MAX_VALUE) {
+      
+      long time = System.currentTimeMillis();
+      
+      ArrayList<QCMutation> mutations2 = new ArrayList<ConditionalWriterImpl.QCMutation>(mutations.size());
+
+      for (QCMutation qcm : mutations) {
+        qcm.resetDelay();
+        if (time + qcm.getDelay(TimeUnit.MILLISECONDS) > qcm.entryTime + timeout) {
+          TimedOutException toe;
+          if (server != null)
+            toe = new TimedOutException(Collections.singleton(server));
+          else
+            toe = new TimedOutException("Conditional mutation timed out");
+          
+          qcm.queueResult(new Result(toe, qcm, server));
+        } else {
+          mutations2.add(qcm);
+        }
+      }
+      
+      if (mutations2.size() > 0)
+        failedMutations.addAll(mutations2);
+
+    } else {
+      for (QCMutation qcm : mutations)
+        qcm.resetDelay();
+      failedMutations.addAll(mutations);
+    }
+  }
+
+  private void queue(List<QCMutation> mutations) {
+    List<QCMutation> failures = new ArrayList<QCMutation>();
+    Map<String,TabletServerMutations<QCMutation>> binnedMutations = new HashMap<String,TabletLocator.TabletServerMutations<QCMutation>>();
+    
+    try {
+      locator.binMutations(mutations, binnedMutations, failures, credentials);
+      
+      if (failures.size() == mutations.size())
+        if (!Tables.exists(instance, tableId))
+          throw new TableDeletedException(tableId);
+        else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+          throw new TableOfflineException(instance, tableId);
+
+    } catch (Exception e) {
+      for (QCMutation qcm : mutations)
+        qcm.queueResult(new Result(e, qcm, null));
+      
+      // do not want to queue anything that was put in before binMutations() failed
+      failures.clear();
+      binnedMutations.clear();
+    }
+    
+    if (failures.size() > 0)
+      queueRetry(failures, null);
+
+    for (Entry<String,TabletServerMutations<QCMutation>> entry : binnedMutations.entrySet()) {
+      queue(entry.getKey(), entry.getValue());
+    }
+
+
+  }
+
+  private void queue(String location, TabletServerMutations<QCMutation> mutations) {
+    
+    ServerQueue serverQueue = getServerQueue(location);
+    
+    synchronized (serverQueue) {
+      serverQueue.queue.add(mutations);
+      // never execute more than one task per server
+      if(!serverQueue.taskQueued){
+        threadPool.execute(new LoggingRunnable(log, new SendTask(location)));
+        serverQueue.taskQueued = true;
+      }
+    }
+   
+  }
+
+  private void reschedule(SendTask task){
+    ServerQueue serverQueue = getServerQueue(task.location);
+    // just finished processing work for this server, could reschedule if it has more work or immediately process the work
+    // this code reschedules the the server for processing later... there may be other queues with
+    // more data that need to be processed... also it will give the current server time to build
+    // up more data... the thinking is that rescheduling instead or processing immediately will result
+    // in bigger batches and less RPC overhead
+    
+    synchronized (serverQueue) {
+      if(serverQueue.queue.size() > 0)
+        threadPool.execute(new LoggingRunnable(log, task));
+      else
+        serverQueue.taskQueued = false;
+    }
+    
+  }
+  
+  private TabletServerMutations<QCMutation> dequeue(String location) {
+    BlockingQueue<TabletServerMutations<QCMutation>> queue = getServerQueue(location).queue;
+    
+    ArrayList<TabletServerMutations<QCMutation>> mutations = new ArrayList<TabletLocator.TabletServerMutations<QCMutation>>();
+    queue.drainTo(mutations);
+    
+    if (mutations.size() == 0)
+      return null;
+    
+    if (mutations.size() == 1) {
+      return mutations.get(0);
+    } else {
+      // merge multiple request to a single tablet server
+      TabletServerMutations<QCMutation> tsm = mutations.get(0);
+      
+      for (int i = 1; i < mutations.size(); i++) {
+        for (Entry<KeyExtent,List<QCMutation>> entry : mutations.get(i).getMutations().entrySet()) {
+          List<QCMutation> list = tsm.getMutations().get(entry.getKey());
+          if (list == null) {
+            list = new ArrayList<QCMutation>();
+            tsm.getMutations().put(entry.getKey(), list);
+          }
+          
+          list.addAll(entry.getValue());
+        }
+      }
+      
+      return tsm;
+    }
+  }
+
+  ConditionalWriterImpl(Instance instance, TCredentials credentials, String tableId, ConditionalWriterConfig config) {
+    this.instance = instance;
+    this.credentials = credentials;
+    this.auths = config.getAuthorizations();
+    this.ve = new VisibilityEvaluator(config.getAuthorizations());
+    this.threadPool = new ScheduledThreadPoolExecutor(config.getMaxWriteThreads());
+    this.threadPool.setMaximumPoolSize(config.getMaxWriteThreads());
+    this.locator = TabletLocator.getLocator(instance, new Text(tableId));
+    this.serverQueues = new HashMap<String,ServerQueue>();
+    this.tableId = tableId;
+    this.timeout = config.getTimeout(TimeUnit.MILLISECONDS);
+
+    Runnable failureHandler = new Runnable() {
+      
+      @Override
+      public void run() {
+          List<QCMutation> mutations = new ArrayList<QCMutation>();
+          failedMutations.drainTo(mutations);
+          queue(mutations);
+      }
+    };
+    
+    failureHandler = new LoggingRunnable(log, failureHandler);
+    
+    threadPool.scheduleAtFixedRate(failureHandler, 250, 250, TimeUnit.MILLISECONDS);
+  }
+
+  public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+
+    BlockingQueue<Result> resultQueue = new LinkedBlockingQueue<Result>();
+
+    List<QCMutation> mutationList = new ArrayList<QCMutation>();
+
+    int count = 0;
+
+    long entryTime = System.currentTimeMillis();
+
+    mloop: while (mutations.hasNext()) {
+      ConditionalMutation mut = mutations.next();
+      count++;
+
+      for (Condition cond : mut.getConditions()) {
+        if (!isVisible(cond.getVisibility())) {
+          resultQueue.add(new Result(Status.INVISIBLE_VISIBILITY, mut, null));
+          continue mloop;
+        }
+      }
+
+      // copy the mutations so that even if caller changes it, it will not matter
+      mutationList.add(new QCMutation(mut, resultQueue, entryTime));
+    }
+
+    queue(mutationList);
+
+    return new RQIterator(resultQueue, count);
+
+  }
+
+  private class SendTask implements Runnable {
+    
+
+    String location;
+    
+    public SendTask(String location) {
+      this.location = location;
+
+    }
+    
+    @Override
+    public void run() {
+      try {
+        TabletServerMutations<QCMutation> mutations = dequeue(location);
+        if (mutations != null)
+          sendToServer(location, mutations);
+      } finally {
+        reschedule(this);
+      }
+    }
+  }
+  
+  private static class CMK {
+
+    QCMutation cm;
+    KeyExtent ke;
+    
+    public CMK(KeyExtent ke, QCMutation cm) {
+      this.ke = ke;
+      this.cm = cm;
+    }
+  }
+
+  private static class SessionID {
+    String location;
+    String lockId;
+    long sessionID;
+    boolean reserved;
+    long lastAccessTime;
+    long ttl;
+    
+    boolean isActive() {
+      return System.currentTimeMillis() - lastAccessTime < ttl * .95;
+    }
+  }
+  
+  private HashMap<String, SessionID> cachedSessionIDs = new HashMap<String, SessionID>();
+  
+  private SessionID reserveSessionID(String location, TabletClientService.Iface client, TInfo tinfo) throws ThriftSecurityException, TException {
+    //avoid cost of repeatedly making RPC to create sessions, reuse sessions
+    synchronized (cachedSessionIDs) {
+      SessionID sid = cachedSessionIDs.get(location);
+      if (sid != null) {
+        if (sid.reserved)
+          throw new IllegalStateException();
+        
+        if (!sid.isActive()) {
+          cachedSessionIDs.remove(location);
+        } else {
+          sid.reserved = true;
+          return sid;
+        }
+      }
+    }
+    
+    TConditionalSession tcs = client.startConditionalUpdate(tinfo, credentials, ByteBufferUtil.toByteBuffers(auths.getAuthorizations()), tableId);
+    
+    synchronized (cachedSessionIDs) {
+      SessionID sid = new SessionID();
+      sid.reserved = true;
+      sid.sessionID = tcs.sessionId;
+      sid.lockId = tcs.tserverLock;
+      sid.ttl = tcs.ttl;
+      sid.location = location;
+      if (cachedSessionIDs.put(location, sid) != null)
+        throw new IllegalStateException();
+
+      return sid;
+    }
+    
+  }
+  
+  private void invalidateSessionID(String location) {
+    synchronized (cachedSessionIDs) {
+      cachedSessionIDs.remove(location);
+    }
+    
+  }
+  
+  private void unreserveSessionID(String location){
+    synchronized (cachedSessionIDs) {
+      SessionID sid = cachedSessionIDs.get(location);
+      if (sid != null) {
+        if (!sid.reserved)
+          throw new IllegalStateException();
+        sid.reserved = false;
+        sid.lastAccessTime = System.currentTimeMillis();
+      }
+    }
+  }
+  
+  List<SessionID> getActiveSessions() {
+    ArrayList<SessionID> activeSessions = new ArrayList<SessionID>();
+    for (SessionID sid : cachedSessionIDs.values())
+      if (sid.isActive())
+        activeSessions.add(sid);
+    return activeSessions;
+  }
+
+  private TabletClientService.Iface getClient(String location) throws TTransportException {
+    TabletClientService.Iface client;
+    if (timeout < instance.getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+      client = ThriftUtil.getTServerClient(location, timeout);
+    else
+      client = ThriftUtil.getTServerClient(location, instance.getConfiguration());
+    return client;
+  }
+
+  private void sendToServer(String location, TabletServerMutations<QCMutation> mutations) {
+    TabletClientService.Iface client = null;
+    
+    TInfo tinfo = Tracer.traceInfo();
+
+    Map<Long,CMK> cmidToCm = new HashMap<Long,CMK>();
+    MutableLong cmid = new MutableLong(0);
+
+    SessionID sessionId = null;
+    
+    try {
+      
+      client = getClient(location);
+
+      Map<TKeyExtent,List<TConditionalMutation>> tmutations = new HashMap<TKeyExtent,List<TConditionalMutation>>();
+
+      CompressedIterators compressedIters = new CompressedIterators();
+      convertMutations(mutations, cmidToCm, cmid, tmutations, compressedIters);
+      
+      List<TCMResult> tresults = null;
+      while (tresults == null) {
+        try {
+          sessionId = reserveSessionID(location, client, tinfo);
+          tresults = client.conditionalUpdate(tinfo, sessionId.sessionID, tmutations, compressedIters.getSymbolTable());
+        } catch (NoSuchScanIDException nssie) {
+          sessionId = null;
+          invalidateSessionID(location);
+        }
+      }
+      
+      HashSet<KeyExtent> extentsToInvalidate = new HashSet<KeyExtent>();
+
+      ArrayList<QCMutation> ignored = new ArrayList<QCMutation>();
+
+      for (TCMResult tcmResult : tresults) {
+        if (tcmResult.status == TCMStatus.IGNORED) {
+          CMK cmk = cmidToCm.get(tcmResult.cmid);
+          ignored.add(cmk.cm);
+          extentsToInvalidate.add(cmk.ke);
+        } else {
+          QCMutation qcm = cmidToCm.get(tcmResult.cmid).cm;
+          qcm.queueResult(new Result(fromThrift(tcmResult.status), qcm, location));
+        }
+      }
+
+      for (KeyExtent ke : extentsToInvalidate) {
+        locator.invalidateCache(ke);
+      }
+
+      queueRetry(ignored, location);
+
+    } catch (ThriftSecurityException tse) {
+      AccumuloSecurityException ase = new AccumuloSecurityException(credentials.getPrincipal(), tse.getCode(), Tables.getPrintableTableInfoFromId(instance,
+          tableId), tse);
+      queueException(location, cmidToCm, ase);
+    } catch (TTransportException e) {
+      locator.invalidateCache(location);
+      invalidateSession(location, mutations, cmidToCm, sessionId);
+    } catch (TApplicationException tae) {
+      queueException(location, cmidToCm, new AccumuloServerException(location, tae));
+    } catch (TException e) {
+      locator.invalidateCache(location);
+      invalidateSession(location, mutations, cmidToCm, sessionId);
+    } catch (Exception e) {
+      queueException(location, cmidToCm, e);
+    } finally {
+      unreserveSessionID(location);
+      ThriftUtil.returnClient((TServiceClient) client);
+    }
+  }
+
+
+  private void queueRetry(Map<Long,CMK> cmidToCm, String location) {
+    ArrayList<QCMutation> ignored = new ArrayList<QCMutation>();
+    for (CMK cmk : cmidToCm.values())
+    	ignored.add(cmk.cm);
+    queueRetry(ignored, location);
+  }
+
+  private void queueException(String location, Map<Long,CMK> cmidToCm, Exception e) {
+    for (CMK cmk : cmidToCm.values())
+      cmk.cm.queueResult(new Result(e, cmk.cm, location));
+  }
+
+  private void invalidateSession(String location, TabletServerMutations<QCMutation> mutations, Map<Long,CMK> cmidToCm, SessionID sessionId) {
+    if(sessionId == null){
+      queueRetry(cmidToCm, location);
+    }else{
+      try {
+        invalidateSession(sessionId, location, mutations);
+        for (CMK cmk : cmidToCm.values())
+          cmk.cm.queueResult(new Result(Status.UNKNOWN, cmk.cm, location));
+      }catch(Exception e2){
+        queueException(location, cmidToCm, e2);
+      }
+    }
+  }
+  
+  /*
+   * The purpose of this code is to ensure that a conditional mutation will not execute when its status is unknown. This allows a user to read the row when the
+   * status is unknown and not have to worry about the tserver applying the mutation after the scan.
+   * 
+   * If a conditional mutation is taking a long time to process, then this method will wait for it to finish... unless this exceeds timeout.
+   */
+  private void invalidateSession(SessionID sessionId, String location, TabletServerMutations<QCMutation> mutations) throws AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
+    
+    ArrayList<QCMutation> mutList = new ArrayList<QCMutation>();
+    
+    for (List<QCMutation> tml : mutations.getMutations().values()) {
+      mutList.addAll(tml);
+    }
+    
+    long sleepTime = 50;
+
+    long startTime = System.currentTimeMillis();
+
+    LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
+
+    while (true) {
+      if (!ZooLock.isLockHeld(ServerClient.getZooCache(instance), lid)) {
+        // TODO if ACCUMULO-1152 adds a tserver lock check to the tablet location cache, then this invalidation would prevent future attempts to contact the
+        // tserver even its gone zombie and is still running w/o a lock
+        locator.invalidateCache(location);
+        return;
+      }
+      
+      try {
+        // if the mutation is currently processing, this method will block until its done or times out
+        invalidateSession(sessionId.sessionID, location);
+
+        return;
+      } catch (TApplicationException tae) {
+        throw new AccumuloServerException(location, tae);
+      } catch (TException e) {
+        locator.invalidateCache(location);
+      }
+      
+      if ((System.currentTimeMillis() - startTime) + sleepTime > timeout)
+        throw new TimedOutException(Collections.singleton(location));
+
+      UtilWaitThread.sleep(sleepTime);
+      sleepTime = Math.min(2 * sleepTime, MAX_SLEEP);
+
+    }
+	
+  }
+  
+  private void invalidateSession(long sessionId, String location) throws TException {
+    TabletClientService.Iface client = null;
+    
+    TInfo tinfo = Tracer.traceInfo();
+    
+    try {
+      client = getClient(location);
+      client.invalidateConditionalUpdate(tinfo, sessionId);
+    } finally {
+      ThriftUtil.returnClient((TServiceClient) client);
+    }
+  }
+
+  private Status fromThrift(TCMStatus status) {
+    switch (status) {
+      case ACCEPTED:
+        return Status.ACCEPTED;
+      case REJECTED:
+        return Status.REJECTED;
+      case VIOLATED:
+        return Status.VIOLATED;
+      default:
+        throw new IllegalArgumentException(status.toString());
+    }
+  }
+
+  private void convertMutations(TabletServerMutations<QCMutation> mutations, Map<Long,CMK> cmidToCm, MutableLong cmid,
+      Map<TKeyExtent,List<TConditionalMutation>> tmutations, CompressedIterators compressedIters) {
+
+    for (Entry<KeyExtent,List<QCMutation>> entry : mutations.getMutations().entrySet()) {
+      TKeyExtent tke = entry.getKey().toThrift();
+      ArrayList<TConditionalMutation> tcondMutaions = new ArrayList<TConditionalMutation>();
+      
+      List<QCMutation> condMutations = entry.getValue();
+      
+      for (QCMutation cm : condMutations) {
+        TMutation tm = cm.toThrift();
+
+        List<TCondition> conditions = convertConditions(cm, compressedIters);
+
+        cmidToCm.put(cmid.longValue(), new CMK(entry.getKey(), cm));
+        TConditionalMutation tcm = new TConditionalMutation(conditions, tm, cmid.longValue());
+        cmid.increment();
+        tcondMutaions.add(tcm);
+      }
+      
+      tmutations.put(tke, tcondMutaions);
+    }
+  }
+
+  private List<TCondition> convertConditions(ConditionalMutation cm, CompressedIterators compressedIters) {
+    List<TCondition> conditions = new ArrayList<TCondition>(cm.getConditions().size());
+    
+    for (Condition cond : cm.getConditions()) {
+      long ts = 0;
+      boolean hasTs = false;
+      
+      if (cond.getTimestamp() != null) {
+        ts = cond.getTimestamp();
+        hasTs = true;
+      }
+      
+      ByteBuffer iters = compressedIters.compress(cond.getIterators());
+      
+      TCondition tc = new TCondition(ByteBufferUtil.toByteBuffers(cond.getFamily()), ByteBufferUtil.toByteBuffers(cond.getQualifier()),
+          ByteBufferUtil.toByteBuffers(cond.getVisibility()), ts, hasTs, ByteBufferUtil.toByteBuffers(cond.getValue()), iters);
+      
+      conditions.add(tc);
+    }
+    
+    return conditions;
+  }
+
+  private boolean isVisible(ByteSequence cv) {
+    Text testVis = new Text(cv.toArray());
+    if (testVis.getLength() == 0)
+      return true;
+    
+    Boolean b = cache.get(testVis);
+    if (b != null)
+      return b;
+    
+    try {
+      Boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+      cache.put(new Text(testVis), bb);
+      return bb;
+    } catch (VisibilityParseException e) {
+      return false;
+    } catch (BadArgumentException e) {
+      return false;
+    }
+  }
+
+  public Result write(ConditionalMutation mutation) {
+    return write(Collections.singleton(mutation).iterator()).next();
+  }
+  
+  @Override
+  public void close() {
+    threadPool.shutdownNow();
+    cleanupThreadPool.execute(new CleanupTask(getActiveSessions()));
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 3858cdc..57e36fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -24,6 +24,8 @@ import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchScanner;
 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.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -130,6 +132,11 @@ public class ConnectorImpl extends Connector {
   }
   
   @Override
+  public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException {
+    return new ConditionalWriterImpl(instance, credentials, getTableId(tableName), config);
+  }
+  
+  @Override
   public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException {
     ArgumentChecker.notNull(tableName, authorizations);
     return new ScannerImpl(instance, credentials, getTableId(tableName), authorizations);
@@ -162,5 +169,4 @@ public class ConnectorImpl extends Connector {
     
     return instanceops;
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 18b2a27..88e5c3a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -42,12 +42,12 @@ public class RootTabletLocator extends TabletLocator {
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+      TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     String rootTabletLocation = instance.getRootTabletLocation();
     if (rootTabletLocation != null) {
-      TabletServerMutations tsm = new TabletServerMutations();
-      for (Mutation mutation : mutations) {
+      TabletServerMutations<T> tsm = new TabletServerMutations<T>();
+      for (T mutation : mutations) {
         tsm.addMutation(RootTable.EXTENT, mutation);
       }
       binnedMutations.put(rootTabletLocation, tsm);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index b933c2b..218bd36 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.ArgumentChecker;
@@ -45,7 +45,7 @@ public class ServerClient {
   private static final Logger log = Logger.getLogger(ServerClient.class);
   private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
   
-  private synchronized static ZooCache getZooCache(Instance instance) {
+  synchronized static ZooCache getZooCache(Instance instance) {
     ZooCache result = zooCaches.get(instance.getZooKeepers());
     if (result == null) {
       result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), null);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
index de8e053..f9110b2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
@@ -43,7 +43,7 @@ public abstract class TabletLocator {
   public abstract TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException;
   
-  public abstract void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures,
+  public abstract <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
       TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
   
   public abstract List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials)
@@ -187,24 +187,24 @@ public abstract class TabletLocator {
     }
   }
   
-  public static class TabletServerMutations {
-    private Map<KeyExtent,List<Mutation>> mutations;
+  public static class TabletServerMutations<T extends Mutation> {
+    private Map<KeyExtent,List<T>> mutations;
     
     public TabletServerMutations() {
-      mutations = new HashMap<KeyExtent,List<Mutation>>();
+      mutations = new HashMap<KeyExtent,List<T>>();
     }
     
-    public void addMutation(KeyExtent ke, Mutation m) {
-      List<Mutation> mutList = mutations.get(ke);
+    public void addMutation(KeyExtent ke, T m) {
+      List<T> mutList = mutations.get(ke);
       if (mutList == null) {
-        mutList = new ArrayList<Mutation>();
+        mutList = new ArrayList<T>();
         mutations.put(ke, mutList);
       }
       
       mutList.add(m);
     }
     
-    public Map<KeyExtent,List<Mutation>> getMutations() {
+    public Map<KeyExtent,List<T>> getMutations() {
       return mutations;
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
index df5d66b..4b2e1d3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
@@ -113,14 +113,14 @@ public class TabletLocatorImpl extends TabletLocator {
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+      TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     
     OpTimer opTimer = null;
     if (log.isTraceEnabled())
       opTimer = new OpTimer(log, Level.TRACE).start("Binning " + mutations.size() + " mutations for table " + tableId);
     
-    ArrayList<Mutation> notInCache = new ArrayList<Mutation>();
+    ArrayList<T> notInCache = new ArrayList<T>();
     Text row = new Text();
     
     rLock.lock();
@@ -133,7 +133,7 @@ public class TabletLocatorImpl extends TabletLocator {
       // For this to be efficient, need to avoid fine grained synchronization and fine grained logging.
       // Therefore methods called by this are not synchronized and should not log.
       
-      for (Mutation mutation : mutations) {
+      for (T mutation : mutations) {
         row.set(mutation.getRow());
         TabletLocation tl = locateTabletInCache(row);
         if (tl == null)
@@ -157,7 +157,7 @@ public class TabletLocatorImpl extends TabletLocator {
       wLock.lock();
       try {
         boolean failed = false;
-        for (Mutation mutation : notInCache) {
+        for (T mutation : notInCache) {
           if (failed) {
             // when one table does not return a location, something is probably
             // screwy, go ahead and fail everything.
@@ -185,11 +185,11 @@ public class TabletLocatorImpl extends TabletLocator {
       opTimer.stop("Binned " + mutations.size() + " mutations for table " + tableId + " to " + binnedMutations.size() + " tservers in %DURATION%");
   }
   
-  private void addMutation(Map<String,TabletServerMutations> binnedMutations, Mutation mutation, TabletLocation tl) {
-    TabletServerMutations tsm = binnedMutations.get(tl.tablet_location);
+  private <T extends Mutation> void addMutation(Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl) {
+    TabletServerMutations<T> tsm = binnedMutations.get(tl.tablet_location);
     
     if (tsm == null) {
-      tsm = new TabletServerMutations();
+      tsm = new TabletServerMutations<T>();
       binnedMutations.put(tl.tablet_location, tsm);
     }
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/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 766cea9..12f5243 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
@@ -415,13 +415,13 @@ public class TabletServerBatchWriter {
     totalSendTime.addAndGet(time);
   }
   
-  public void updateBinningStats(int count, long time, Map<String,TabletServerMutations> binnedMutations) {
+  public void updateBinningStats(int count, long time, Map<String,TabletServerMutations<Mutation>> binnedMutations) {
     totalBinTime.addAndGet(time);
     totalBinned.addAndGet(count);
     updateBatchStats(binnedMutations);
   }
   
-  private synchronized void updateBatchStats(Map<String,TabletServerMutations> binnedMutations) {
+  private synchronized void updateBatchStats(Map<String,TabletServerMutations<Mutation>> binnedMutations) {
     tabletServersBatchSum += binnedMutations.size();
     
     minTabletServersBatch = Math.min(minTabletServersBatch, binnedMutations.size());
@@ -429,8 +429,8 @@ public class TabletServerBatchWriter {
     
     int numTablets = 0;
     
-    for (Entry<String,TabletServerMutations> entry : binnedMutations.entrySet()) {
-      TabletServerMutations tsm = entry.getValue();
+    for (Entry<String,TabletServerMutations<Mutation>> entry : binnedMutations.entrySet()) {
+      TabletServerMutations<Mutation> tsm = entry.getValue();
       numTablets += tsm.getMutations().size();
     }
     
@@ -577,7 +577,7 @@ public class TabletServerBatchWriter {
       init().addAll(failures);
     }
     
-    synchronized void add(String location, TabletServerMutations tsm) {
+    synchronized void add(String location, TabletServerMutations<Mutation> tsm) {
       init();
       for (Entry<KeyExtent,List<Mutation>> entry : tsm.getMutations().entrySet()) {
         recentFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
@@ -617,12 +617,12 @@ public class TabletServerBatchWriter {
     
     private static final int MUTATION_BATCH_SIZE = 1 << 17;
     private ExecutorService sendThreadPool;
-    private Map<String,TabletServerMutations> serversMutations;
+    private Map<String,TabletServerMutations<Mutation>> serversMutations;
     private Set<String> queued;
     private Map<String,TabletLocator> locators;
     
     public MutationWriter(int numSendThreads) {
-      serversMutations = new HashMap<String,TabletServerMutations>();
+      serversMutations = new HashMap<String,TabletServerMutations<Mutation>>();
       queued = new HashSet<String>();
       sendThreadPool = new SimpleThreadPool(numSendThreads, this.getClass().getName());
       locators = new HashMap<String,TabletLocator>();
@@ -639,7 +639,7 @@ public class TabletServerBatchWriter {
       return ret;
     }
     
-    private void binMutations(MutationSet mutationsToProcess, Map<String,TabletServerMutations> binnedMutations) {
+    private void binMutations(MutationSet mutationsToProcess, Map<String,TabletServerMutations<Mutation>> binnedMutations) {
       try {
         Set<Entry<String,List<Mutation>>> es = mutationsToProcess.getMutations().entrySet();
         for (Entry<String,List<Mutation>> entry : es) {
@@ -687,7 +687,7 @@ public class TabletServerBatchWriter {
     }
     
     void addMutations(MutationSet mutationsToSend) {
-      Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
+      Map<String,TabletServerMutations<Mutation>> binnedMutations = new HashMap<String,TabletServerMutations<Mutation>>();
       Span span = Trace.start("binMutations");
       try {
         long t1 = System.currentTimeMillis();
@@ -700,15 +700,15 @@ public class TabletServerBatchWriter {
       addMutations(binnedMutations);
     }
     
-    private synchronized void addMutations(Map<String,TabletServerMutations> binnedMutations) {
+    private synchronized void addMutations(Map<String,TabletServerMutations<Mutation>> binnedMutations) {
       
       int count = 0;
       
       // merge mutations into existing mutations for a tablet server
-      for (Entry<String,TabletServerMutations> entry : binnedMutations.entrySet()) {
+      for (Entry<String,TabletServerMutations<Mutation>> entry : binnedMutations.entrySet()) {
         String server = entry.getKey();
         
-        TabletServerMutations currentMutations = serversMutations.get(server);
+        TabletServerMutations<Mutation> currentMutations = serversMutations.get(server);
         
         if (currentMutations == null) {
           serversMutations.put(server, entry.getValue());
@@ -740,8 +740,8 @@ public class TabletServerBatchWriter {
         }
     }
     
-    private synchronized TabletServerMutations getMutationsToSend(String server) {
-      TabletServerMutations tsmuts = serversMutations.remove(server);
+    private synchronized TabletServerMutations<Mutation> getMutationsToSend(String server) {
+      TabletServerMutations<Mutation> tsmuts = serversMutations.remove(server);
       if (tsmuts == null)
         queued.remove(server);
       
@@ -759,7 +759,7 @@ public class TabletServerBatchWriter {
       @Override
       public void run() {
         try {
-          TabletServerMutations tsmuts = getMutationsToSend(location);
+          TabletServerMutations<Mutation> tsmuts = getMutationsToSend(location);
           
           while (tsmuts != null) {
             send(tsmuts);
@@ -772,7 +772,7 @@ public class TabletServerBatchWriter {
         }
       }
       
-      public void send(TabletServerMutations tsm) throws AccumuloServerException, AccumuloSecurityException {
+      public void send(TabletServerMutations<Mutation> tsm) throws AccumuloServerException, AccumuloSecurityException {
         
         MutationSet failures = null;
         
@@ -846,6 +846,8 @@ public class TabletServerBatchWriter {
         return new MutationSet();
       }
       TInfo tinfo = Tracer.traceInfo();
+      
+      // TODO remove this
       TTransport transport = null;
       
       timeoutTracker.startingWrite();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
index 62518ec..e8cd678 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
@@ -76,8 +76,8 @@ public class TimeoutTabletLocator extends TabletLocator {
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+      TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     try {
       locator.binMutations(mutations, binnedMutations, failures, credentials);
       

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 1179559..4af2ea5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -22,6 +22,8 @@ import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchScanner;
 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.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -128,4 +130,10 @@ public class MockConnector extends Connector {
     return new MockInstanceOperations(acu);
   }
   
+  @Override
+  public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException {
+    // TODO add implementation
+    throw new UnsupportedOperationException();
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
index aca3ba4..8d91d5d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
@@ -40,10 +40,10 @@ public class MockTabletLocator extends TabletLocator {
   }
   
   @Override
-  public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
-      AccumuloSecurityException, TableNotFoundException {
-    TabletServerMutations tsm = new TabletServerMutations();
-    for (Mutation m : mutations)
+  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+      TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    TabletServerMutations<T> tsm = new TabletServerMutations<T>();
+    for (T m : mutations)
       tsm.addMutation(new KeyExtent(), m);
     binnedMutations.put("", tsm);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index d44a7a6..eaa61b9 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.data;
 
 import java.io.Serializable;
+import java.nio.ByteBuffer;
 
 public class ArrayByteSequence extends ByteSequence implements Serializable {
   
@@ -48,6 +49,18 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
     this(s.getBytes());
   }
   
+  public ArrayByteSequence(ByteBuffer buffer) {
+    if (buffer.hasArray()) {
+      this.data = buffer.array();
+      this.offset = buffer.arrayOffset();
+      this.length = buffer.limit();
+    } else {
+      this.data = new byte[buffer.remaining()];
+      this.offset = 0;
+      buffer.get(data);
+    }
+  }
+
   @Override
   public byte byteAt(int i) {
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/Condition.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Condition.java b/core/src/main/java/org/apache/accumulo/core/data/Condition.java
new file mode 100644
index 0000000..97df7e0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/Condition.java
@@ -0,0 +1,148 @@
+/*
+ * 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.data;
+
+import java.util.HashSet;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.hadoop.io.Text;
+
+/**
+ * 
+ * @since 1.6.0
+ */
+public class Condition {
+  
+  private ByteSequence cf;
+  private ByteSequence cq;
+  private ByteSequence cv;
+  private ByteSequence val;
+  private Long ts;
+  private IteratorSetting iterators[] = new IteratorSetting[0];
+  private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[0]);
+  
+
+  public Condition(CharSequence cf, CharSequence cq) {
+    ArgumentChecker.notNull(cf, cq);
+    this.cf = new ArrayByteSequence(cf.toString().getBytes(Constants.UTF8));
+    this.cq = new ArrayByteSequence(cq.toString().getBytes(Constants.UTF8));
+    this.cv = EMPTY;
+  }
+  
+  public Condition(byte[] cf, byte[] cq) {
+    ArgumentChecker.notNull(cf, cq);
+    this.cf = new ArrayByteSequence(cf);
+    this.cq = new ArrayByteSequence(cq);
+    this.cv = EMPTY;
+  }
+
+  public Condition(Text cf, Text cq) {
+    ArgumentChecker.notNull(cf, cq);
+    this.cf = new ArrayByteSequence(cf.getBytes(), 0, cf.getLength());
+    this.cq = new ArrayByteSequence(cq.getBytes(), 0, cq.getLength());
+    this.cv = EMPTY;
+  }
+
+  public Condition(ByteSequence cf, ByteSequence cq) {
+    ArgumentChecker.notNull(cf, cq);
+    this.cf = cf;
+    this.cq = cq;
+    this.cv = EMPTY;
+  }
+
+  public ByteSequence getFamily() {
+    return cf;
+  }
+  
+  public ByteSequence getQualifier() {
+    return cq;
+  }
+
+  public Condition setTimestamp(long ts) {
+    this.ts = ts;
+    return this;
+  }
+  
+  public Long getTimestamp() {
+    return ts;
+  }
+
+  public Condition setValue(CharSequence value) {
+    ArgumentChecker.notNull(value);
+    this.val = new ArrayByteSequence(value.toString().getBytes(Constants.UTF8));
+    return this;
+  }
+
+  public Condition setValue(byte[] value) {
+    ArgumentChecker.notNull(value);
+    this.val = new ArrayByteSequence(value);
+    return this;
+  }
+  
+  public Condition setValue(Text value) {
+    ArgumentChecker.notNull(value);
+    this.val = new ArrayByteSequence(value.getBytes(), 0, value.getLength());
+    return this;
+  }
+  
+  public Condition setValue(ByteSequence value) {
+    ArgumentChecker.notNull(value);
+    this.val = value;
+    return this;
+  }
+
+  public ByteSequence getValue() {
+    return val;
+  }
+
+  public Condition setVisibility(ColumnVisibility cv) {
+    ArgumentChecker.notNull(cv);
+    this.cv = new ArrayByteSequence(cv.getExpression());
+    return this;
+  }
+
+  public ByteSequence getVisibility() {
+    return cv;
+  }
+
+  public Condition setIterators(IteratorSetting... iterators) {
+    ArgumentChecker.notNull(iterators);
+    
+    if (iterators.length > 1) {
+      HashSet<String> names = new HashSet<String>();
+      HashSet<Integer> prios = new HashSet<Integer>();
+      
+      for (IteratorSetting iteratorSetting : iterators) {
+        if (!names.add(iteratorSetting.getName()))
+          throw new IllegalArgumentException("iterator name used more than once " + iteratorSetting.getName());
+        if (!prios.add(iteratorSetting.getPriority()))
+          throw new IllegalArgumentException("iterator priority used more than once " + iteratorSetting.getPriority());
+      }
+    }
+    
+    this.iterators = iterators;
+    return this;
+  }
+
+  public IteratorSetting[] getIterators() {
+    return iterators;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
new file mode 100644
index 0000000..23bf7d0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.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.core.data;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.hadoop.io.Text;
+
+/**
+ * @since 1.6.0
+ */
+public class ConditionalMutation extends Mutation {
+  
+  private List<Condition> conditions = new ArrayList<Condition>();
+
+  public ConditionalMutation(byte[] row, Condition condition, Condition... conditions) {
+    super(row);
+    init(condition, conditions);
+  }
+  
+  public ConditionalMutation(byte[] row, int start, int length, Condition condition, Condition... conditions) {
+    super(row, start, length);
+    init(condition, conditions);
+  }
+  
+  public ConditionalMutation(Text row, Condition condition, Condition... conditions) {
+    super(row);
+    init(condition, conditions);
+  }
+  
+  public ConditionalMutation(CharSequence row, Condition condition, Condition... conditions) {
+    super(row);
+    init(condition, conditions);
+  }
+  
+  public ConditionalMutation(ByteSequence row, Condition condition, Condition... conditions) {
+    // TODO add ByteSequence methods to mutations
+    super(row.toArray());
+    init(condition, conditions);
+  }
+  
+  public ConditionalMutation(ConditionalMutation cm) {
+    super(cm);
+    this.conditions = new ArrayList<Condition>(cm.conditions);
+  }
+
+  private void init(Condition condition, Condition... conditions) {
+    ArgumentChecker.notNull(condition);
+    this.conditions.add(condition);
+    if (conditions.length > 0) {
+      this.conditions.addAll(Arrays.asList(conditions));
+    }
+  }
+  
+  public void addCondition(Condition condition) {
+    ArgumentChecker.notNull(condition);
+    this.conditions.add(condition);
+  }
+  
+  public List<Condition> getConditions() {
+    return Collections.unmodifiableList(conditions);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 00cefbf..4ac3f0c 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
@@ -65,121 +66,8 @@ public class Mutation implements Writable {
   private byte[] data;
   private int entries;
   private List<byte[]> values;
-
-  // created this little class instead of using ByteArrayOutput stream and DataOutputStream
-  // because both are synchronized... lots of small syncs slow things down
-  private static class ByteBuffer {
-    
-    int offset;
-    byte data[] = new byte[64];
-    
-    private void reserve(int l) {
-      if (offset + l > data.length) {
-        int newSize = data.length * 2;
-        while (newSize <= offset + l)
-          newSize = newSize * 2;
-        
-        byte[] newData = new byte[newSize];
-        System.arraycopy(data, 0, newData, 0, offset);
-        data = newData;
-      }
-      
-    }
-    
-    public void add(byte[] bytes, int off, int length) {
-      reserve(length);
-      System.arraycopy(bytes, off, data, offset, length);
-      offset += length;
-    }
-    
-    void add(boolean b) {
-      reserve(1);
-      if (b)
-        data[offset++] = 1;
-      else
-        data[offset++] = 0;
-    }
-    
-    public byte[] toArray() {
-      byte ret[] = new byte[offset];
-      System.arraycopy(data, 0, ret, 0, offset);
-      return ret;
-    }
-    
-    public void writeVLong(long i) {
-      reserve(9);
-      if (i >= -112 && i <= 127) {
-        data[offset++] = (byte)i;
-        return;
-      }
-        
-      int len = -112;
-      if (i < 0) {
-        i ^= -1L; // take one's complement'
-        len = -120;
-      }
-        
-      long tmp = i;
-      while (tmp != 0) {
-        tmp = tmp >> 8;
-        len--;
-      }
-        
-      data[offset++] = (byte)len;
-        
-      len = (len < -120) ? -(len + 120) : -(len + 112);
-        
-      for (int idx = len; idx != 0; idx--) {
-        int shiftbits = (idx - 1) * 8;
-        long mask = 0xFFL << shiftbits;
-        data[offset++] = (byte)((i & mask) >> shiftbits);
-      }
-    }
-  }
-  
-  private static class SimpleReader {
-    int offset;
-    byte data[];
-    
-    SimpleReader(byte b[]) {
-      this.data = b;
-    }
-
-    int readInt() {
-      return (data[offset++] << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0);
-    }
-    
-    long readLong() {
-      return (((long) data[offset++] << 56) + ((long) (data[offset++] & 255) << 48) + ((long) (data[offset++] & 255) << 40)
-          + ((long) (data[offset++] & 255) << 32) + ((long) (data[offset++] & 255) << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0));
-    }
-    
-    void readBytes(byte b[]) {
-      System.arraycopy(data, offset, b, 0, b.length);
-      offset += b.length;
-    }
-    
-    boolean readBoolean() {
-      return (data[offset++] == 1);
-    }
-    
-    long readVLong() {
-      byte firstByte = data[offset++];
-      int len =  WritableUtils.decodeVIntSize(firstByte);
-      if (len == 1) {
-        return firstByte;
-      }
-      long i = 0;
-      for (int idx = 0; idx < len-1; idx++) {
-        byte b = data[offset++];
-        i = i << 8;
-        i = i | (b & 0xFF);
-      }
-      return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
-    }
-  }
   
-  private ByteBuffer buffer;
+  private UnsynchronizedBuffer.Writer buffer;
   
   private List<ColumnUpdate> updates;
   
@@ -205,7 +93,7 @@ public class Mutation implements Writable {
   public Mutation(byte[] row, int start, int length) {
     this.row = new byte[length];
     System.arraycopy(row, start, this.row, 0, length);
-    buffer = new ByteBuffer();
+    buffer = new UnsynchronizedBuffer.Writer();
   }
   
   public Mutation(Text row) {
@@ -445,7 +333,7 @@ public class Mutation implements Writable {
     put(columnFamily, columnQualifier, columnVisibility.getExpression(), true, timestamp, true, EMPTY_BYTES);
   }
 
-  private byte[] oldReadBytes(SimpleReader in) {
+  private byte[] oldReadBytes(UnsynchronizedBuffer.Reader in) {
     int len = in.readInt();
     if (len == 0)
       return EMPTY_BYTES;
@@ -455,7 +343,7 @@ public class Mutation implements Writable {
     return bytes;
   }
   
-  private byte[] readBytes(SimpleReader in) {
+  private byte[] readBytes(UnsynchronizedBuffer.Reader in) {
     int len = (int)in.readVLong();
     if (len == 0)
       return EMPTY_BYTES;
@@ -468,7 +356,7 @@ public class Mutation implements Writable {
   public List<ColumnUpdate> getUpdates() {
     serialize();
     
-    SimpleReader in = new SimpleReader(data);
+    UnsynchronizedBuffer.Reader in = new UnsynchronizedBuffer.Reader(data);
     
     if (updates == null) {
       if (entries == 1) {
@@ -490,7 +378,7 @@ public class Mutation implements Writable {
     return new ColumnUpdate(cf, cq, cv, hasts, ts, deleted, val);
   }
 
-  private ColumnUpdate deserializeColumnUpdate(SimpleReader in) {
+  private ColumnUpdate deserializeColumnUpdate(UnsynchronizedBuffer.Reader in) {
     byte[] cf = readBytes(in);
     byte[] cq = readBytes(in);
     byte[] cv = readBytes(in);
@@ -623,8 +511,8 @@ public class Mutation implements Writable {
     }
     
     // convert data to new format
-    SimpleReader din = new SimpleReader(localData);
-    buffer = new ByteBuffer();
+    UnsynchronizedBuffer.Reader din = new UnsynchronizedBuffer.Reader(localData);
+    buffer = new UnsynchronizedBuffer.Writer();
     for (int i = 0; i < localEntries; i++) {
       byte[] cf = oldReadBytes(din);
       byte[] cq = oldReadBytes(din);


[2/6] ACCUMULO-1000 added conditional mutations to Accumulo

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
index ce84054..6f6a9ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.hadoop.io.Text;
 
 public class ByteBufferUtil {
@@ -76,4 +77,16 @@ public class ByteBufferUtil {
   public static String toString(ByteBuffer bytes) {
     return new String(bytes.array(), bytes.position(), bytes.remaining());
   }
+  
+  public static ByteBuffer toByteBuffers(ByteSequence bs) {
+    if (bs == null)
+      return null;
+
+    if (bs.isBackedByArray()) {
+      return ByteBuffer.wrap(bs.getBackingArray(), bs.offset(), bs.length());
+    } else {
+      // TODO create more efficient impl
+      return ByteBuffer.wrap(bs.toArray());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
new file mode 100644
index 0000000..b640581
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
@@ -0,0 +1,195 @@
+/*
+ * 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.util;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * 
+ */
+public class UnsynchronizedBuffer {
+  // created this little class instead of using ByteArrayOutput stream and DataOutputStream
+  // because both are synchronized... lots of small syncs slow things down
+  public static class Writer {
+    
+    int offset = 0;
+    byte data[];
+    
+    public Writer() {
+      data = new byte[64];
+    }
+    
+    public Writer(int initialCapacity) {
+      data = new byte[initialCapacity];
+    }
+    
+    private void reserve(int l) {
+      if (offset + l > data.length) {
+        int newSize = UnsynchronizedBuffer.nextArraySize(offset + l);
+
+        byte[] newData = new byte[newSize];
+        System.arraycopy(data, 0, newData, 0, offset);
+        data = newData;
+      }
+      
+    }
+    
+    public void add(byte[] bytes, int off, int length) {
+      reserve(length);
+      System.arraycopy(bytes, off, data, offset, length);
+      offset += length;
+    }
+    
+    public void add(boolean b) {
+      reserve(1);
+      if (b)
+        data[offset++] = 1;
+      else
+        data[offset++] = 0;
+    }
+    
+    public byte[] toArray() {
+      byte ret[] = new byte[offset];
+      System.arraycopy(data, 0, ret, 0, offset);
+      return ret;
+    }
+    
+    public ByteBuffer toByteBuffer() {
+      return ByteBuffer.wrap(data, 0, offset);
+    }
+
+    public void writeVInt(int i) {
+      writeVLong(i);
+    }
+
+    public void writeVLong(long i) {
+      reserve(9);
+      if (i >= -112 && i <= 127) {
+        data[offset++] = (byte) i;
+        return;
+      }
+      
+      int len = -112;
+      if (i < 0) {
+        i ^= -1L; // take one's complement'
+        len = -120;
+      }
+      
+      long tmp = i;
+      while (tmp != 0) {
+        tmp = tmp >> 8;
+        len--;
+      }
+      
+      data[offset++] = (byte) len;
+      
+      len = (len < -120) ? -(len + 120) : -(len + 112);
+      
+      for (int idx = len; idx != 0; idx--) {
+        int shiftbits = (idx - 1) * 8;
+        long mask = 0xFFL << shiftbits;
+        data[offset++] = (byte) ((i & mask) >> shiftbits);
+      }
+    }
+  }
+  
+  public static class Reader {
+    int offset;
+    byte data[];
+    
+    public Reader(byte b[]) {
+      this.data = b;
+    }
+    
+    public Reader(ByteBuffer buffer) {
+      if (buffer.hasArray()) {
+        offset = buffer.arrayOffset();
+        data = buffer.array();
+      } else {
+        data = new byte[buffer.remaining()];
+        buffer.get(data);
+      }
+    }
+
+    public int readInt() {
+      return (data[offset++] << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0);
+    }
+    
+    public long readLong() {
+      return (((long) data[offset++] << 56) + ((long) (data[offset++] & 255) << 48) + ((long) (data[offset++] & 255) << 40)
+          + ((long) (data[offset++] & 255) << 32) + ((long) (data[offset++] & 255) << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0));
+    }
+    
+    public void readBytes(byte b[]) {
+      System.arraycopy(data, offset, b, 0, b.length);
+      offset += b.length;
+    }
+    
+    public boolean readBoolean() {
+      return (data[offset++] == 1);
+    }
+    
+    public int readVInt() {
+      return (int) readVLong();
+    }
+
+    public long readVLong() {
+      byte firstByte = data[offset++];
+      int len = WritableUtils.decodeVIntSize(firstByte);
+      if (len == 1) {
+        return firstByte;
+      }
+      long i = 0;
+      for (int idx = 0; idx < len - 1; idx++) {
+        byte b = data[offset++];
+        i = i << 8;
+        i = i | (b & 0xFF);
+      }
+      return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+    }
+  }
+
+  /**
+   * Determines what next array size should be by rounding up to next power of two.
+   * 
+   */
+  public static int nextArraySize(int i) {
+    if (i < 0)
+      throw new IllegalArgumentException();
+    
+    if (i > (1 << 30))
+      return Integer.MAX_VALUE; // this is the next power of 2 minus one... a special case
+  
+    if (i == 0) {
+      return 1;
+    }
+    
+    // round up to next power of two
+    int ret = i;
+    ret--;
+    ret |= ret >> 1;
+    ret |= ret >> 2;
+    ret |= ret >> 4;
+    ret |= ret >> 8;
+    ret |= ret >> 16;
+    ret++;
+    
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/thrift/data.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/data.thrift b/core/src/main/thrift/data.thrift
index 4e2d3e6..ae6f439 100644
--- a/core/src/main/thrift/data.thrift
+++ b/core/src/main/thrift/data.thrift
@@ -110,10 +110,46 @@ struct UpdateErrors {
 	3:map<TKeyExtent, client.SecurityErrorCode> authorizationFailures
 }
 
+enum TCMStatus {
+	ACCEPTED,
+	REJECTED,
+	VIOLATED,
+	IGNORED
+}
+
+struct TCMResult {
+	1:i64 cmid,
+	2:TCMStatus status
+}
+
 struct MapFileInfo {
 	1:i64 estimatedSize
 }
 
+struct TCondition {
+	1:binary cf;
+	2:binary cq;
+	3:binary cv;
+	4:i64 ts;
+	5:bool hasTimestamp;
+	6:binary val;
+	7:binary iterators;
+}
+
+struct TConditionalMutation {
+	1:list<TCondition> conditions;
+	2:TMutation mutation;
+	3:i64 id;
+}
+
+struct TConditionalSession {
+	1:i64 sessionId;
+	2:string tserverLock;
+        3:i64 ttl;
+}
+
+typedef map<TKeyExtent,list<TConditionalMutation>> CMBatch
+
 typedef map<TKeyExtent,list<TMutation>> UpdateBatch
 
 typedef map<TKeyExtent, map<string, MapFileInfo>> TabletFiles

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index e6adbf4..4f9f13a 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -160,13 +160,21 @@ service TabletClientService extends client.ClientService {
   data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.TCredentials credentials) 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)
     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)
+     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)
+     throws (1:NoSuchScanIDException nssi);
+  void invalidateConditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID);
+  oneway void closeConditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID);
+
   // on success, returns an empty list
   list<data.TKeyExtent> bulkImport(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:client.ThriftSecurityException sec),
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 0a34575..fe2f09c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -219,7 +219,7 @@ public class TabletLocatorImplTest extends TestCase {
   }
   
   private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
-    Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
+    Map<String,TabletServerMutations<Mutation>> binnedMutations = new HashMap<String,TabletServerMutations<Mutation>>();
     List<Mutation> afailures = new ArrayList<Mutation>();
     metaCache.binMutations(ml, binnedMutations, afailures, credential);
     
@@ -239,11 +239,11 @@ public class TabletLocatorImplTest extends TestCase {
     
   }
   
-  private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations> actual) {
+  private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations<Mutation>> actual) {
     assertEquals(expected.keySet(), actual.keySet());
     
     for (String server : actual.keySet()) {
-      TabletServerMutations atb = actual.get(server);
+      TabletServerMutations<Mutation> atb = actual.get(server);
       Map<KeyExtent,List<String>> etb = expected.get(server);
       
       assertEquals(etb.keySet(), atb.getMutations().keySet());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
index bf577c1..1608576 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.rfile.RelativeKey.MByteSequence;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -39,25 +40,25 @@ public class RelativeKeyTest {
   
   @Test
   public void testBasicRelativeKey() {
-    assertEquals(1, RelativeKey.nextArraySize(0));
-    assertEquals(1, RelativeKey.nextArraySize(1));
-    assertEquals(2, RelativeKey.nextArraySize(2));
-    assertEquals(4, RelativeKey.nextArraySize(3));
-    assertEquals(4, RelativeKey.nextArraySize(4));
-    assertEquals(8, RelativeKey.nextArraySize(5));
-    assertEquals(8, RelativeKey.nextArraySize(8));
-    assertEquals(16, RelativeKey.nextArraySize(9));
+    assertEquals(1, UnsynchronizedBuffer.nextArraySize(0));
+    assertEquals(1, UnsynchronizedBuffer.nextArraySize(1));
+    assertEquals(2, UnsynchronizedBuffer.nextArraySize(2));
+    assertEquals(4, UnsynchronizedBuffer.nextArraySize(3));
+    assertEquals(4, UnsynchronizedBuffer.nextArraySize(4));
+    assertEquals(8, UnsynchronizedBuffer.nextArraySize(5));
+    assertEquals(8, UnsynchronizedBuffer.nextArraySize(8));
+    assertEquals(16, UnsynchronizedBuffer.nextArraySize(9));
     
-    assertEquals(1 << 16, RelativeKey.nextArraySize((1 << 16) - 1));
-    assertEquals(1 << 16, RelativeKey.nextArraySize(1 << 16));
-    assertEquals(1 << 17, RelativeKey.nextArraySize((1 << 16) + 1));
+    assertEquals(1 << 16, UnsynchronizedBuffer.nextArraySize((1 << 16) - 1));
+    assertEquals(1 << 16, UnsynchronizedBuffer.nextArraySize(1 << 16));
+    assertEquals(1 << 17, UnsynchronizedBuffer.nextArraySize((1 << 16) + 1));
     
-    assertEquals(1 << 30, RelativeKey.nextArraySize((1 << 30) - 1));
+    assertEquals(1 << 30, UnsynchronizedBuffer.nextArraySize((1 << 30) - 1));
 
-    assertEquals(1 << 30, RelativeKey.nextArraySize(1 << 30));
+    assertEquals(1 << 30, UnsynchronizedBuffer.nextArraySize(1 << 30));
 
-    assertEquals(Integer.MAX_VALUE, RelativeKey.nextArraySize(Integer.MAX_VALUE - 1));
-    assertEquals(Integer.MAX_VALUE, RelativeKey.nextArraySize(Integer.MAX_VALUE));
+    assertEquals(Integer.MAX_VALUE, UnsynchronizedBuffer.nextArraySize(Integer.MAX_VALUE - 1));
+    assertEquals(Integer.MAX_VALUE, UnsynchronizedBuffer.nextArraySize(Integer.MAX_VALUE));
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java b/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
new file mode 100644
index 0000000..7487ba3
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
@@ -0,0 +1,58 @@
+/*
+ * 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.server.data;
+
+import java.util.List;
+
+import org.apache.accumulo.core.client.impl.Translator;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+
+/**
+ * 
+ */
+public class ServerConditionalMutation extends ServerMutation {
+  
+  public static class TCMTranslator extends Translator<TConditionalMutation,ServerConditionalMutation> {
+    @Override
+    public ServerConditionalMutation translate(TConditionalMutation input) {
+      return new ServerConditionalMutation(input);
+    }
+  }
+  
+  public static final TCMTranslator TCMT = new TCMTranslator();
+
+  private long cmid;
+  private List<TCondition> conditions;
+  
+  public ServerConditionalMutation(TConditionalMutation input) {
+    super(input.mutation);
+
+    this.cmid = input.id;
+    this.conditions = input.conditions;
+  }
+
+  public long getID() {
+    return cmid;
+  }
+  
+  public List<TCondition> getConditions() {
+    return conditions;
+  }
+  
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 2b98331..cebc338 100644
--- a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -328,6 +328,13 @@ public class SecurityOperation {
     return hasTablePermission(credentials, table, TablePermission.WRITE, true);
   }
   
+  public boolean canConditionallyUpdate(TCredentials credentials, String tableID, List<ByteBuffer> authorizations) throws ThriftSecurityException {
+    
+    authenticate(credentials);
+    
+    return hasTablePermission(credentials, tableID, TablePermission.WRITE, true) && hasTablePermission(credentials, tableID, TablePermission.READ, true);
+  }
+
   public boolean canSplitTablet(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.ALTER_TABLE, false) || hasSystemPermission(credentials, SystemPermission.SYSTEM, false)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
new file mode 100644
index 0000000..c25e729
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
@@ -0,0 +1,91 @@
+/*
+ * 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.server.tabletserver;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * 
+ */
+public class ConditionalMutationSet {
+
+  static interface DeferFilter {
+    void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred);
+  }
+  
+  static class DuplicateFitler implements DeferFilter {
+    public void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred) {
+      okMutations.add(scml.get(0));
+      for (int i = 1; i < scml.size(); i++) {
+        if (Arrays.equals(scml.get(i - 1).getRow(), scml.get(i).getRow())) {
+          deferred.add(scml.get(i));
+        } else {
+          okMutations.add(scml.get(i));
+        }
+      }
+    }
+  }
+  
+  static void defer(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferredMutations, DeferFilter filter) {
+    for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : updates.entrySet()) {
+      List<ServerConditionalMutation> scml = entry.getValue();
+      List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(scml.size());
+      List<ServerConditionalMutation> deferred = new ArrayList<ServerConditionalMutation>();
+      filter.defer(scml, okMutations, deferred);
+      
+      if (deferred.size() > 0) {
+        scml.clear();
+        scml.addAll(okMutations);
+        List<ServerConditionalMutation> l = deferredMutations.get(entry.getKey());
+        if (l == null) {
+          l = deferred;
+          deferredMutations.put(entry.getKey(), l);
+        } else {
+          l.addAll(deferred);
+        }
+
+      }
+    }
+  }
+  
+  static void deferDuplicatesRows(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferred) {
+    defer(updates, deferred, new DuplicateFitler());
+  }
+
+  static void sortConditionalMutations(Map<KeyExtent,List<ServerConditionalMutation>> updates) {
+    for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : updates.entrySet()) {
+      // TODO check if its already in sorted order?
+      // TODO maybe the potential benefit of sorting is not worth the cost
+      Collections.sort(entry.getValue(), new Comparator<ServerConditionalMutation>() {
+        @Override
+        public int compare(ServerConditionalMutation o1, ServerConditionalMutation o2) {
+          return WritableComparator.compareBytes(o1.getRow(), 0, o1.getRow().length, o2.getRow(), 0, o2.getRow().length);
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
new file mode 100644
index 0000000..f057ca3
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
@@ -0,0 +1,162 @@
+/*
+ * 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.server.tabletserver;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
+import org.apache.accumulo.server.tabletserver.ConditionalMutationSet.DeferFilter;
+
+/**
+ * 
+ */
+class RowLocks {
+  
+  private Map<ByteSequence,RowLock> rowLocks = new HashMap<ByteSequence,RowLock>();
+  
+  static class RowLock {
+    ReentrantLock rlock;
+    int count;
+    ByteSequence rowSeq;
+    
+    RowLock(ReentrantLock rlock, ByteSequence rowSeq) {
+      this.rlock = rlock;
+      this.count = 0;
+      this.rowSeq = rowSeq;
+    }
+    
+    public boolean tryLock() {
+      return rlock.tryLock();
+    }
+    
+    public void lock() {
+      rlock.lock();
+    }
+    
+    public void unlock() {
+      rlock.unlock();
+    }
+  }
+  
+  private RowLock getRowLock(ArrayByteSequence rowSeq) {
+      RowLock lock = rowLocks.get(rowSeq);
+      if (lock == null) {
+        lock = new RowLock(new ReentrantLock(), rowSeq);
+        rowLocks.put(rowSeq, lock);
+      }
+      
+      lock.count++;
+      return lock;
+  }
+  
+  private void returnRowLock(RowLock lock) {
+      if (lock.count == 0)
+        throw new IllegalStateException();
+      lock.count--;
+      
+      if (lock.count == 0) {
+        rowLocks.remove(lock.rowSeq);
+      }
+  }
+  
+  List<RowLock> acquireRowlocks(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferred) {
+    ArrayList<RowLock> locks = new ArrayList<RowLock>();
+    
+    // assume that mutations are in sorted order to avoid deadlock
+    synchronized (rowLocks) {
+      for (List<ServerConditionalMutation> scml : updates.values()) {
+        for (ServerConditionalMutation scm : scml) {
+          locks.add(getRowLock(new ArrayByteSequence(scm.getRow())));
+        }
+      }
+    }
+    
+    HashSet<ByteSequence> rowsNotLocked = null;
+
+    // acquire as many locks as possible, not blocking on rows that are already locked
+    if (locks.size() > 1) {
+      for (RowLock rowLock : locks) {
+        if (!rowLock.tryLock()) {
+          if (rowsNotLocked == null)
+            rowsNotLocked = new HashSet<ByteSequence>();
+          rowsNotLocked.add(rowLock.rowSeq);
+        }
+      }
+    } else {
+      // if there is only one lock, then wait for it
+      locks.get(0).lock();
+    }
+    
+    if (rowsNotLocked != null) {
+      
+      final HashSet<ByteSequence> rnlf = rowsNotLocked;
+      // assume will get locks needed, do something expensive otherwise
+      ConditionalMutationSet.defer(updates, deferred, new DeferFilter() {
+        @Override
+        public void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred) {
+          for (ServerConditionalMutation scm : scml) {
+            if (rnlf.contains(new ArrayByteSequence(scm.getRow())))
+              deferred.add(scm);
+            else
+              okMutations.add(scm);
+            
+          }
+        }
+      });
+      
+      ArrayList<RowLock> filteredLocks = new ArrayList<RowLock>();
+      ArrayList<RowLock> locksToReturn = new ArrayList<RowLock>();
+      for (RowLock rowLock : locks) {
+        if (rowsNotLocked.contains(rowLock.rowSeq)) {
+          locksToReturn.add(rowLock);
+        } else {
+          filteredLocks.add(rowLock);
+        }
+      }
+      
+      synchronized (rowLocks) {
+        for (RowLock rowLock : locksToReturn) {
+          returnRowLock(rowLock);
+        }
+      }
+
+      locks = filteredLocks;
+    }
+    return locks;
+  }
+  
+  void releaseRowLocks(List<RowLock> locks) {
+    for (RowLock rowLock : locks) {
+      rowLock.unlock();
+    }
+    
+    synchronized (rowLocks) {
+      for (RowLock rowLock : locks) {
+        returnRowLock(rowLock);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
index e9b973a..01b0dc2 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
@@ -1651,7 +1651,7 @@ public class Tablet {
     }
   }
   
-  private Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, HashSet<Column> columns) throws IOException {
+  private Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, Set<Column> columns) throws IOException {
     
     // log.info("In nextBatch..");
     
@@ -1739,7 +1739,7 @@ public class Tablet {
     public long numBytes;
   }
   
-  Scanner createScanner(Range range, int num, HashSet<Column> columns, Authorizations authorizations, List<IterInfo> ssiList,
+  Scanner createScanner(Range range, int num, Set<Column> columns, Authorizations authorizations, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, boolean isolated, AtomicBoolean interruptFlag) {
     // do a test to see if this range falls within the tablet, if it does not
     // then clip will throw an exception
@@ -1873,14 +1873,14 @@ public class Tablet {
     // scan options
     Authorizations authorizations;
     byte[] defaultLabels;
-    HashSet<Column> columnSet;
+    Set<Column> columnSet;
     List<IterInfo> ssiList;
     Map<String,Map<String,String>> ssio;
     AtomicBoolean interruptFlag;
     int num;
     boolean isolated;
     
-    ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList,
+    ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, Set<Column> columnSet, List<IterInfo> ssiList,
         Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag, boolean isolated) {
       this.num = num;
       this.authorizations = authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index 7425fed..ccb95fc 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -65,6 +65,8 @@ 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.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.ScannerImpl;
 import org.apache.accumulo.core.client.impl.TabletType;
 import org.apache.accumulo.core.client.impl.Translator;
@@ -87,7 +89,12 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.MapFileInfo;
 import org.apache.accumulo.core.data.thrift.MultiScanResult;
 import org.apache.accumulo.core.data.thrift.ScanResult;
+import org.apache.accumulo.core.data.thrift.TCMResult;
+import org.apache.accumulo.core.data.thrift.TCMStatus;
 import org.apache.accumulo.core.data.thrift.TColumn;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
 import org.apache.accumulo.core.data.thrift.TKey;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TKeyValue;
@@ -140,6 +147,7 @@ import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -158,6 +166,7 @@ import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.tabletserver.Compactor.CompactionInfo;
+import org.apache.accumulo.server.tabletserver.RowLocks.RowLock;
 import org.apache.accumulo.server.tabletserver.Tablet.CommitSession;
 import org.apache.accumulo.server.tabletserver.Tablet.KVEntry;
 import org.apache.accumulo.server.tabletserver.Tablet.LookupResult;
@@ -336,12 +345,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     
     SecureRandom random;
     Map<Long,Session> sessions;
+    long maxIdle;
     
     SessionManager(AccumuloConfiguration conf) {
       random = new SecureRandom();
       sessions = new HashMap<Long,Session>();
       
-      final long maxIdle = conf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
+      maxIdle = conf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
       
       Runnable r = new Runnable() {
         @Override
@@ -369,6 +379,10 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       return sid;
     }
     
+    long getMaxIdleTime() {
+      return maxIdle;
+    }
+
     /**
      * while a session is reserved, it cannot be canceled or removed
      * 
@@ -387,9 +401,30 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       
     }
     
+    synchronized Session reserveSession(long sessionId, boolean wait) {
+      Session session = sessions.get(sessionId);
+      if (session != null) {
+        while(wait && session.reserved){
+          try {
+            wait(1000);
+          } catch (InterruptedException e) {
+            throw new RuntimeException();
+          }
+        }
+        
+        if (session.reserved)
+          throw new IllegalStateException();
+        session.reserved = true;
+      }
+      
+      return session;
+      
+    }
+    
     synchronized void unreserveSession(Session session) {
       if (!session.reserved)
         throw new IllegalStateException();
+      notifyAll();
       session.reserved = false;
       session.lastAccessTime = System.currentTimeMillis();
     }
@@ -399,7 +434,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (session != null)
         unreserveSession(session);
     }
-    
+        
     synchronized Session getSession(long sessionId) {
       Session session = sessions.get(sessionId);
       if (session != null)
@@ -408,9 +443,15 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
     
     Session removeSession(long sessionId) {
+      return removeSession(sessionId, false);
+    }
+    
+    Session removeSession(long sessionId, boolean unreserve) {
       Session session = null;
       synchronized (this) {
         session = sessions.remove(sessionId);
+        if(unreserve && session != null)
+          unreserveSession(session);
       }
       
       // do clean up out side of lock..
@@ -709,6 +750,18 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     
   }
   
+  private static class ConditionalSession extends Session {
+    public TCredentials credentials;
+    public Authorizations auths;
+    public String tableId;
+    public AtomicBoolean interruptFlag;
+    
+    @Override
+    public void cleanup() {
+      interruptFlag.set(true);
+    }
+  }
+  
   private static class UpdateSession extends Session {
     public Tablet currentTablet;
     public MapCounter<Tablet> successfulCommits = new MapCounter<Tablet>();
@@ -858,6 +911,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     
     WriteTracker writeTracker = new WriteTracker();
     
+    private RowLocks rowLocks = new RowLocks();
+
     ThriftClientHandler() {
       super(instance, watcher);
       log.debug(ThriftClientHandler.class.getName() + " created");
@@ -1687,6 +1742,250 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         writeTracker.finishWrite(opid);
       }
     }
+
+    private void checkConditions(Map<KeyExtent,List<ServerConditionalMutation>> updates, ArrayList<TCMResult> results, ConditionalSession cs,
+        List<String> symbols) throws IOException {
+      Iterator<Entry<KeyExtent,List<ServerConditionalMutation>>> iter = updates.entrySet().iterator();
+      
+      CompressedIterators compressedIters = new CompressedIterators(symbols);
+
+      while (iter.hasNext()) {
+        Entry<KeyExtent,List<ServerConditionalMutation>> entry = iter.next();
+        Tablet tablet = onlineTablets.get(entry.getKey());
+        
+        if (tablet == null || tablet.isClosed()) {
+          for (ServerConditionalMutation scm : entry.getValue())
+            results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+          iter.remove();
+        } else {
+          List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(entry.getValue().size());
+
+          for (ServerConditionalMutation scm : entry.getValue()) {
+            if (checkCondition(results, cs, compressedIters, tablet, scm))
+              okMutations.add(scm);
+          }
+          
+          entry.setValue(okMutations);
+        }
+        
+      }
+    }
+
+    boolean checkCondition(ArrayList<TCMResult> results, ConditionalSession cs, CompressedIterators compressedIters,
+        Tablet tablet, ServerConditionalMutation scm) throws IOException {
+      boolean add = true;
+      
+      Set<Column> emptyCols = Collections.emptySet();
+
+      for(TCondition tc : scm.getConditions()){
+      
+        Range range;
+        if (tc.hasTimestamp)
+          range = Range.exact(new Text(scm.getRow()), new Text(tc.getCf()), new Text(tc.getCq()), new Text(tc.getCv()), tc.getTs());
+        else
+          range = Range.exact(new Text(scm.getRow()), new Text(tc.getCf()), new Text(tc.getCq()), new Text(tc.getCv()));
+        
+        IterConfig ic = compressedIters.decompress(tc.iterators);
+
+        Scanner scanner = tablet.createScanner(range, 1, emptyCols, cs.auths, ic.ssiList, ic.ssio, false, cs.interruptFlag);
+        
+        try {
+          ScanBatch batch = scanner.read();
+          
+          Value val = null;
+          
+          for (KVEntry entry2 : batch.results) {
+            val = entry2.getValue();
+            break;
+          }
+          
+          if ((val == null ^ tc.getVal() == null) || (val != null && !Arrays.equals(tc.getVal(), val.get()))) {
+            results.add(new TCMResult(scm.getID(), TCMStatus.REJECTED));
+            add = false;
+            break;
+          }
+          
+        } catch (TabletClosedException e) {
+          results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+          add = false;
+          break;
+        } catch (IterationInterruptedException iie) {
+          results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+          add = false;
+          break;
+        } catch (TooManyFilesException tmfe) {
+          results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+          add = false;
+          break;
+        }
+      }
+      return add;
+    }
+
+    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>>();
+
+      boolean sessionCanceled = sess.interruptFlag.get();
+
+      for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : es) {
+        Tablet tablet = onlineTablets.get(entry.getKey());
+        if (tablet == null || tablet.isClosed() || sessionCanceled) {
+          for (ServerConditionalMutation scm : entry.getValue())
+            results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+        } else {
+          try {
+            
+            List<Mutation> mutations = (List<Mutation>) (List<? extends Mutation>) entry.getValue();
+            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);
+              }
+            }
+          } catch (TConstraintViolationException e) {
+            if (e.getNonViolators().size() > 0) {
+              sendables.put(e.getCommitSession(), e.getNonViolators());
+              for (Mutation m : e.getNonViolators())
+                results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.ACCEPTED));
+            }
+            
+            for (Mutation m : e.getViolators())
+              results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.VIOLATED));
+          }
+        }
+      }
+      
+      while (true && sendables.size() > 0) {
+        try {
+          logger.logManyTablets(sendables);
+          break;
+        } catch (IOException ex) {
+          log.warn("logging mutations failed, retrying");
+        } catch (FSError ex) { // happens when DFS is localFS
+          log.warn("logging mutations failed, retrying");
+        } catch (Throwable t) {
+          log.error("Unknown exception logging mutations, counts for mutations in flight not decremented!", t);
+          throw new RuntimeException(t);
+        }
+      }
+      
+      for (Entry<CommitSession,? extends List<Mutation>> entry : sendables.entrySet()) {
+        CommitSession commitSession = entry.getKey();
+        List<Mutation> mutations = entry.getValue();
+        
+        commitSession.commit(mutations);
+      }
+
+    }
+
+    private Map<KeyExtent,List<ServerConditionalMutation>> conditionalUpdate(ConditionalSession cs, Map<KeyExtent,List<ServerConditionalMutation>> updates,
+        ArrayList<TCMResult> results, List<String> symbols) throws IOException {
+      // sort each list of mutations, this is done to avoid deadlock and doing seeks in order is more efficient and detect duplicate rows.
+      ConditionalMutationSet.sortConditionalMutations(updates);
+      
+      Map<KeyExtent,List<ServerConditionalMutation>> deferred = new HashMap<KeyExtent,List<ServerConditionalMutation>>();
+
+      // can not process two mutations for the same row, because one will not see what the other writes
+      ConditionalMutationSet.deferDuplicatesRows(updates, deferred);
+
+      // get as many locks as possible w/o blocking... defer any rows that are locked
+      List<RowLock> locks = rowLocks.acquireRowlocks(updates, deferred);
+      try {
+        checkConditions(updates, results, cs, symbols);
+        writeConditionalMutations(updates, results, cs);
+      } finally {
+        rowLocks.releaseRowLocks(locks);
+      }
+      return deferred;
+    }
+    
+    @Override
+    public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID)
+        throws ThriftSecurityException, TException {
+      
+      Authorizations userauths = null;
+      if (!security.canConditionallyUpdate(credentials, tableID, authorizations))
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+      
+      userauths = security.getUserAuthorizations(credentials);
+      for (ByteBuffer auth : authorizations)
+        if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
+          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
+
+      ConditionalSession cs = new ConditionalSession();
+      cs.auths = new Authorizations(authorizations);
+      cs.credentials = credentials;
+      cs.tableId = tableID;
+      cs.interruptFlag = new AtomicBoolean();
+      
+      long sid = sessionManager.createSession(cs, false);
+      return new TConditionalSession(sid, lockID, sessionManager.getMaxIdleTime());
+    }
+
+    @Override
+    public List<TCMResult> conditionalUpdate(TInfo tinfo, long sessID, Map<TKeyExtent,List<TConditionalMutation>> mutations, List<String> symbols)
+        throws NoSuchScanIDException, TException {
+      
+      ConditionalSession cs = (ConditionalSession) sessionManager.reserveSession(sessID);
+      
+      if (cs == null || cs.interruptFlag.get())
+        throw new NoSuchScanIDException();
+      
+      Text tid = new Text(cs.tableId);
+      long opid = writeTracker.startWrite(TabletType.type(new KeyExtent(tid, null, null)));
+      
+      try{
+        Map<KeyExtent,List<ServerConditionalMutation>> updates = Translator.translate(mutations, Translator.TKET,
+            new Translator.ListTranslator<TConditionalMutation,ServerConditionalMutation>(ServerConditionalMutation.TCMT));
+
+        for(KeyExtent ke : updates.keySet())
+          if(!ke.getTableId().equals(tid))
+            throw new IllegalArgumentException("Unexpected table id "+tid+" != "+ke.getTableId());
+        
+        ArrayList<TCMResult> results = new ArrayList<TCMResult>();
+        
+        Map<KeyExtent,List<ServerConditionalMutation>> deferred = conditionalUpdate(cs, updates, results, symbols);
+  
+        while (deferred.size() > 0) {
+          deferred = conditionalUpdate(cs, deferred, results, symbols);
+        }
+  
+        return results;
+      } catch (IOException ioe) {
+        throw new TException(ioe);
+      }finally{
+        writeTracker.finishWrite(opid);
+        sessionManager.unreserveSession(sessID);
+      }
+    }
+
+    @Override
+    public void invalidateConditionalUpdate(TInfo tinfo, long sessID) throws TException {
+      //this method should wait for any running conditional update to complete
+      //after this method returns a conditional update should not be able to start
+      
+      ConditionalSession cs = (ConditionalSession) sessionManager.getSession(sessID);
+      if (cs != null)
+        cs.interruptFlag.set(true);
+      
+      cs = (ConditionalSession) sessionManager.reserveSession(sessID, true);
+      if(cs != null)
+        sessionManager.removeSession(sessID, true);
+    }
+
+    @Override
+    public void closeConditionalUpdate(TInfo tinfo, long sessID) throws TException {
+      sessionManager.removeSession(sessID, false);
+    }
     
     @Override
     public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint) throws NotServingTabletException,
@@ -2584,6 +2883,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   
   private DistributedWorkQueue bulkFailedCopyQ;
   
+  private String lockID;
+  
   private static final String METRICS_PREFIX = "tserver";
   
   private static ObjectName OBJECT_NAME = null;
@@ -2705,6 +3006,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         
         if (tabletServerLock.tryLock(lw, lockContent)) {
           log.debug("Obtained tablet server lock " + tabletServerLock.getLockPath());
+          lockID = tabletServerLock.getLockID().serialize(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/");
           return;
         }
         log.info("Waiting for tablet server lock");
@@ -2735,7 +3037,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
     clientAddress = new InetSocketAddress(clientAddress.getHostName(), clientPort);
     announceExistence();
-    
+
     ThreadPoolExecutor distWorkQThreadPool = new SimpleThreadPool(getSystemConfiguration().getCount(Property.TSERV_WORKQ_THREADS), "distributed work queue");
     
     bulkFailedCopyQ = new DistributedWorkQueue(ZooUtil.getRoot(instance) + Constants.ZBULK_FAILED_COPYQ);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index 451a079..205cebc 100644
--- a/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -23,8 +23,6 @@ import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.junit.Assert;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -44,6 +42,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class BulkImporterTest {
@@ -67,8 +66,8 @@ public class BulkImporterTest {
     }
     
     @Override
-    public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
-        AccumuloSecurityException, TableNotFoundException {
+    public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+        TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       throw new NotImplementedException();
     }
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
new file mode 100644
index 0000000..7e7480f
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.data.ConditionalMutation;
+
+
+/**
+ * A writer that will sometimes return unknown. When it returns unknown the condition may or may not have been written.
+ */
+public class FaultyConditionalWriter implements ConditionalWriter {
+  
+  private ConditionalWriter cw;
+  private double up;
+  private Random rand;
+  private double wp;
+  
+  public FaultyConditionalWriter(ConditionalWriter cw, double unknownProbability, double writeProbability) {
+    this.cw = cw;
+    this.up = unknownProbability;
+    this.wp = writeProbability;
+    this.rand = new Random();
+
+  }
+
+  public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+    ArrayList<Result> resultList = new ArrayList<Result>();
+    ArrayList<ConditionalMutation> writes = new ArrayList<ConditionalMutation>();
+    
+    while (mutations.hasNext()) {
+      ConditionalMutation cm = mutations.next();
+      if (rand.nextDouble() <= up && rand.nextDouble() > wp)
+        resultList.add(new Result(Status.UNKNOWN, cm, null));
+      else
+        writes.add(cm);
+    }
+    
+    if (writes.size() > 0) {
+      Iterator<Result> results = cw.write(writes.iterator());
+      
+      while (results.hasNext()) {
+        Result result = results.next();
+        
+        if (rand.nextDouble() <= up && rand.nextDouble() <= wp)
+          result = new Result(Status.UNKNOWN, result.getMutation(), result.getTabletServer());
+        resultList.add(result);
+      }
+    }
+    return resultList.iterator();
+  }
+  
+  public Result write(ConditionalMutation mutation) {
+    return write(Collections.singleton(mutation).iterator()).next();
+  }
+  
+  @Override
+  public void close() {
+    cw.close();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
index e2db273..1c62720 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
@@ -30,6 +30,11 @@ public class BadIterator extends WrappingIterator {
   }
   
   @Override
+  public boolean hasTop() {
+    throw new NullPointerException();
+  }
+  
+  @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
index a71b1ad..03eaefb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
@@ -17,10 +17,13 @@
 package org.apache.accumulo.test.functional;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Map;
 
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -30,13 +33,19 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 public class SlowIterator extends WrappingIterator {
 
   static private final String SLEEP_TIME = "sleepTime";
+  static private final String SEEK_SLEEP_TIME = "seekSleepTime";
   
-  long sleepTime;
+  private long sleepTime = 0;
+  private long seekSleepTime = 0;
   
   public static void setSleepTime(IteratorSetting is, long millis) {
     is.addOption(SLEEP_TIME, Long.toString(millis));  
   }
   
+  public static void setSeekSleepTime(IteratorSetting is, long t) {
+    is.addOption(SEEK_SLEEP_TIME, Long.toString(t));
+  }
+
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
     throw new UnsupportedOperationException();
@@ -49,9 +58,20 @@ public class SlowIterator extends WrappingIterator {
   }
   
   @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    UtilWaitThread.sleep(seekSleepTime);
+    super.seek(range, columnFamilies, inclusive);
+  }
+  
+  @Override
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
-    sleepTime = Long.parseLong(options.get(SLEEP_TIME));
+    if (options.containsKey(SLEEP_TIME))
+      sleepTime = Long.parseLong(options.get(SLEEP_TIME));
+    
+    if (options.containsKey(SEEK_SLEEP_TIME))
+      seekSleepTime = Long.parseLong(options.get(SEEK_SLEEP_TIME));
   }
+
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/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 e33603f..c84fd7f 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
@@ -40,7 +40,10 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.MapFileInfo;
 import org.apache.accumulo.core.data.thrift.MultiScanResult;
 import org.apache.accumulo.core.data.thrift.ScanResult;
+import org.apache.accumulo.core.data.thrift.TCMResult;
 import org.apache.accumulo.core.data.thrift.TColumn;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.data.thrift.TMutation;
@@ -50,6 +53,7 @@ import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 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.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -200,6 +204,25 @@ public class NullTserver {
     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)
+        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 {


[4/6] ACCUMULO-1000 added conditional mutations to Accumulo

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
new file mode 100644
index 0000000..b3bfa15
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
@@ -0,0 +1,578 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TConditionalSession implements org.apache.thrift.TBase<TConditionalSession, TConditionalSession._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TConditionalSession");
+
+  private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField TSERVER_LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("tserverLock", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TTL_FIELD_DESC = new org.apache.thrift.protocol.TField("ttl", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TConditionalSessionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TConditionalSessionTupleSchemeFactory());
+  }
+
+  public long sessionId; // required
+  public String tserverLock; // required
+  public long ttl; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SESSION_ID((short)1, "sessionId"),
+    TSERVER_LOCK((short)2, "tserverLock"),
+    TTL((short)3, "ttl");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SESSION_ID
+          return SESSION_ID;
+        case 2: // TSERVER_LOCK
+          return TSERVER_LOCK;
+        case 3: // TTL
+          return TTL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __SESSIONID_ISSET_ID = 0;
+  private static final int __TTL_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  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.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.TSERVER_LOCK, new org.apache.thrift.meta_data.FieldMetaData("tserverLock", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TTL, new org.apache.thrift.meta_data.FieldMetaData("ttl", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TConditionalSession.class, metaDataMap);
+  }
+
+  public TConditionalSession() {
+  }
+
+  public TConditionalSession(
+    long sessionId,
+    String tserverLock,
+    long ttl)
+  {
+    this();
+    this.sessionId = sessionId;
+    setSessionIdIsSet(true);
+    this.tserverLock = tserverLock;
+    this.ttl = ttl;
+    setTtlIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TConditionalSession(TConditionalSession other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.sessionId = other.sessionId;
+    if (other.isSetTserverLock()) {
+      this.tserverLock = other.tserverLock;
+    }
+    this.ttl = other.ttl;
+  }
+
+  public TConditionalSession deepCopy() {
+    return new TConditionalSession(this);
+  }
+
+  @Override
+  public void clear() {
+    setSessionIdIsSet(false);
+    this.sessionId = 0;
+    this.tserverLock = null;
+    setTtlIsSet(false);
+    this.ttl = 0;
+  }
+
+  public long getSessionId() {
+    return this.sessionId;
+  }
+
+  public TConditionalSession setSessionId(long sessionId) {
+    this.sessionId = sessionId;
+    setSessionIdIsSet(true);
+    return this;
+  }
+
+  public void unsetSessionId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+  }
+
+  /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */
+  public boolean isSetSessionId() {
+    return EncodingUtils.testBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+  }
+
+  public void setSessionIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SESSIONID_ISSET_ID, value);
+  }
+
+  public String getTserverLock() {
+    return this.tserverLock;
+  }
+
+  public TConditionalSession setTserverLock(String tserverLock) {
+    this.tserverLock = tserverLock;
+    return this;
+  }
+
+  public void unsetTserverLock() {
+    this.tserverLock = null;
+  }
+
+  /** Returns true if field tserverLock is set (has been assigned a value) and false otherwise */
+  public boolean isSetTserverLock() {
+    return this.tserverLock != null;
+  }
+
+  public void setTserverLockIsSet(boolean value) {
+    if (!value) {
+      this.tserverLock = null;
+    }
+  }
+
+  public long getTtl() {
+    return this.ttl;
+  }
+
+  public TConditionalSession setTtl(long ttl) {
+    this.ttl = ttl;
+    setTtlIsSet(true);
+    return this;
+  }
+
+  public void unsetTtl() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TTL_ISSET_ID);
+  }
+
+  /** Returns true if field ttl is set (has been assigned a value) and false otherwise */
+  public boolean isSetTtl() {
+    return EncodingUtils.testBit(__isset_bitfield, __TTL_ISSET_ID);
+  }
+
+  public void setTtlIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TTL_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SESSION_ID:
+      if (value == null) {
+        unsetSessionId();
+      } else {
+        setSessionId((Long)value);
+      }
+      break;
+
+    case TSERVER_LOCK:
+      if (value == null) {
+        unsetTserverLock();
+      } else {
+        setTserverLock((String)value);
+      }
+      break;
+
+    case TTL:
+      if (value == null) {
+        unsetTtl();
+      } else {
+        setTtl((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SESSION_ID:
+      return Long.valueOf(getSessionId());
+
+    case TSERVER_LOCK:
+      return getTserverLock();
+
+    case TTL:
+      return Long.valueOf(getTtl());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SESSION_ID:
+      return isSetSessionId();
+    case TSERVER_LOCK:
+      return isSetTserverLock();
+    case TTL:
+      return isSetTtl();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TConditionalSession)
+      return this.equals((TConditionalSession)that);
+    return false;
+  }
+
+  public boolean equals(TConditionalSession that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_sessionId = true;
+    boolean that_present_sessionId = true;
+    if (this_present_sessionId || that_present_sessionId) {
+      if (!(this_present_sessionId && that_present_sessionId))
+        return false;
+      if (this.sessionId != that.sessionId)
+        return false;
+    }
+
+    boolean this_present_tserverLock = true && this.isSetTserverLock();
+    boolean that_present_tserverLock = true && that.isSetTserverLock();
+    if (this_present_tserverLock || that_present_tserverLock) {
+      if (!(this_present_tserverLock && that_present_tserverLock))
+        return false;
+      if (!this.tserverLock.equals(that.tserverLock))
+        return false;
+    }
+
+    boolean this_present_ttl = true;
+    boolean that_present_ttl = true;
+    if (this_present_ttl || that_present_ttl) {
+      if (!(this_present_ttl && that_present_ttl))
+        return false;
+      if (this.ttl != that.ttl)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  public int compareTo(TConditionalSession other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TConditionalSession typedOther = (TConditionalSession)other;
+
+    lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(typedOther.isSetSessionId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSessionId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, typedOther.sessionId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTserverLock()).compareTo(typedOther.isSetTserverLock());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTserverLock()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tserverLock, typedOther.tserverLock);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTtl()).compareTo(typedOther.isSetTtl());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTtl()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ttl, typedOther.ttl);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TConditionalSession(");
+    boolean first = true;
+
+    sb.append("sessionId:");
+    sb.append(this.sessionId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tserverLock:");
+    if (this.tserverLock == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tserverLock);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("ttl:");
+    sb.append(this.ttl);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TConditionalSessionStandardSchemeFactory implements SchemeFactory {
+    public TConditionalSessionStandardScheme getScheme() {
+      return new TConditionalSessionStandardScheme();
+    }
+  }
+
+  private static class TConditionalSessionStandardScheme extends StandardScheme<TConditionalSession> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TConditionalSession struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SESSION_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.sessionId = iprot.readI64();
+              struct.setSessionIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TSERVER_LOCK
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tserverLock = iprot.readString();
+              struct.setTserverLockIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TTL
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.ttl = iprot.readI64();
+              struct.setTtlIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TConditionalSession struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(SESSION_ID_FIELD_DESC);
+      oprot.writeI64(struct.sessionId);
+      oprot.writeFieldEnd();
+      if (struct.tserverLock != null) {
+        oprot.writeFieldBegin(TSERVER_LOCK_FIELD_DESC);
+        oprot.writeString(struct.tserverLock);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TTL_FIELD_DESC);
+      oprot.writeI64(struct.ttl);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TConditionalSessionTupleSchemeFactory implements SchemeFactory {
+    public TConditionalSessionTupleScheme getScheme() {
+      return new TConditionalSessionTupleScheme();
+    }
+  }
+
+  private static class TConditionalSessionTupleScheme extends TupleScheme<TConditionalSession> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TConditionalSession struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSessionId()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTserverLock()) {
+        optionals.set(1);
+      }
+      if (struct.isSetTtl()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetSessionId()) {
+        oprot.writeI64(struct.sessionId);
+      }
+      if (struct.isSetTserverLock()) {
+        oprot.writeString(struct.tserverLock);
+      }
+      if (struct.isSetTtl()) {
+        oprot.writeI64(struct.ttl);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TConditionalSession struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.sessionId = iprot.readI64();
+        struct.setSessionIdIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.tserverLock = iprot.readString();
+        struct.setTserverLockIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.ttl = iprot.readI64();
+        struct.setTtlIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
index 4ba0eb6..97001ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 
@@ -472,37 +473,9 @@ public class RelativeKey implements Writable {
     read(in, mbseq, len);
   }
   
-  /**
-   * Determines what next array size should be by rounding up to next power of two.
-   * 
-   */
-  static int nextArraySize(int i) {
-    if (i < 0)
-      throw new IllegalArgumentException();
-    
-    if (i > (1 << 30))
-      return Integer.MAX_VALUE; // this is the next power of 2 minus one... a special case
-
-    if (i == 0) {
-      return 1;
-    }
-    
-    // round up to next power of two
-    int ret = i;
-    ret--;
-    ret |= ret >> 1;
-    ret |= ret >> 2;
-    ret |= ret >> 4;
-    ret |= ret >> 8;
-    ret |= ret >> 16;
-    ret++;
-    
-    return ret;
-  }
-
   private static void read(DataInput in, MByteSequence mbseqDestination, int len) throws IOException {
     if (mbseqDestination.getBackingArray().length < len) {
-      mbseqDestination.setArray(new byte[nextArraySize(len)]);
+      mbseqDestination.setArray(new byte[UnsynchronizedBuffer.nextArraySize(len)]);
     }
     
     in.readFully(mbseqDestination.getBackingArray(), 0, len);
@@ -529,7 +502,7 @@ public class RelativeKey implements Writable {
     int remainingLen = WritableUtils.readVInt(in);
     int len = prefixLen + remainingLen;
     if (dest.getBackingArray().length < len) {
-      dest.setArray(new byte[nextArraySize(len)]);
+      dest.setArray(new byte[UnsynchronizedBuffer.nextArraySize(len)]);
     }
     if (prefixSource.isBackedByArray()) {
       System.arraycopy(prefixSource.getBackingArray(), prefixSource.offset(), dest.getBackingArray(), 0, prefixLen);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
index 1595f5a..d5ca3b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.iterators.system;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.Set;
 
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -36,7 +37,7 @@ public class ColumnQualifierFilter extends Filter {
   
   public ColumnQualifierFilter() {}
   
-  public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<Column> columns) {
+  public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, Set<Column> columns) {
     setSource(iterator);
     init(columns);
   }
@@ -63,7 +64,7 @@ public class ColumnQualifierFilter extends Filter {
     return cfset != null && cfset.contains(key.getColumnFamilyData());
   }
   
-  public void init(HashSet<Column> columns) {
+  public void init(Set<Column> columns) {
     this.columnFamilies = new HashSet<ByteSequence>();
     this.columnsQualifiers = new HashMap<ByteSequence,HashSet<ByteSequence>>();
     


[3/6] ACCUMULO-1000 added conditional mutations to Accumulo

Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/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 9144eb0..19fbb98 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
@@ -1,19 +1,3 @@
-/*
- * 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.0)
  *
@@ -46,7 +30,7 @@ import java.util.Arrays;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@SuppressWarnings("all") public class TabletClientService {
+public class TabletClientService {
 
   public interface Iface extends org.apache.accumulo.core.client.impl.thrift.ClientService.Iface {
 
@@ -70,6 +54,14 @@ 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) 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 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;
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+    public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
     public List<org.apache.accumulo.core.data.thrift.TKeyExtent> bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
     public void splitTablet(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, ByteBuffer splitPoint) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, org.apache.thrift.TException;
@@ -126,6 +118,14 @@ 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<AsyncClient.update_call> 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<AsyncClient.startConditionalUpdate_call> 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<AsyncClient.conditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.invalidateConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.closeConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
     public void bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.bulkImport_call> resultHandler) throws org.apache.thrift.TException;
 
     public void splitTablet(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.splitTablet_call> resultHandler) throws org.apache.thrift.TException;
@@ -453,6 +453,98 @@ 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
+    {
+      send_startConditionalUpdate(tinfo, credentials, authorizations, tableID);
+      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
+    {
+      startConditionalUpdate_args args = new startConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setAuthorizations(authorizations);
+      args.setTableID(tableID);
+      sendBase("startConditionalUpdate", args);
+    }
+
+    public org.apache.accumulo.core.data.thrift.TConditionalSession recv_startConditionalUpdate() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startConditionalUpdate_result result = new startConditionalUpdate_result();
+      receiveBase(result, "startConditionalUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startConditionalUpdate failed: unknown result");
+    }
+
+    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
+    {
+      send_conditionalUpdate(tinfo, sessID, mutations, symbols);
+      return recv_conditionalUpdate();
+    }
+
+    public void send_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 org.apache.thrift.TException
+    {
+      conditionalUpdate_args args = new conditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      args.setMutations(mutations);
+      args.setSymbols(symbols);
+      sendBase("conditionalUpdate", args);
+    }
+
+    public List<org.apache.accumulo.core.data.thrift.TCMResult> recv_conditionalUpdate() throws NoSuchScanIDException, org.apache.thrift.TException
+    {
+      conditionalUpdate_result result = new conditionalUpdate_result();
+      receiveBase(result, "conditionalUpdate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.nssi != null) {
+        throw result.nssi;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "conditionalUpdate failed: unknown result");
+    }
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      send_invalidateConditionalUpdate(tinfo, sessID);
+      recv_invalidateConditionalUpdate();
+    }
+
+    public void send_invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      sendBase("invalidateConditionalUpdate", args);
+    }
+
+    public void recv_invalidateConditionalUpdate() throws org.apache.thrift.TException
+    {
+      invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+      receiveBase(result, "invalidateConditionalUpdate");
+      return;
+    }
+
+    public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      send_closeConditionalUpdate(tinfo, sessID);
+    }
+
+    public void send_closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+    {
+      closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+      args.setTinfo(tinfo);
+      args.setSessID(sessID);
+      sendBase("closeConditionalUpdate", args);
+    }
+
     public List<org.apache.accumulo.core.data.thrift.TKeyExtent> bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
       send_bulkImport(tinfo, credentials, tid, files, setTime);
@@ -1219,6 +1311,157 @@ 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<startConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      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<startConditionalUpdate_call> 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;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startConditionalUpdate_args args = new startConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setAuthorizations(authorizations);
+        args.setTableID(tableID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public org.apache.accumulo.core.data.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_startConditionalUpdate();
+      }
+    }
+
+    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<conditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private long sessID;
+      private Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations;
+      private List<String> symbols;
+      public conditionalUpdate_call(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<conditionalUpdate_call> 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.sessID = sessID;
+        this.mutations = mutations;
+        this.symbols = symbols;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        conditionalUpdate_args args = new conditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.setMutations(mutations);
+        args.setSymbols(symbols);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<org.apache.accumulo.core.data.thrift.TCMResult> getResult() throws NoSuchScanIDException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_conditionalUpdate();
+      }
+    }
+
+    public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<invalidateConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      invalidateConditionalUpdate_call method_call = new invalidateConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class invalidateConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private long sessID;
+      public invalidateConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<invalidateConditionalUpdate_call> 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.sessID = sessID;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("invalidateConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      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!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_invalidateConditionalUpdate();
+      }
+    }
+
+    public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<closeConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      closeConditionalUpdate_call method_call = new closeConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class closeConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private long sessID;
+      public closeConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<closeConditionalUpdate_call> 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, true);
+        this.tinfo = tinfo;
+        this.sessID = sessID;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+        args.setTinfo(tinfo);
+        args.setSessID(sessID);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      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!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+      }
+    }
+
     public void bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<bulkImport_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       bulkImport_call method_call = new bulkImport_call(tinfo, credentials, tid, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
@@ -1872,6 +2115,10 @@ import org.slf4j.LoggerFactory;
       processMap.put("applyUpdates", new applyUpdates());
       processMap.put("closeUpdate", new closeUpdate());
       processMap.put("update", new update());
+      processMap.put("startConditionalUpdate", new startConditionalUpdate());
+      processMap.put("conditionalUpdate", new conditionalUpdate());
+      processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+      processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
       processMap.put("bulkImport", new bulkImport());
       processMap.put("splitTablet", new splitTablet());
       processMap.put("loadTablet", new loadTablet());
@@ -2134,6 +2381,93 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class startConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startConditionalUpdate_args> {
+      public startConditionalUpdate() {
+        super("startConditionalUpdate");
+      }
+
+      public startConditionalUpdate_args getEmptyArgsInstance() {
+        return new startConditionalUpdate_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      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);
+        } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class conditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, conditionalUpdate_args> {
+      public conditionalUpdate() {
+        super("conditionalUpdate");
+      }
+
+      public conditionalUpdate_args getEmptyArgsInstance() {
+        return new conditionalUpdate_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public conditionalUpdate_result getResult(I iface, conditionalUpdate_args args) throws org.apache.thrift.TException {
+        conditionalUpdate_result result = new conditionalUpdate_result();
+        try {
+          result.success = iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols);
+        } catch (NoSuchScanIDException nssi) {
+          result.nssi = nssi;
+        }
+        return result;
+      }
+    }
+
+    public static class invalidateConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, invalidateConditionalUpdate_args> {
+      public invalidateConditionalUpdate() {
+        super("invalidateConditionalUpdate");
+      }
+
+      public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+        return new invalidateConditionalUpdate_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public invalidateConditionalUpdate_result getResult(I iface, invalidateConditionalUpdate_args args) throws org.apache.thrift.TException {
+        invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+        iface.invalidateConditionalUpdate(args.tinfo, args.sessID);
+        return result;
+      }
+    }
+
+    public static class closeConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeConditionalUpdate_args> {
+      public closeConditionalUpdate() {
+        super("closeConditionalUpdate");
+      }
+
+      public closeConditionalUpdate_args getEmptyArgsInstance() {
+        return new closeConditionalUpdate_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, closeConditionalUpdate_args args) throws org.apache.thrift.TException {
+        iface.closeConditionalUpdate(args.tinfo, args.sessID);
+        return null;
+      }
+    }
+
     public static class bulkImport<I extends Iface> extends org.apache.thrift.ProcessFunction<I, bulkImport_args> {
       public bulkImport() {
         super("bulkImport");
@@ -2516,7 +2850,7 @@ import org.slf4j.LoggerFactory;
     public boolean isolated; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)11, "tinfo"),
       CREDENTIALS((short)1, "credentials"),
       EXTENT((short)2, "extent"),
@@ -3685,7 +4019,7 @@ import org.slf4j.LoggerFactory;
                   struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list106.size);
                   for (int _i107 = 0; _i107 < _list106.size; ++_i107)
                   {
-                    org.apache.accumulo.core.data.thrift.TColumn _elem108; // optional
+                    org.apache.accumulo.core.data.thrift.TColumn _elem108; // required
                     _elem108 = new org.apache.accumulo.core.data.thrift.TColumn();
                     _elem108.read(iprot);
                     struct.columns.add(_elem108);
@@ -3712,7 +4046,7 @@ import org.slf4j.LoggerFactory;
                   struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list109.size);
                   for (int _i110 = 0; _i110 < _list109.size; ++_i110)
                   {
-                    org.apache.accumulo.core.data.thrift.IterInfo _elem111; // optional
+                    org.apache.accumulo.core.data.thrift.IterInfo _elem111; // required
                     _elem111 = new org.apache.accumulo.core.data.thrift.IterInfo();
                     _elem111.read(iprot);
                     struct.ssiList.add(_elem111);
@@ -3763,7 +4097,7 @@ import org.slf4j.LoggerFactory;
                   struct.authorizations = new ArrayList<ByteBuffer>(_list120.size);
                   for (int _i121 = 0; _i121 < _list120.size; ++_i121)
                   {
-                    ByteBuffer _elem122; // optional
+                    ByteBuffer _elem122; // required
                     _elem122 = iprot.readBinary();
                     struct.authorizations.add(_elem122);
                   }
@@ -4040,7 +4374,7 @@ import org.slf4j.LoggerFactory;
             struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list133.size);
             for (int _i134 = 0; _i134 < _list133.size; ++_i134)
             {
-              org.apache.accumulo.core.data.thrift.TColumn _elem135; // optional
+              org.apache.accumulo.core.data.thrift.TColumn _elem135; // required
               _elem135 = new org.apache.accumulo.core.data.thrift.TColumn();
               _elem135.read(iprot);
               struct.columns.add(_elem135);
@@ -4058,7 +4392,7 @@ import org.slf4j.LoggerFactory;
             struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list136.size);
             for (int _i137 = 0; _i137 < _list136.size; ++_i137)
             {
-              org.apache.accumulo.core.data.thrift.IterInfo _elem138; // optional
+              org.apache.accumulo.core.data.thrift.IterInfo _elem138; // required
               _elem138 = new org.apache.accumulo.core.data.thrift.IterInfo();
               _elem138.read(iprot);
               struct.ssiList.add(_elem138);
@@ -4098,7 +4432,7 @@ import org.slf4j.LoggerFactory;
             struct.authorizations = new ArrayList<ByteBuffer>(_list147.size);
             for (int _i148 = 0; _i148 < _list147.size; ++_i148)
             {
-              ByteBuffer _elem149; // optional
+              ByteBuffer _elem149; // required
               _elem149 = iprot.readBinary();
               struct.authorizations.add(_elem149);
             }
@@ -4138,7 +4472,7 @@ import org.slf4j.LoggerFactory;
     public TooManyFilesException tmfe; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       SEC((short)1, "sec"),
       NSTE((short)2, "nste"),
@@ -4799,7 +5133,7 @@ import org.slf4j.LoggerFactory;
     public long scanID; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       SCAN_ID((short)1, "scanID");
 
@@ -5260,7 +5594,7 @@ import org.slf4j.LoggerFactory;
     public TooManyFilesException tmfe; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       NSSI((short)1, "nssi"),
       NSTE((short)2, "nste"),
@@ -5921,7 +6255,7 @@ import org.slf4j.LoggerFactory;
     public long scanID; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       SCAN_ID((short)1, "scanID");
 
@@ -6390,7 +6724,7 @@ import org.slf4j.LoggerFactory;
     public boolean waitForWrites; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)8, "tinfo"),
       CREDENTIALS((short)1, "credentials"),
       BATCH((short)2, "batch"),
@@ -7338,7 +7672,7 @@ import org.slf4j.LoggerFactory;
                       _val153 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list154.size);
                       for (int _i155 = 0; _i155 < _list154.size; ++_i155)
                       {
-                        org.apache.accumulo.core.data.thrift.TRange _elem156; // optional
+                        org.apache.accumulo.core.data.thrift.TRange _elem156; // required
                         _elem156 = new org.apache.accumulo.core.data.thrift.TRange();
                         _elem156.read(iprot);
                         _val153.add(_elem156);
@@ -7361,7 +7695,7 @@ import org.slf4j.LoggerFactory;
                   struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list157.size);
                   for (int _i158 = 0; _i158 < _list157.size; ++_i158)
                   {
-                    org.apache.accumulo.core.data.thrift.TColumn _elem159; // optional
+                    org.apache.accumulo.core.data.thrift.TColumn _elem159; // required
                     _elem159 = new org.apache.accumulo.core.data.thrift.TColumn();
                     _elem159.read(iprot);
                     struct.columns.add(_elem159);
@@ -7380,7 +7714,7 @@ import org.slf4j.LoggerFactory;
                   struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list160.size);
                   for (int _i161 = 0; _i161 < _list160.size; ++_i161)
                   {
-                    org.apache.accumulo.core.data.thrift.IterInfo _elem162; // optional
+                    org.apache.accumulo.core.data.thrift.IterInfo _elem162; // required
                     _elem162 = new org.apache.accumulo.core.data.thrift.IterInfo();
                     _elem162.read(iprot);
                     struct.ssiList.add(_elem162);
@@ -7431,7 +7765,7 @@ import org.slf4j.LoggerFactory;
                   struct.authorizations = new ArrayList<ByteBuffer>(_list171.size);
                   for (int _i172 = 0; _i172 < _list171.size; ++_i172)
                   {
-                    ByteBuffer _elem173; // optional
+                    ByteBuffer _elem173; // required
                     _elem173 = iprot.readBinary();
                     struct.authorizations.add(_elem173);
                   }
@@ -7698,7 +8032,7 @@ import org.slf4j.LoggerFactory;
                 _val191 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list192.size);
                 for (int _i193 = 0; _i193 < _list192.size; ++_i193)
                 {
-                  org.apache.accumulo.core.data.thrift.TRange _elem194; // optional
+                  org.apache.accumulo.core.data.thrift.TRange _elem194; // required
                   _elem194 = new org.apache.accumulo.core.data.thrift.TRange();
                   _elem194.read(iprot);
                   _val191.add(_elem194);
@@ -7715,7 +8049,7 @@ import org.slf4j.LoggerFactory;
             struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list195.size);
             for (int _i196 = 0; _i196 < _list195.size; ++_i196)
             {
-              org.apache.accumulo.core.data.thrift.TColumn _elem197; // optional
+              org.apache.accumulo.core.data.thrift.TColumn _elem197; // required
               _elem197 = new org.apache.accumulo.core.data.thrift.TColumn();
               _elem197.read(iprot);
               struct.columns.add(_elem197);
@@ -7729,7 +8063,7 @@ import org.slf4j.LoggerFactory;
             struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list198.size);
             for (int _i199 = 0; _i199 < _list198.size; ++_i199)
             {
-              org.apache.accumulo.core.data.thrift.IterInfo _elem200; // optional
+              org.apache.accumulo.core.data.thrift.IterInfo _elem200; // required
               _elem200 = new org.apache.accumulo.core.data.thrift.IterInfo();
               _elem200.read(iprot);
               struct.ssiList.add(_elem200);
@@ -7769,7 +8103,7 @@ import org.slf4j.LoggerFactory;
             struct.authorizations = new ArrayList<ByteBuffer>(_list209.size);
             for (int _i210 = 0; _i210 < _list209.size; ++_i210)
             {
-              ByteBuffer _elem211; // optional
+              ByteBuffer _elem211; // required
               _elem211 = iprot.readBinary();
               struct.authorizations.add(_elem211);
             }
@@ -7801,7 +8135,7 @@ import org.slf4j.LoggerFactory;
     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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       SEC((short)1, "sec");
 
@@ -8262,7 +8596,7 @@ import org.slf4j.LoggerFactory;
     public long scanID; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       SCAN_ID((short)1, "scanID");
 
@@ -8719,7 +9053,7 @@ import org.slf4j.LoggerFactory;
     public NoSuchScanIDException nssi; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       NSSI((short)1, "nssi");
 
@@ -9180,7 +9514,7 @@ import org.slf4j.LoggerFactory;
     public long scanID; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       SCAN_ID((short)1, "scanID");
 
@@ -9635,7 +9969,7 @@ import org.slf4j.LoggerFactory;
     public NoSuchScanIDException nssi; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       NSSI((short)1, "nssi");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -9993,7 +10327,7 @@ import org.slf4j.LoggerFactory;
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       CREDENTIALS((short)1, "credentials");
 
@@ -10457,7 +10791,7 @@ import org.slf4j.LoggerFactory;
     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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       SEC((short)1, "sec");
 
@@ -10917,7 +11251,7 @@ import org.slf4j.LoggerFactory;
     public List<org.apache.accumulo.core.data.thrift.TMutation> mutations; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)1, "tinfo"),
       UPDATE_ID((short)2, "updateID"),
       KEY_EXTENT((short)3, "keyExtent"),
@@ -11473,7 +11807,7 @@ import org.slf4j.LoggerFactory;
                   struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list212.size);
                   for (int _i213 = 0; _i213 < _list212.size; ++_i213)
                   {
-                    org.apache.accumulo.core.data.thrift.TMutation _elem214; // optional
+                    org.apache.accumulo.core.data.thrift.TMutation _elem214; // required
                     _elem214 = new org.apache.accumulo.core.data.thrift.TMutation();
                     _elem214.read(iprot);
                     struct.mutations.add(_elem214);
@@ -11600,7 +11934,7 @@ import org.slf4j.LoggerFactory;
             struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list217.size);
             for (int _i218 = 0; _i218 < _list217.size; ++_i218)
             {
-              org.apache.accumulo.core.data.thrift.TMutation _elem219; // optional
+              org.apache.accumulo.core.data.thrift.TMutation _elem219; // required
               _elem219 = new org.apache.accumulo.core.data.thrift.TMutation();
               _elem219.read(iprot);
               struct.mutations.add(_elem219);
@@ -11629,7 +11963,7 @@ import org.slf4j.LoggerFactory;
     public long updateID; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)2, "tinfo"),
       UPDATE_ID((short)1, "updateID");
 
@@ -12086,7 +12420,7 @@ import org.slf4j.LoggerFactory;
     public NoSuchScanIDException nssi; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SUCCESS((short)0, "success"),
       NSSI((short)1, "nssi");
 
@@ -12551,7 +12885,7 @@ import org.slf4j.LoggerFactory;
     public org.apache.accumulo.core.data.thrift.TMutation mutation; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       TINFO((short)4, "tinfo"),
       CREDENTIALS((short)1, "credentials"),
       KEY_EXTENT((short)2, "keyExtent"),
@@ -13223,7 +13557,7 @@ import org.slf4j.LoggerFactory;
     public ConstraintViolationException cve; // 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 {
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
       SEC((short)1, "sec"),
       NSTE((short)2, "nste"),
       CVE((short)3, "cve");
@@ -13765,6 +14099,3649 @@ import org.slf4j.LoggerFactory;
 
   }
 
+  public static class startConditionalUpdate_args implements org.apache.thrift.TBase<startConditionalUpdate_args, startConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_args");
+
+    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)1);
+    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 Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new startConditionalUpdate_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new startConditionalUpdate_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public List<ByteBuffer> authorizations; // required
+    public String tableID; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    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");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // AUTHORIZATIONS
+            return AUTHORIZATIONS;
+          case 4: // TABLE_ID
+            return TABLE_ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              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)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_args.class, metaDataMap);
+    }
+
+    public startConditionalUpdate_args() {
+    }
+
+    public startConditionalUpdate_args(
+      org.apache.accumulo.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      List<ByteBuffer> authorizations,
+      String tableID)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.authorizations = authorizations;
+      this.tableID = tableID;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startConditionalUpdate_args(startConditionalUpdate_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetAuthorizations()) {
+        List<ByteBuffer> __this__authorizations = new ArrayList<ByteBuffer>();
+        for (ByteBuffer other_element : other.authorizations) {
+          ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element);
+;
+          __this__authorizations.add(temp_binary_element);
+        }
+        this.authorizations = __this__authorizations;
+      }
+      if (other.isSetTableID()) {
+        this.tableID = other.tableID;
+      }
+    }
+
+    public startConditionalUpdate_args deepCopy() {
+      return new startConditionalUpdate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.authorizations = null;
+      this.tableID = null;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startConditionalUpdate_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public startConditionalUpdate_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public int getAuthorizationsSize() {
+      return (this.authorizations == null) ? 0 : this.authorizations.size();
+    }
+
+    public java.util.Iterator<ByteBuffer> getAuthorizationsIterator() {
+      return (this.authorizations == null) ? null : this.authorizations.iterator();
+    }
+
+    public void addToAuthorizations(ByteBuffer elem) {
+      if (this.authorizations == null) {
+        this.authorizations = new ArrayList<ByteBuffer>();
+      }
+      this.authorizations.add(elem);
+    }
+
+    public List<ByteBuffer> getAuthorizations() {
+      return this.authorizations;
+    }
+
+    public startConditionalUpdate_args setAuthorizations(List<ByteBuffer> authorizations) {
+      this.authorizations = authorizations;
+      return this;
+    }
+
+    public void unsetAuthorizations() {
+      this.authorizations = null;
+    }
+
+    /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
+    public boolean isSetAuthorizations() {
+      return this.authorizations != null;
+    }
+
+    public void setAuthorizationsIsSet(boolean value) {
+      if (!value) {
+        this.authorizations = null;
+      }
+    }
+
+    public String getTableID() {
+      return this.tableID;
+    }
+
+    public startConditionalUpdate_args setTableID(String tableID) {
+      this.tableID = tableID;
+      return this;
+    }
+
+    public void unsetTableID() {
+      this.tableID = null;
+    }
+
+    /** Returns true if field tableID is set (has been assigned a value) and false otherwise */
+    public boolean isSetTableID() {
+      return this.tableID != null;
+    }
+
+    public void setTableIDIsSet(boolean value) {
+      if (!value) {
+        this.tableID = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case AUTHORIZATIONS:
+        if (value == null) {
+          unsetAuthorizations();
+        } else {
+          setAuthorizations((List<ByteBuffer>)value);
+        }
+        break;
+
+      case TABLE_ID:
+        if (value == null) {
+          unsetTableID();
+        } else {
+          setTableID((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case AUTHORIZATIONS:
+        return getAuthorizations();
+
+      case TABLE_ID:
+        return getTableID();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case AUTHORIZATIONS:
+        return isSetAuthorizations();
+      case TABLE_ID:
+        return isSetTableID();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof startConditionalUpdate_args)
+        return this.equals((startConditionalUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(startConditionalUpdate_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_authorizations = true && this.isSetAuthorizations();
+      boolean that_present_authorizations = true && that.isSetAuthorizations();
+      if (this_present_authorizations || that_present_authorizations) {
+        if (!(this_present_authorizations && that_present_authorizations))
+          return false;
+        if (!this.authorizations.equals(that.authorizations))
+          return false;
+      }
+
+      boolean this_present_tableID = true && this.isSetTableID();
+      boolean that_present_tableID = true && that.isSetTableID();
+      if (this_present_tableID || that_present_tableID) {
+        if (!(this_present_tableID && that_present_tableID))
+          return false;
+        if (!this.tableID.equals(that.tableID))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(startConditionalUpdate_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      startConditionalUpdate_args typedOther = (startConditionalUpdate_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetAuthorizations()).compareTo(typedOther.isSetAuthorizations());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetAuthorizations()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, typedOther.authorizations);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTableID()).compareTo(typedOther.isSetTableID());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTableID()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableID, typedOther.tableID);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("startConditionalUpdate_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("authorizations:");
+      if (this.authorizations == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.authorizations);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tableID:");
+      if (this.tableID == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableID);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class startConditionalUpdate_argsStandardSchemeFactory implements SchemeFactory {
+      public startConditionalUpdate_argsStandardScheme getScheme() {
+        return new startConditionalUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_argsStandardScheme extends StandardScheme<startConditionalUpdate_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // AUTHORIZATIONS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list220 = iprot.readListBegin();
+                  struct.authorizations = new ArrayList<ByteBuffer>(_list220.size);
+                  for (int _i221 = 0; _i221 < _list220.size; ++_i221)
+                  {
+                    ByteBuffer _elem222; // required
+                    _elem222 = iprot.readBinary();
+                    struct.authorizations.add(_elem222);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setAuthorizationsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // TABLE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableID = iprot.readString();
+                struct.setTableIDIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.authorizations != null) {
+          oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
+            for (ByteBuffer _iter223 : struct.authorizations)
+            {
+              oprot.writeBinary(_iter223);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableID != null) {
+          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+          oprot.writeString(struct.tableID);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startConditionalUpdate_argsTupleSchemeFactory implements SchemeFactory {
+      public startConditionalUpdate_argsTupleScheme getScheme() {
+        return new startConditionalUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_argsTupleScheme extends TupleScheme<startConditionalUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetAuthorizations()) {
+          optionals.set(2);
+        }
+        if (struct.isSetTableID()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetAuthorizations()) {
+          {
+            oprot.writeI32(struct.authorizations.size());
+            for (ByteBuffer _iter224 : struct.authorizations)
+            {
+              oprot.writeBinary(_iter224);
+            }
+          }
+        }
+        if (struct.isSetTableID()) {
+          oprot.writeString(struct.tableID);
+        }
+      }
+
+      @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);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          {
+            org.apache.thrift.protocol.TList _list225 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.authorizations = new ArrayList<ByteBuffer>(_list225.size);
+            for (int _i226 = 0; _i226 < _list225.size; ++_i226)
+            {
+              ByteBuffer _elem227; // required
+              _elem227 = iprot.readBinary();
+              struct.authorizations.add(_elem227);
+            }
+          }
+          struct.setAuthorizationsIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.tableID = iprot.readString();
+          struct.setTableIDIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class startConditionalUpdate_result implements org.apache.thrift.TBase<startConditionalUpdate_result, startConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    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 {
+      schemes.put(StandardScheme.class, new startConditionalUpdate_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new startConditionalUpdate_resultTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.core.data.thrift.TConditionalSession success; // required
+    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. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      SEC((short)1, "sec");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // SEC
+            return SEC;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TConditionalSession.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(startConditionalUpdate_result.class, metaDataMap);
+    }
+
+    public startConditionalUpdate_result() {
+    }
+
+    public startConditionalUpdate_result(
+      org.apache.accumulo.core.data.thrift.TConditionalSession success,
+      org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.success = success;
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startConditionalUpdate_result(startConditionalUpdate_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.data.thrift.TConditionalSession(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    public startConditionalUpdate_result deepCopy() {
+      return new startConditionalUpdate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    public org.apache.accumulo.core.data.thrift.TConditionalSession getSuccess() {
+      return this.success;
+    }
+
+    public startConditionalUpdate_result setSuccess(org.apache.accumulo.core.data.thrift.TConditionalSession success) {
+      this.success = success;
+      return this;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public startConditionalUpdate_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 SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((org.apache.accumulo.core.data.thrift.TConditionalSession)value);
+        }
+        break;
+
+      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 SUCCESS:
+        return getSuccess();
+
+      case SEC:
+        return getSec();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof startConditionalUpdate_result)
+        return this.equals((startConditionalUpdate_result)that);
+      return false;
+    }
+
+    public boolean equals(startConditionalUpdate_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          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;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(startConditionalUpdate_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      startConditionalUpdate_result typedOther = (startConditionalUpdate_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("startConditionalUpdate_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class startConditionalUpdate_resultStandardSchemeFactory implements SchemeFactory {
+      public startConditionalUpdate_resultStandardScheme getScheme() {
+        return new startConditionalUpdate_resultStandardScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_resultStandardScheme extends StandardScheme<startConditionalUpdate_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new org.apache.accumulo.core.data.thrift.TConditionalSession();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            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);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startConditionalUpdate_resultTupleSchemeFactory implements SchemeFactory {
+      public startConditionalUpdate_resultTupleScheme getScheme() {
+        return new startConditionalUpdate_resultTupleScheme();
+      }
+    }
+
+    private static class startConditionalUpdate_resultTupleScheme extends TupleScheme<startConditionalUpdate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetSec()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = new org.apache.accumulo.core.data.thrift.TConditionalSession();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class conditionalUpdate_args implements org.apache.thrift.TBase<conditionalUpdate_args, conditionalUpdate_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_args");
+
+    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)1);
+    private static final org.apache.thrift.protocol.TField SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+    private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.MAP, (short)3);
+    private static final org.apache.thrift.protocol.TField SYMBOLS_FIELD_DESC = new org.apache.thrift.protocol.TField("symbols", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new conditionalUpdate_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new conditionalUpdate_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public long sessID; // required
+    public Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations; // required
+    public List<String> symbols; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      SESS_ID((short)2, "sessID"),
+      MUTATIONS((short)3, "mutations"),
+      SYMBOLS((short)4, "symbols");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {

<TRUNCATED>