You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/06/21 09:49:03 UTC

[1/2] Return current values on failed CAS

Updated Branches:
  refs/heads/trunk 56d2296ad -> 0ba5cf12b


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/KsDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/KsDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/KsDef.java
index 3f2ee29..826e259 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/KsDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/KsDef.java
@@ -869,7 +869,7 @@ public class KsDef implements org.apache.thrift.TBase<KsDef, KsDef._Fields>, jav
                 struct.cf_defs = new ArrayList<CfDef>(_list144.size);
                 for (int _i145 = 0; _i145 < _list144.size; ++_i145)
                 {
-                  CfDef _elem146; // optional
+                  CfDef _elem146; // required
                   _elem146 = new CfDef();
                   _elem146.read(iprot);
                   struct.cf_defs.add(_elem146);
@@ -1018,7 +1018,7 @@ public class KsDef implements org.apache.thrift.TBase<KsDef, KsDef._Fields>, jav
         struct.cf_defs = new ArrayList<CfDef>(_list151.size);
         for (int _i152 = 0; _i152 < _list151.size; ++_i152)
         {
-          CfDef _elem153; // optional
+          CfDef _elem153; // required
           _elem153 = new CfDef();
           _elem153.read(iprot);
           struct.cf_defs.add(_elem153);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/SlicePredicate.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/SlicePredicate.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/SlicePredicate.java
index 1a3dfd5..0624e93 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/SlicePredicate.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/SlicePredicate.java
@@ -467,7 +467,7 @@ public class SlicePredicate implements org.apache.thrift.TBase<SlicePredicate, S
                 struct.column_names = new ArrayList<ByteBuffer>(_list16.size);
                 for (int _i17 = 0; _i17 < _list16.size; ++_i17)
                 {
-                  ByteBuffer _elem18; // optional
+                  ByteBuffer _elem18; // required
                   _elem18 = iprot.readBinary();
                   struct.column_names.add(_elem18);
                 }
@@ -572,7 +572,7 @@ public class SlicePredicate implements org.apache.thrift.TBase<SlicePredicate, S
           struct.column_names = new ArrayList<ByteBuffer>(_list21.size);
           for (int _i22 = 0; _i22 < _list21.size; ++_i22)
           {
-            ByteBuffer _elem23; // optional
+            ByteBuffer _elem23; // required
             _elem23 = iprot.readBinary();
             struct.column_names.add(_elem23);
           }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
index aa14815..3a30a21 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
@@ -485,7 +485,7 @@ public class SuperColumn implements org.apache.thrift.TBase<SuperColumn, SuperCo
                 struct.columns = new ArrayList<Column>(_list0.size);
                 for (int _i1 = 0; _i1 < _list0.size; ++_i1)
                 {
-                  Column _elem2; // optional
+                  Column _elem2; // required
                   _elem2 = new Column();
                   _elem2.read(iprot);
                   struct.columns.add(_elem2);
@@ -566,7 +566,7 @@ public class SuperColumn implements org.apache.thrift.TBase<SuperColumn, SuperCo
         struct.columns = new ArrayList<Column>(_list5.size);
         for (int _i6 = 0; _i6 < _list5.size; ++_i6)
         {
-          Column _elem7; // optional
+          Column _elem7; // required
           _elem7 = new Column();
           _elem7.read(iprot);
           struct.columns.add(_elem7);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/TokenRange.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/TokenRange.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/TokenRange.java
index fc32d02..1393fd5 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/TokenRange.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/TokenRange.java
@@ -774,7 +774,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
                 struct.endpoints = new ArrayList<String>(_list48.size);
                 for (int _i49 = 0; _i49 < _list48.size; ++_i49)
                 {
-                  String _elem50; // optional
+                  String _elem50; // required
                   _elem50 = iprot.readString();
                   struct.endpoints.add(_elem50);
                 }
@@ -792,7 +792,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
                 struct.rpc_endpoints = new ArrayList<String>(_list51.size);
                 for (int _i52 = 0; _i52 < _list51.size; ++_i52)
                 {
-                  String _elem53; // optional
+                  String _elem53; // required
                   _elem53 = iprot.readString();
                   struct.rpc_endpoints.add(_elem53);
                 }
@@ -810,7 +810,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
                 struct.endpoint_details = new ArrayList<EndpointDetails>(_list54.size);
                 for (int _i55 = 0; _i55 < _list54.size; ++_i55)
                 {
-                  EndpointDetails _elem56; // optional
+                  EndpointDetails _elem56; // required
                   _elem56 = new EndpointDetails();
                   _elem56.read(iprot);
                   struct.endpoint_details.add(_elem56);
@@ -953,7 +953,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
         struct.endpoints = new ArrayList<String>(_list63.size);
         for (int _i64 = 0; _i64 < _list63.size; ++_i64)
         {
-          String _elem65; // optional
+          String _elem65; // required
           _elem65 = iprot.readString();
           struct.endpoints.add(_elem65);
         }
@@ -966,7 +966,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
           struct.rpc_endpoints = new ArrayList<String>(_list66.size);
           for (int _i67 = 0; _i67 < _list66.size; ++_i67)
           {
-            String _elem68; // optional
+            String _elem68; // required
             _elem68 = iprot.readString();
             struct.rpc_endpoints.add(_elem68);
           }
@@ -979,7 +979,7 @@ public class TokenRange implements org.apache.thrift.TBase<TokenRange, TokenRang
           struct.endpoint_details = new ArrayList<EndpointDetails>(_list69.size);
           for (int _i70 = 0; _i70 < _list69.size; ++_i70)
           {
-            EndpointDetails _elem71; // optional
+            EndpointDetails _elem71; // required
             _elem71 = new EndpointDetails();
             _elem71.read(iprot);
             struct.endpoint_details.add(_elem71);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 62f7fbd..62bd976 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -372,11 +372,34 @@ public abstract class ModificationStatement implements CQLStatement
         ColumnFamily updates = updateForKey(key, clusteringPrefix, params);
         ColumnFamily expected = buildConditions(key, clusteringPrefix, params);
 
-        boolean result = StorageProxy.cas(keyspace(), columnFamily(), key, expected, updates, cl);
+        ColumnFamily result = StorageProxy.cas(keyspace(), columnFamily(), key, clusteringPrefix, expected, updates, cl);
+        return result == null
+             ? new ResultMessage.Void()
+             : new ResultMessage.Rows(buildCasFailureResultSet(key, result));
+    }
+
+    private ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
+    {
+        CFDefinition cfDef = cfm.getCfDef();
+
+        Selection selection;
+        if (ifNotExists)
+        {
+            selection = Selection.wildcard(cfDef);
+        }
+        else
+        {
+            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>(columnConditions.size());
+            for (Operation condition : columnConditions)
+                names.add(cfDef.get(condition.columnName));
+            selection = Selection.forColumns(names);
+        }
+
+        long now = System.currentTimeMillis();
+        Selection.ResultSetBuilder builder = selection.resultSetBuilder(now);
+        SelectStatement.forSelection(cfDef, selection).processColumnFamily(key, cf, Collections.<ByteBuffer>emptyList(), Integer.MAX_VALUE, now, builder);
 
-        ResultSet.Metadata metadata = new ResultSet.Metadata(Collections.singletonList(new ColumnSpecification(keyspace(), columnFamily(), RESULT_COLUMN, BooleanType.instance)));
-        List<List<ByteBuffer>> newRows = Collections.singletonList(Collections.singletonList(BooleanType.instance.decompose(result)));
-        return new ResultMessage.Rows(new ResultSet(metadata, newRows));
+        return builder.build();
     }
 
     public ResultMessage executeInternal(QueryState queryState) throws RequestValidationException, RequestExecutionException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index bdfc326..2be85c9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -76,6 +76,9 @@ public class SelectStatement implements CQLStatement
 
     private Map<CFDefinition.Name, Integer> orderingIndexes;
 
+    // Used by forSelection below
+    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, null, false);
+
     private static enum Bound
     {
         START(0), END(1);
@@ -104,6 +107,14 @@ public class SelectStatement implements CQLStatement
         this.limit = limit;
     }
 
+    // Creates a simple select based on the given selection.
+    // Note that the results select statement should not be used for actual queries, but only for processing already
+    // queried data through processColumnFamily.
+    static SelectStatement forSelection(CFDefinition cfDef, Selection selection)
+    {
+        return new SelectStatement(cfDef, 0, defaultParameters, selection, null);
+    }
+
     public int getBoundsTerms()
     {
         return boundTerms;
@@ -618,6 +629,9 @@ public class SelectStatement implements CQLStatement
 
     private Iterable<Column> columnsInOrder(final ColumnFamily cf, final List<ByteBuffer> variables) throws InvalidRequestException
     {
+        if (columnRestrictions.length == 0)
+            return cf.getSortedColumns();
+
         // If the restriction for the last column alias is an IN, respect
         // requested order
         Restriction last = columnRestrictions[columnRestrictions.length - 1];
@@ -665,105 +679,111 @@ public class SelectStatement implements CQLStatement
             if (row.cf == null)
                 continue;
 
-            ByteBuffer[] keyComponents = cfDef.hasCompositeKey
-                                       ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(row.key.key)
-                                       : new ByteBuffer[]{ row.key.key };
-
-            if (cfDef.isCompact)
-            {
-                // One cqlRow per column
-                for (Column c : columnsInOrder(row.cf, variables))
-                {
-                    if (c.isMarkedForDelete(now))
-                        continue;
+            processColumnFamily(row.key.key, row.cf, variables, limit, now, result);
+        }
 
-                    ByteBuffer[] components = null;
-                    if (cfDef.isComposite)
-                    {
-                        components = ((CompositeType)cfDef.cfm.comparator).split(c.name());
-                    }
-                    else if (sliceRestriction != null)
-                    {
-                        // For dynamic CF, the column could be out of the requested bounds, filter here
-                        if (!sliceRestriction.isInclusive(Bound.START) && c.name().equals(sliceRestriction.bound(Bound.START).bindAndGet(variables)))
-                            continue;
-                        if (!sliceRestriction.isInclusive(Bound.END) && c.name().equals(sliceRestriction.bound(Bound.END).bindAndGet(variables)))
-                            continue;
-                    }
+        ResultSet cqlRows = result.build();
 
-                    result.newRow();
-                    // Respect selection order
-                    for (CFDefinition.Name name : selection.getColumnsList())
-                    {
-                        switch (name.kind)
-                        {
-                            case KEY_ALIAS:
-                                result.add(keyComponents[name.position]);
-                                break;
-                            case COLUMN_ALIAS:
-                                ByteBuffer val = cfDef.isComposite
-                                               ? (name.position < components.length ? components[name.position] : null)
-                                               : c.name();
-                                result.add(val);
-                                break;
-                            case VALUE_ALIAS:
-                                result.add(c);
-                                break;
-                            case COLUMN_METADATA:
-                                // This should not happen for compact CF
-                                throw new AssertionError();
-                            default:
-                                throw new AssertionError();
-                        }
-                    }
-                }
-            }
-            else if (cfDef.isComposite)
-            {
-                // Sparse case: group column in cqlRow when composite prefix is equal
-                CompositeType composite = (CompositeType)cfDef.cfm.comparator;
+        orderResults(cqlRows);
 
-                ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfDef.hasCollections, now);
+        // Internal calls always return columns in the comparator order, even when reverse was set
+        if (isReversed)
+            cqlRows.reverse();
 
-                for (Column c : row.cf)
-                {
-                    if (c.isMarkedForDelete(now))
-                        continue;
+        // Trim result if needed to respect the limit
+        cqlRows.trim(limit);
+        return cqlRows;
+    }
 
-                    builder.add(c);
-                }
+    // Used by ModificationStatement for CAS operations
+    void processColumnFamily(ByteBuffer key, ColumnFamily cf, List<ByteBuffer> variables, int limit, long now, Selection.ResultSetBuilder result) throws InvalidRequestException
+    {
+        ByteBuffer[] keyComponents = cfDef.hasCompositeKey
+                                   ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(key)
+                                   : new ByteBuffer[]{ key };
 
-                for (ColumnGroupMap group : builder.groups())
-                    handleGroup(selection, result, keyComponents, group);
-            }
-            else
+        if (cfDef.isCompact)
+        {
+            // One cqlRow per column
+            for (Column c : columnsInOrder(cf, variables))
             {
-                if (row.cf.hasOnlyTombstones(now))
+                if (c.isMarkedForDelete(now))
                     continue;
 
-                // Static case: One cqlRow for all columns
+                ByteBuffer[] components = null;
+                if (cfDef.isComposite)
+                {
+                    components = ((CompositeType)cfDef.cfm.comparator).split(c.name());
+                }
+                else if (sliceRestriction != null)
+                {
+                    // For dynamic CF, the column could be out of the requested bounds, filter here
+                    if (!sliceRestriction.isInclusive(Bound.START) && c.name().equals(sliceRestriction.bound(Bound.START).bindAndGet(variables)))
+                        continue;
+                    if (!sliceRestriction.isInclusive(Bound.END) && c.name().equals(sliceRestriction.bound(Bound.END).bindAndGet(variables)))
+                        continue;
+                }
+
                 result.newRow();
+                // Respect selection order
                 for (CFDefinition.Name name : selection.getColumnsList())
                 {
-                    if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
-                        result.add(keyComponents[name.position]);
-                    else
-                        result.add(row.cf.getColumn(name.name.key));
+                    switch (name.kind)
+                    {
+                        case KEY_ALIAS:
+                            result.add(keyComponents[name.position]);
+                            break;
+                        case COLUMN_ALIAS:
+                            ByteBuffer val = cfDef.isComposite
+                                           ? (name.position < components.length ? components[name.position] : null)
+                                           : c.name();
+                            result.add(val);
+                            break;
+                        case VALUE_ALIAS:
+                            result.add(c);
+                            break;
+                        case COLUMN_METADATA:
+                            // This should not happen for compact CF
+                            throw new AssertionError();
+                        default:
+                            throw new AssertionError();
+                    }
                 }
             }
         }
+        else if (cfDef.isComposite)
+        {
+            // Sparse case: group column in cqlRow when composite prefix is equal
+            CompositeType composite = (CompositeType)cfDef.cfm.comparator;
 
-        ResultSet cqlRows = result.build();
+            ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfDef.hasCollections, now);
 
-        orderResults(cqlRows);
+            for (Column c : cf)
+            {
+                if (c.isMarkedForDelete(now))
+                    continue;
 
-        // Internal calls always return columns in the comparator order, even when reverse was set
-        if (isReversed)
-            cqlRows.reverse();
+                builder.add(c);
+            }
 
-        // Trim result if needed to respect the limit
-        cqlRows.trim(limit);
-        return cqlRows;
+            for (ColumnGroupMap group : builder.groups())
+                handleGroup(selection, result, keyComponents, group);
+        }
+        else
+        {
+            if (cf.hasOnlyTombstones(now))
+                return;
+
+            // Static case: One cqlRow for all columns
+            result.newRow();
+            for (CFDefinition.Name name : selection.getColumnsList())
+            {
+                if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
+                    result.add(keyComponents[name.position]);
+                else
+                    result.add(cf.getColumn(name.name.key));
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index cf2b62e..af0804a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -57,6 +57,11 @@ public abstract class Selection
         return new SimpleSelection(all);
     }
 
+    public static Selection forColumns(List<CFDefinition.Name> columnsList)
+    {
+        return new SimpleSelection(columnsList);
+    }
+
     private static boolean isUsingFunction(List<RawSelector> rawSelectors)
     {
         for (RawSelector rawSelector : rawSelectors)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index eb3d908..612f89b 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.ColumnNameBuilder;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.filter.ColumnSlice;
@@ -159,7 +160,7 @@ public class StorageProxy implements StorageProxyMBean
 
     /**
      * Apply @param updates if and only if the current values in the row for @param key
-     * match the ones given by @param old.  The algorithm is "raw" Paxos: that is, Paxos
+     * match the ones given by @param expected.  The algorithm is "raw" Paxos: that is, Paxos
      * minus leader election -- any node in the cluster may propose changes for any row,
      * which (that is, the row) is the unit of values being proposed, not single columns.
      *
@@ -188,9 +189,21 @@ public class StorageProxy implements StorageProxyMBean
      *  values) between the prepare and accept phases.  This gives us a slightly longer window for another
      *  coordinator to come along and trump our own promise with a newer one but is otherwise safe.
      *
-     * @return true if the operation succeeds in updating the row
+     * @param table the table for the CAS
+     * @param cfName the column family for the CAS
+     * @param key the row key for the row to CAS
+     * @param prefix a column name prefix that selects the CQL3 row to check if {@code expected} is null. If {@code expected}
+     * is not null, this is ignored. If {@code expected} is null and this is null, the full row existing is checked (by querying
+     * the first live column of the row).
+     * @param expected the expected column values. This can be null to check for existence (see {@code prefix}).
+     * @param updates the value to insert if {@code expected matches the current values}.
+     * @param consistencyLevel the consistency for the operation.
+     *
+     * @return null if the operation succeeds in updating the row, or the current values for the columns contained in
+     * expected (since, if the CAS doesn't succeed, it means the current value do not match the one in expected). If
+     * expected == null and the CAS is unsuccessfull, the first live column of the CF is returned.
      */
-    public static boolean cas(String table, String cfName, ByteBuffer key, ColumnFamily expected, ColumnFamily updates, ConsistencyLevel consistencyLevel)
+    public static ColumnFamily cas(String table, String cfName, ByteBuffer key, ColumnNameBuilder prefix, ColumnFamily expected, ColumnFamily updates, ConsistencyLevel consistencyLevel)
     throws UnavailableException, IsBootstrappingException, ReadTimeoutException, WriteTimeoutException, InvalidRequestException
     {
         consistencyLevel.validateForCas(table);
@@ -213,18 +226,24 @@ public class StorageProxy implements StorageProxyMBean
             // read the current value and compare with expected
             Tracing.trace("Reading existing values for CAS precondition");
             long timestamp = System.currentTimeMillis();
-            IDiskAtomFilter filter = expected == null
-                                   ? new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1)
-                                   : new NamesQueryFilter(ImmutableSortedSet.copyOf(expected.getColumnNames()));
-            ReadCommand readCommand = filter instanceof SliceQueryFilter
-                                    ? new SliceFromReadCommand(table, key, cfName, timestamp, (SliceQueryFilter) filter)
-                                    : new SliceByNamesReadCommand(table, key, cfName, timestamp, (NamesQueryFilter) filter);
+            ReadCommand readCommand;
+            if (expected == null)
+            {
+                SliceQueryFilter filter = prefix == null
+                                        ? new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1)
+                                        : new SliceQueryFilter(prefix.build(), prefix.buildAsEndOfRange(), false, 1, prefix.componentCount());
+                readCommand = new SliceFromReadCommand(table, key, cfName, timestamp, filter);
+            }
+            else
+            {
+                readCommand = new SliceByNamesReadCommand(table, key, cfName, timestamp, new NamesQueryFilter(ImmutableSortedSet.copyOf(expected.getColumnNames())));
+            }
             List<Row> rows = read(Arrays.asList(readCommand), ConsistencyLevel.QUORUM);
             ColumnFamily current = rows.get(0).cf;
             if (!casApplies(expected, current))
             {
                 Tracing.trace("CAS precondition {} does not match current values {}", expected, current);
-                return false;
+                return current;
             }
 
             // finish the paxos round w/ the desired updates
@@ -238,7 +257,7 @@ public class StorageProxy implements StorageProxyMBean
                 else
                     commitPaxos(proposal, consistencyLevel);
                 Tracing.trace("CAS successful");
-                return true;
+                return null;
             }
 
             Tracing.trace("Paxos proposal not accepted (pre-empted by a higher ballot)");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 8f1fd21..4e71a84 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -160,6 +160,19 @@ public class CassandraServer implements Cassandra.Iface
         return thrift_column;
     }
 
+    private List<Column> thriftifyColumnsAsColumns(Collection<org.apache.cassandra.db.Column> columns, long now)
+    {
+        List<Column> thriftColumns = new ArrayList<Column>(columns.size());
+        for (org.apache.cassandra.db.Column column : columns)
+        {
+            if (column.isMarkedForDelete(now))
+                continue;
+
+            thriftColumns.add(thriftifySubColumn(column));
+        }
+        return thriftColumns;
+    }
+
     private CounterColumn thriftifySubCounter(org.apache.cassandra.db.Column column)
     {
         assert column instanceof org.apache.cassandra.db.CounterColumn;
@@ -690,7 +703,7 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    public boolean cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level)
+    public List<Column> cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (startSessionIfRequested())
@@ -746,7 +759,10 @@ public class CassandraServer implements Cassandra.Iface
             }
 
             schedule(DatabaseDescriptor.getWriteRpcTimeout());
-            return StorageProxy.cas(cState.getKeyspace(), column_family, key, cfExpected, cfUpdates, ThriftConversion.fromThrift(consistency_level));
+            ColumnFamily result = StorageProxy.cas(cState.getKeyspace(), column_family, key, null, cfExpected, cfUpdates, ThriftConversion.fromThrift(consistency_level));
+            return result == null
+                 ? null
+                 : thriftifyColumnsAsColumns(result.getSortedColumns(), System.currentTimeMillis());
         }
         catch (RequestTimeoutException e)
         {
@@ -759,7 +775,7 @@ public class CassandraServer implements Cassandra.Iface
         catch (RequestExecutionException e)
         {
             ThriftConversion.rethrow(e);
-            return false; // makes javac happy -- it can't tell that rethrow always throws
+            return null; // makes javac happy -- it can't tell that rethrow always throws
         }
         finally
         {


[2/2] git commit: Return current values on failed CAS

Posted by sl...@apache.org.
Return current values on failed CAS

patch by slebresne; reviewed by jbellis for CASSANDRA-5619


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

Branch: refs/heads/trunk
Commit: 0ba5cf12b9e1b7558ccf0c3a8bbd663f028024ce
Parents: 56d2296
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Jun 21 09:47:50 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Jun 21 09:47:50 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 interface/cassandra.thrift                      |  13 +-
 .../org/apache/cassandra/thrift/Cassandra.java  | 718 ++++++++++---------
 .../org/apache/cassandra/thrift/CfDef.java      |   4 +-
 .../cassandra/thrift/CounterSuperColumn.java    |   4 +-
 .../cassandra/thrift/CqlPreparedResult.java     |   8 +-
 .../org/apache/cassandra/thrift/CqlResult.java  |   4 +-
 .../org/apache/cassandra/thrift/CqlRow.java     |   4 +-
 .../apache/cassandra/thrift/IndexClause.java    |   4 +-
 .../org/apache/cassandra/thrift/KeyRange.java   |   4 +-
 .../org/apache/cassandra/thrift/KeySlice.java   |   4 +-
 .../org/apache/cassandra/thrift/KsDef.java      |   4 +-
 .../apache/cassandra/thrift/SlicePredicate.java |   4 +-
 .../apache/cassandra/thrift/SuperColumn.java    |   4 +-
 .../org/apache/cassandra/thrift/TokenRange.java |  12 +-
 .../cql3/statements/ModificationStatement.java  |  31 +-
 .../cql3/statements/SelectStatement.java        | 180 ++---
 .../cassandra/cql3/statements/Selection.java    |   5 +
 .../apache/cassandra/service/StorageProxy.java  |  41 +-
 .../cassandra/thrift/CassandraServer.java       |  22 +-
 20 files changed, 607 insertions(+), 465 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3c4bda9..ca27791 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,7 +9,7 @@
  * Removed compatibility with pre-1.2.5 sstables and network messages
    (CASSANDRA-5511)
  * removed PBSPredictor (CASSANDRA-5455)
- * CAS support (CASSANDRA-5062, 5441, 5442, 5443)
+ * CAS support (CASSANDRA-5062, 5441, 5442, 5443, 5619)
  * Leveled compaction performs size-tiered compactions in L0 
    (CASSANDRA-5371, 5439)
  * Add yaml network topology snitch for mixed ec2/other envs (CASSANDRA-5339)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index 2f50349..5038da1 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -646,12 +646,15 @@ service Cassandra {
 
   /**
    * Atomic compare and set
+   *
+   * The returned list of columns will be null if the cas succeed. Otherwise, it will contain the current
+   * values for the columns in {@param expected}.
    */
-  bool cas(1:required binary key, 
-           2:required string column_family,
-           3:list<Column> expected,
-           4:list<Column> updates,
-           5:required ConsistencyLevel consistency_level=ConsistencyLevel.QUORUM)
+  list<Column> cas(1:required binary key,
+                   2:required string column_family,
+                   3:list<Column> expected,
+                   4:list<Column> updates,
+                   5:required ConsistencyLevel consistency_level=ConsistencyLevel.QUORUM)
        throws (1:InvalidRequestException ire, 2:UnavailableException ue, 3:TimedOutException te),
 
   /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 27e01be..70deba9 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -166,13 +166,16 @@ public class Cassandra {
     /**
      * Atomic compare and set
      * 
+     * The returned list of columns will be null if the cas succeed. Otherwise, it will contain the current
+     * values for the columns in {@param expected}.
+     * 
      * @param key
      * @param column_family
      * @param expected
      * @param updates
      * @param consistency_level
      */
-    public boolean cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException;
+    public List<Column> cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException;
 
     /**
      * Remove data from the row specified by key at the granularity specified by column_path, and the given timestamp. Note
@@ -904,7 +907,7 @@ public class Cassandra {
       return;
     }
 
-    public boolean cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException
+    public List<Column> cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException
     {
       send_cas(key, column_family, expected, updates, consistency_level);
       return recv_cas();
@@ -921,7 +924,7 @@ public class Cassandra {
       sendBase("cas", args);
     }
 
-    public boolean recv_cas() throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException
+    public List<Column> recv_cas() throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException
     {
       cas_result result = new cas_result();
       receiveBase(result, "cas");
@@ -2310,7 +2313,7 @@ public class Cassandra {
         prot.writeMessageEnd();
       }
 
-      public boolean getResult() throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException {
+      public List<Column> getResult() throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -3728,7 +3731,6 @@ public class Cassandra {
         cas_result result = new cas_result();
         try {
           result.success = iface.cas(args.key, args.column_family, args.expected, args.updates, args.consistency_level);
-          result.setSuccessIsSet(true);
         } catch (InvalidRequestException ire) {
           result.ire = ire;
         } catch (UnavailableException ue) {
@@ -8694,7 +8696,7 @@ public class Cassandra {
                   struct.success = new ArrayList<ColumnOrSuperColumn>(_list210.size);
                   for (int _i211 = 0; _i211 < _list210.size; ++_i211)
                   {
-                    ColumnOrSuperColumn _elem212; // optional
+                    ColumnOrSuperColumn _elem212; // required
                     _elem212 = new ColumnOrSuperColumn();
                     _elem212.read(iprot);
                     struct.success.add(_elem212);
@@ -8836,7 +8838,7 @@ public class Cassandra {
             struct.success = new ArrayList<ColumnOrSuperColumn>(_list215.size);
             for (int _i216 = 0; _i216 < _list215.size; ++_i216)
             {
-              ColumnOrSuperColumn _elem217; // optional
+              ColumnOrSuperColumn _elem217; // required
               _elem217 = new ColumnOrSuperColumn();
               _elem217.read(iprot);
               struct.success.add(_elem217);
@@ -10848,7 +10850,7 @@ public class Cassandra {
                   struct.keys = new ArrayList<ByteBuffer>(_list218.size);
                   for (int _i219 = 0; _i219 < _list218.size; ++_i219)
                   {
-                    ByteBuffer _elem220; // optional
+                    ByteBuffer _elem220; // required
                     _elem220 = iprot.readBinary();
                     struct.keys.add(_elem220);
                   }
@@ -10964,7 +10966,7 @@ public class Cassandra {
           struct.keys = new ArrayList<ByteBuffer>(_list223.size);
           for (int _i224 = 0; _i224 < _list223.size; ++_i224)
           {
-            ByteBuffer _elem225; // optional
+            ByteBuffer _elem225; // required
             _elem225 = iprot.readBinary();
             struct.keys.add(_elem225);
           }
@@ -11567,7 +11569,7 @@ public class Cassandra {
                       _val229 = new ArrayList<ColumnOrSuperColumn>(_list230.size);
                       for (int _i231 = 0; _i231 < _list230.size; ++_i231)
                       {
-                        ColumnOrSuperColumn _elem232; // optional
+                        ColumnOrSuperColumn _elem232; // required
                         _elem232 = new ColumnOrSuperColumn();
                         _elem232.read(iprot);
                         _val229.add(_elem232);
@@ -11736,7 +11738,7 @@ public class Cassandra {
                 _val240 = new ArrayList<ColumnOrSuperColumn>(_list241.size);
                 for (int _i242 = 0; _i242 < _list241.size; ++_i242)
                 {
-                  ColumnOrSuperColumn _elem243; // optional
+                  ColumnOrSuperColumn _elem243; // required
                   _elem243 = new ColumnOrSuperColumn();
                   _elem243.read(iprot);
                   _val240.add(_elem243);
@@ -12372,7 +12374,7 @@ public class Cassandra {
                   struct.keys = new ArrayList<ByteBuffer>(_list244.size);
                   for (int _i245 = 0; _i245 < _list244.size; ++_i245)
                   {
-                    ByteBuffer _elem246; // optional
+                    ByteBuffer _elem246; // required
                     _elem246 = iprot.readBinary();
                     struct.keys.add(_elem246);
                   }
@@ -12488,7 +12490,7 @@ public class Cassandra {
           struct.keys = new ArrayList<ByteBuffer>(_list249.size);
           for (int _i250 = 0; _i250 < _list249.size; ++_i250)
           {
-            ByteBuffer _elem251; // optional
+            ByteBuffer _elem251; // required
             _elem251 = iprot.readBinary();
             struct.keys.add(_elem251);
           }
@@ -14510,7 +14512,7 @@ public class Cassandra {
                   struct.success = new ArrayList<KeySlice>(_list262.size);
                   for (int _i263 = 0; _i263 < _list262.size; ++_i263)
                   {
-                    KeySlice _elem264; // optional
+                    KeySlice _elem264; // required
                     _elem264 = new KeySlice();
                     _elem264.read(iprot);
                     struct.success.add(_elem264);
@@ -14652,7 +14654,7 @@ public class Cassandra {
             struct.success = new ArrayList<KeySlice>(_list267.size);
             for (int _i268 = 0; _i268 < _list267.size; ++_i268)
             {
-              KeySlice _elem269; // optional
+              KeySlice _elem269; // required
               _elem269 = new KeySlice();
               _elem269.read(iprot);
               struct.success.add(_elem269);
@@ -15938,7 +15940,7 @@ public class Cassandra {
                   struct.success = new ArrayList<KeySlice>(_list270.size);
                   for (int _i271 = 0; _i271 < _list270.size; ++_i271)
                   {
-                    KeySlice _elem272; // optional
+                    KeySlice _elem272; // required
                     _elem272 = new KeySlice();
                     _elem272.read(iprot);
                     struct.success.add(_elem272);
@@ -16080,7 +16082,7 @@ public class Cassandra {
             struct.success = new ArrayList<KeySlice>(_list275.size);
             for (int _i276 = 0; _i276 < _list275.size; ++_i276)
             {
-              KeySlice _elem277; // optional
+              KeySlice _elem277; // required
               _elem277 = new KeySlice();
               _elem277.read(iprot);
               struct.success.add(_elem277);
@@ -17365,7 +17367,7 @@ public class Cassandra {
                   struct.success = new ArrayList<KeySlice>(_list278.size);
                   for (int _i279 = 0; _i279 < _list278.size; ++_i279)
                   {
-                    KeySlice _elem280; // optional
+                    KeySlice _elem280; // required
                     _elem280 = new KeySlice();
                     _elem280.read(iprot);
                     struct.success.add(_elem280);
@@ -17507,7 +17509,7 @@ public class Cassandra {
             struct.success = new ArrayList<KeySlice>(_list283.size);
             for (int _i284 = 0; _i284 < _list283.size; ++_i284)
             {
-              KeySlice _elem285; // optional
+              KeySlice _elem285; // required
               _elem285 = new KeySlice();
               _elem285.read(iprot);
               struct.success.add(_elem285);
@@ -20806,7 +20808,7 @@ public class Cassandra {
                   struct.expected = new ArrayList<Column>(_list286.size);
                   for (int _i287 = 0; _i287 < _list286.size; ++_i287)
                   {
-                    Column _elem288; // optional
+                    Column _elem288; // required
                     _elem288 = new Column();
                     _elem288.read(iprot);
                     struct.expected.add(_elem288);
@@ -20825,7 +20827,7 @@ public class Cassandra {
                   struct.updates = new ArrayList<Column>(_list289.size);
                   for (int _i290 = 0; _i290 < _list289.size; ++_i290)
                   {
-                    Column _elem291; // optional
+                    Column _elem291; // required
                     _elem291 = new Column();
                     _elem291.read(iprot);
                     struct.updates.add(_elem291);
@@ -20963,7 +20965,7 @@ public class Cassandra {
             struct.expected = new ArrayList<Column>(_list296.size);
             for (int _i297 = 0; _i297 < _list296.size; ++_i297)
             {
-              Column _elem298; // optional
+              Column _elem298; // required
               _elem298 = new Column();
               _elem298.read(iprot);
               struct.expected.add(_elem298);
@@ -20977,7 +20979,7 @@ public class Cassandra {
             struct.updates = new ArrayList<Column>(_list299.size);
             for (int _i300 = 0; _i300 < _list299.size; ++_i300)
             {
-              Column _elem301; // optional
+              Column _elem301; // required
               _elem301 = new Column();
               _elem301.read(iprot);
               struct.updates.add(_elem301);
@@ -20993,7 +20995,7 @@ public class Cassandra {
   public static class cas_result implements org.apache.thrift.TBase<cas_result, cas_result._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cas_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
     private static final org.apache.thrift.protocol.TField IRE_FIELD_DESC = new org.apache.thrift.protocol.TField("ire", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField UE_FIELD_DESC = new org.apache.thrift.protocol.TField("ue", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField TE_FIELD_DESC = new org.apache.thrift.protocol.TField("te", org.apache.thrift.protocol.TType.STRUCT, (short)3);
@@ -21004,7 +21006,7 @@ public class Cassandra {
       schemes.put(TupleScheme.class, new cas_resultTupleSchemeFactory());
     }
 
-    public boolean success; // required
+    public List<Column> success; // required
     public InvalidRequestException ire; // required
     public UnavailableException ue; // required
     public TimedOutException te; // required
@@ -21077,13 +21079,12 @@ public class Cassandra {
     }
 
     // isset id assignments
-    private static final int __SUCCESS_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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+          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, Column.class))));
       tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -21098,14 +21099,13 @@ public class Cassandra {
     }
 
     public cas_result(
-      boolean success,
+      List<Column> success,
       InvalidRequestException ire,
       UnavailableException ue,
       TimedOutException te)
     {
       this();
       this.success = success;
-      setSuccessIsSet(true);
       this.ire = ire;
       this.ue = ue;
       this.te = te;
@@ -21115,8 +21115,13 @@ public class Cassandra {
      * Performs a deep copy on <i>other</i>.
      */
     public cas_result(cas_result other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.success = other.success;
+      if (other.isSetSuccess()) {
+        List<Column> __this__success = new ArrayList<Column>();
+        for (Column other_element : other.success) {
+          __this__success.add(new Column(other_element));
+        }
+        this.success = __this__success;
+      }
       if (other.isSetIre()) {
         this.ire = new InvalidRequestException(other.ire);
       }
@@ -21134,34 +21139,49 @@ public class Cassandra {
 
     @Override
     public void clear() {
-      setSuccessIsSet(false);
-      this.success = false;
+      this.success = null;
       this.ire = null;
       this.ue = null;
       this.te = null;
     }
 
-    public boolean isSuccess() {
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<Column> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(Column elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<Column>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<Column> getSuccess() {
       return this.success;
     }
 
-    public cas_result setSuccess(boolean success) {
+    public cas_result setSuccess(List<Column> success) {
       this.success = success;
-      setSuccessIsSet(true);
       return this;
     }
 
     public void unsetSuccess() {
-      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+      this.success = null;
     }
 
     /** Returns true if field success is set (has been assigned a value) and false otherwise */
     public boolean isSetSuccess() {
-      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+      return this.success != null;
     }
 
     public void setSuccessIsSet(boolean value) {
-      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+      if (!value) {
+        this.success = null;
+      }
     }
 
     public InvalidRequestException getIre() {
@@ -21242,7 +21262,7 @@ public class Cassandra {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((Boolean)value);
+          setSuccess((List<Column>)value);
         }
         break;
 
@@ -21276,7 +21296,7 @@ public class Cassandra {
     public Object getFieldValue(_Fields field) {
       switch (field) {
       case SUCCESS:
-        return Boolean.valueOf(isSuccess());
+        return getSuccess();
 
       case IRE:
         return getIre();
@@ -21323,12 +21343,12 @@ public class Cassandra {
       if (that == null)
         return false;
 
-      boolean this_present_success = true;
-      boolean that_present_success = true;
+      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 != that.success)
+        if (!this.success.equals(that.success))
           return false;
       }
 
@@ -21366,7 +21386,7 @@ public class Cassandra {
     public int hashCode() {
       HashCodeBuilder builder = new HashCodeBuilder();
 
-      boolean present_success = true;
+      boolean present_success = true && (isSetSuccess());
       builder.append(present_success);
       if (present_success)
         builder.append(success);
@@ -21458,7 +21478,11 @@ public class Cassandra {
       boolean first = true;
 
       sb.append("success:");
-      sb.append(this.success);
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
       first = false;
       if (!first) sb.append(", ");
       sb.append("ire:");
@@ -21503,8 +21527,6 @@ public class Cassandra {
 
     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);
@@ -21530,8 +21552,19 @@ public class Cassandra {
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.success = iprot.readBool();
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list302 = iprot.readListBegin();
+                  struct.success = new ArrayList<Column>(_list302.size);
+                  for (int _i303 = 0; _i303 < _list302.size; ++_i303)
+                  {
+                    Column _elem304; // required
+                    _elem304 = new Column();
+                    _elem304.read(iprot);
+                    struct.success.add(_elem304);
+                  }
+                  iprot.readListEnd();
+                }
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -21579,9 +21612,16 @@ public class Cassandra {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
+        if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeBool(struct.success);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (Column _iter305 : struct.success)
+            {
+              _iter305.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
         if (struct.ire != null) {
@@ -21631,7 +21671,13 @@ public class Cassandra {
         }
         oprot.writeBitSet(optionals, 4);
         if (struct.isSetSuccess()) {
-          oprot.writeBool(struct.success);
+          {
+            oprot.writeI32(struct.success.size());
+            for (Column _iter306 : struct.success)
+            {
+              _iter306.write(oprot);
+            }
+          }
         }
         if (struct.isSetIre()) {
           struct.ire.write(oprot);
@@ -21649,7 +21695,17 @@ public class Cassandra {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
-          struct.success = iprot.readBool();
+          {
+            org.apache.thrift.protocol.TList _list307 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Column>(_list307.size);
+            for (int _i308 = 0; _i308 < _list307.size; ++_i308)
+            {
+              Column _elem309; // required
+              _elem309 = new Column();
+              _elem309.read(iprot);
+              struct.success.add(_elem309);
+            }
+          }
           struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
@@ -24561,38 +24617,38 @@ public class Cassandra {
             case 1: // MUTATION_MAP
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map302 = iprot.readMapBegin();
-                  struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map302.size);
-                  for (int _i303 = 0; _i303 < _map302.size; ++_i303)
+                  org.apache.thrift.protocol.TMap _map310 = iprot.readMapBegin();
+                  struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map310.size);
+                  for (int _i311 = 0; _i311 < _map310.size; ++_i311)
                   {
-                    ByteBuffer _key304; // required
-                    Map<String,List<Mutation>> _val305; // required
-                    _key304 = iprot.readBinary();
+                    ByteBuffer _key312; // required
+                    Map<String,List<Mutation>> _val313; // required
+                    _key312 = iprot.readBinary();
                     {
-                      org.apache.thrift.protocol.TMap _map306 = iprot.readMapBegin();
-                      _val305 = new HashMap<String,List<Mutation>>(2*_map306.size);
-                      for (int _i307 = 0; _i307 < _map306.size; ++_i307)
+                      org.apache.thrift.protocol.TMap _map314 = iprot.readMapBegin();
+                      _val313 = new HashMap<String,List<Mutation>>(2*_map314.size);
+                      for (int _i315 = 0; _i315 < _map314.size; ++_i315)
                       {
-                        String _key308; // required
-                        List<Mutation> _val309; // required
-                        _key308 = iprot.readString();
+                        String _key316; // required
+                        List<Mutation> _val317; // required
+                        _key316 = iprot.readString();
                         {
-                          org.apache.thrift.protocol.TList _list310 = iprot.readListBegin();
-                          _val309 = new ArrayList<Mutation>(_list310.size);
-                          for (int _i311 = 0; _i311 < _list310.size; ++_i311)
+                          org.apache.thrift.protocol.TList _list318 = iprot.readListBegin();
+                          _val317 = new ArrayList<Mutation>(_list318.size);
+                          for (int _i319 = 0; _i319 < _list318.size; ++_i319)
                           {
-                            Mutation _elem312; // optional
-                            _elem312 = new Mutation();
-                            _elem312.read(iprot);
-                            _val309.add(_elem312);
+                            Mutation _elem320; // required
+                            _elem320 = new Mutation();
+                            _elem320.read(iprot);
+                            _val317.add(_elem320);
                           }
                           iprot.readListEnd();
                         }
-                        _val305.put(_key308, _val309);
+                        _val313.put(_key316, _val317);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.mutation_map.put(_key304, _val305);
+                    struct.mutation_map.put(_key312, _val313);
                   }
                   iprot.readMapEnd();
                 }
@@ -24628,19 +24684,19 @@ public class Cassandra {
           oprot.writeFieldBegin(MUTATION_MAP_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.mutation_map.size()));
-            for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter313 : struct.mutation_map.entrySet())
+            for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter321 : struct.mutation_map.entrySet())
             {
-              oprot.writeBinary(_iter313.getKey());
+              oprot.writeBinary(_iter321.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, _iter313.getValue().size()));
-                for (Map.Entry<String, List<Mutation>> _iter314 : _iter313.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, _iter321.getValue().size()));
+                for (Map.Entry<String, List<Mutation>> _iter322 : _iter321.getValue().entrySet())
                 {
-                  oprot.writeString(_iter314.getKey());
+                  oprot.writeString(_iter322.getKey());
                   {
-                    oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter314.getValue().size()));
-                    for (Mutation _iter315 : _iter314.getValue())
+                    oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter322.getValue().size()));
+                    for (Mutation _iter323 : _iter322.getValue())
                     {
-                      _iter315.write(oprot);
+                      _iter323.write(oprot);
                     }
                     oprot.writeListEnd();
                   }
@@ -24676,19 +24732,19 @@ public class Cassandra {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         {
           oprot.writeI32(struct.mutation_map.size());
-          for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter316 : struct.mutation_map.entrySet())
+          for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter324 : struct.mutation_map.entrySet())
           {
-            oprot.writeBinary(_iter316.getKey());
+            oprot.writeBinary(_iter324.getKey());
             {
-              oprot.writeI32(_iter316.getValue().size());
-              for (Map.Entry<String, List<Mutation>> _iter317 : _iter316.getValue().entrySet())
+              oprot.writeI32(_iter324.getValue().size());
+              for (Map.Entry<String, List<Mutation>> _iter325 : _iter324.getValue().entrySet())
               {
-                oprot.writeString(_iter317.getKey());
+                oprot.writeString(_iter325.getKey());
                 {
-                  oprot.writeI32(_iter317.getValue().size());
-                  for (Mutation _iter318 : _iter317.getValue())
+                  oprot.writeI32(_iter325.getValue().size());
+                  for (Mutation _iter326 : _iter325.getValue())
                   {
-                    _iter318.write(oprot);
+                    _iter326.write(oprot);
                   }
                 }
               }
@@ -24702,36 +24758,36 @@ public class Cassandra {
       public void read(org.apache.thrift.protocol.TProtocol prot, batch_mutate_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         {
-          org.apache.thrift.protocol.TMap _map319 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-          struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map319.size);
-          for (int _i320 = 0; _i320 < _map319.size; ++_i320)
+          org.apache.thrift.protocol.TMap _map327 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+          struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map327.size);
+          for (int _i328 = 0; _i328 < _map327.size; ++_i328)
           {
-            ByteBuffer _key321; // required
-            Map<String,List<Mutation>> _val322; // required
-            _key321 = iprot.readBinary();
+            ByteBuffer _key329; // required
+            Map<String,List<Mutation>> _val330; // required
+            _key329 = iprot.readBinary();
             {
-              org.apache.thrift.protocol.TMap _map323 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-              _val322 = new HashMap<String,List<Mutation>>(2*_map323.size);
-              for (int _i324 = 0; _i324 < _map323.size; ++_i324)
+              org.apache.thrift.protocol.TMap _map331 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+              _val330 = new HashMap<String,List<Mutation>>(2*_map331.size);
+              for (int _i332 = 0; _i332 < _map331.size; ++_i332)
               {
-                String _key325; // required
-                List<Mutation> _val326; // required
-                _key325 = iprot.readString();
+                String _key333; // required
+                List<Mutation> _val334; // required
+                _key333 = iprot.readString();
                 {
-                  org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                  _val326 = new ArrayList<Mutation>(_list327.size);
-                  for (int _i328 = 0; _i328 < _list327.size; ++_i328)
+                  org.apache.thrift.protocol.TList _list335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                  _val334 = new ArrayList<Mutation>(_list335.size);
+                  for (int _i336 = 0; _i336 < _list335.size; ++_i336)
                   {
-                    Mutation _elem329; // optional
-                    _elem329 = new Mutation();
-                    _elem329.read(iprot);
-                    _val326.add(_elem329);
+                    Mutation _elem337; // required
+                    _elem337 = new Mutation();
+                    _elem337.read(iprot);
+                    _val334.add(_elem337);
                   }
                 }
-                _val322.put(_key325, _val326);
+                _val330.put(_key333, _val334);
               }
             }
-            struct.mutation_map.put(_key321, _val322);
+            struct.mutation_map.put(_key329, _val330);
           }
         }
         struct.setMutation_mapIsSet(true);
@@ -25765,38 +25821,38 @@ public class Cassandra {
             case 1: // MUTATION_MAP
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map330 = iprot.readMapBegin();
-                  struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map330.size);
-                  for (int _i331 = 0; _i331 < _map330.size; ++_i331)
+                  org.apache.thrift.protocol.TMap _map338 = iprot.readMapBegin();
+                  struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map338.size);
+                  for (int _i339 = 0; _i339 < _map338.size; ++_i339)
                   {
-                    ByteBuffer _key332; // required
-                    Map<String,List<Mutation>> _val333; // required
-                    _key332 = iprot.readBinary();
+                    ByteBuffer _key340; // required
+                    Map<String,List<Mutation>> _val341; // required
+                    _key340 = iprot.readBinary();
                     {
-                      org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin();
-                      _val333 = new HashMap<String,List<Mutation>>(2*_map334.size);
-                      for (int _i335 = 0; _i335 < _map334.size; ++_i335)
+                      org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
+                      _val341 = new HashMap<String,List<Mutation>>(2*_map342.size);
+                      for (int _i343 = 0; _i343 < _map342.size; ++_i343)
                       {
-                        String _key336; // required
-                        List<Mutation> _val337; // required
-                        _key336 = iprot.readString();
+                        String _key344; // required
+                        List<Mutation> _val345; // required
+                        _key344 = iprot.readString();
                         {
-                          org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
-                          _val337 = new ArrayList<Mutation>(_list338.size);
-                          for (int _i339 = 0; _i339 < _list338.size; ++_i339)
+                          org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                          _val345 = new ArrayList<Mutation>(_list346.size);
+                          for (int _i347 = 0; _i347 < _list346.size; ++_i347)
                           {
-                            Mutation _elem340; // optional
-                            _elem340 = new Mutation();
-                            _elem340.read(iprot);
-                            _val337.add(_elem340);
+                            Mutation _elem348; // required
+                            _elem348 = new Mutation();
+                            _elem348.read(iprot);
+                            _val345.add(_elem348);
                           }
                           iprot.readListEnd();
                         }
-                        _val333.put(_key336, _val337);
+                        _val341.put(_key344, _val345);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.mutation_map.put(_key332, _val333);
+                    struct.mutation_map.put(_key340, _val341);
                   }
                   iprot.readMapEnd();
                 }
@@ -25832,19 +25888,19 @@ public class Cassandra {
           oprot.writeFieldBegin(MUTATION_MAP_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.mutation_map.size()));
-            for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter341 : struct.mutation_map.entrySet())
+            for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter349 : struct.mutation_map.entrySet())
             {
-              oprot.writeBinary(_iter341.getKey());
+              oprot.writeBinary(_iter349.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, _iter341.getValue().size()));
-                for (Map.Entry<String, List<Mutation>> _iter342 : _iter341.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, _iter349.getValue().size()));
+                for (Map.Entry<String, List<Mutation>> _iter350 : _iter349.getValue().entrySet())
                 {
-                  oprot.writeString(_iter342.getKey());
+                  oprot.writeString(_iter350.getKey());
                   {
-                    oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter342.getValue().size()));
-                    for (Mutation _iter343 : _iter342.getValue())
+                    oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter350.getValue().size()));
+                    for (Mutation _iter351 : _iter350.getValue())
                     {
-                      _iter343.write(oprot);
+                      _iter351.write(oprot);
                     }
                     oprot.writeListEnd();
                   }
@@ -25880,19 +25936,19 @@ public class Cassandra {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         {
           oprot.writeI32(struct.mutation_map.size());
-          for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter344 : struct.mutation_map.entrySet())
+          for (Map.Entry<ByteBuffer, Map<String,List<Mutation>>> _iter352 : struct.mutation_map.entrySet())
           {
-            oprot.writeBinary(_iter344.getKey());
+            oprot.writeBinary(_iter352.getKey());
             {
-              oprot.writeI32(_iter344.getValue().size());
-              for (Map.Entry<String, List<Mutation>> _iter345 : _iter344.getValue().entrySet())
+              oprot.writeI32(_iter352.getValue().size());
+              for (Map.Entry<String, List<Mutation>> _iter353 : _iter352.getValue().entrySet())
               {
-                oprot.writeString(_iter345.getKey());
+                oprot.writeString(_iter353.getKey());
                 {
-                  oprot.writeI32(_iter345.getValue().size());
-                  for (Mutation _iter346 : _iter345.getValue())
+                  oprot.writeI32(_iter353.getValue().size());
+                  for (Mutation _iter354 : _iter353.getValue())
                   {
-                    _iter346.write(oprot);
+                    _iter354.write(oprot);
                   }
                 }
               }
@@ -25906,36 +25962,36 @@ public class Cassandra {
       public void read(org.apache.thrift.protocol.TProtocol prot, atomic_batch_mutate_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         {
-          org.apache.thrift.protocol.TMap _map347 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-          struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map347.size);
-          for (int _i348 = 0; _i348 < _map347.size; ++_i348)
+          org.apache.thrift.protocol.TMap _map355 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+          struct.mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>(2*_map355.size);
+          for (int _i356 = 0; _i356 < _map355.size; ++_i356)
           {
-            ByteBuffer _key349; // required
-            Map<String,List<Mutation>> _val350; // required
-            _key349 = iprot.readBinary();
+            ByteBuffer _key357; // required
+            Map<String,List<Mutation>> _val358; // required
+            _key357 = iprot.readBinary();
             {
-              org.apache.thrift.protocol.TMap _map351 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-              _val350 = new HashMap<String,List<Mutation>>(2*_map351.size);
-              for (int _i352 = 0; _i352 < _map351.size; ++_i352)
+              org.apache.thrift.protocol.TMap _map359 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+              _val358 = new HashMap<String,List<Mutation>>(2*_map359.size);
+              for (int _i360 = 0; _i360 < _map359.size; ++_i360)
               {
-                String _key353; // required
-                List<Mutation> _val354; // required
-                _key353 = iprot.readString();
+                String _key361; // required
+                List<Mutation> _val362; // required
+                _key361 = iprot.readString();
                 {
-                  org.apache.thrift.protocol.TList _list355 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-                  _val354 = new ArrayList<Mutation>(_list355.size);
-                  for (int _i356 = 0; _i356 < _list355.size; ++_i356)
+                  org.apache.thrift.protocol.TList _list363 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+                  _val362 = new ArrayList<Mutation>(_list363.size);
+                  for (int _i364 = 0; _i364 < _list363.size; ++_i364)
                   {
-                    Mutation _elem357; // optional
-                    _elem357 = new Mutation();
-                    _elem357.read(iprot);
-                    _val354.add(_elem357);
+                    Mutation _elem365; // required
+                    _elem365 = new Mutation();
+                    _elem365.read(iprot);
+                    _val362.add(_elem365);
                   }
                 }
-                _val350.put(_key353, _val354);
+                _val358.put(_key361, _val362);
               }
             }
-            struct.mutation_map.put(_key349, _val350);
+            struct.mutation_map.put(_key357, _val358);
           }
         }
         struct.setMutation_mapIsSet(true);
@@ -28109,25 +28165,25 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map358 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,List<String>>(2*_map358.size);
-                  for (int _i359 = 0; _i359 < _map358.size; ++_i359)
+                  org.apache.thrift.protocol.TMap _map366 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,List<String>>(2*_map366.size);
+                  for (int _i367 = 0; _i367 < _map366.size; ++_i367)
                   {
-                    String _key360; // required
-                    List<String> _val361; // required
-                    _key360 = iprot.readString();
+                    String _key368; // required
+                    List<String> _val369; // required
+                    _key368 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
-                      _val361 = new ArrayList<String>(_list362.size);
-                      for (int _i363 = 0; _i363 < _list362.size; ++_i363)
+                      org.apache.thrift.protocol.TList _list370 = iprot.readListBegin();
+                      _val369 = new ArrayList<String>(_list370.size);
+                      for (int _i371 = 0; _i371 < _list370.size; ++_i371)
                       {
-                        String _elem364; // optional
-                        _elem364 = iprot.readString();
-                        _val361.add(_elem364);
+                        String _elem372; // required
+                        _elem372 = iprot.readString();
+                        _val369.add(_elem372);
                       }
                       iprot.readListEnd();
                     }
-                    struct.success.put(_key360, _val361);
+                    struct.success.put(_key368, _val369);
                   }
                   iprot.readMapEnd();
                 }
@@ -28164,14 +28220,14 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.success.size()));
-            for (Map.Entry<String, List<String>> _iter365 : struct.success.entrySet())
+            for (Map.Entry<String, List<String>> _iter373 : struct.success.entrySet())
             {
-              oprot.writeString(_iter365.getKey());
+              oprot.writeString(_iter373.getKey());
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter365.getValue().size()));
-                for (String _iter366 : _iter365.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter373.getValue().size()));
+                for (String _iter374 : _iter373.getValue())
                 {
-                  oprot.writeString(_iter366);
+                  oprot.writeString(_iter374);
                 }
                 oprot.writeListEnd();
               }
@@ -28213,14 +28269,14 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, List<String>> _iter367 : struct.success.entrySet())
+            for (Map.Entry<String, List<String>> _iter375 : struct.success.entrySet())
             {
-              oprot.writeString(_iter367.getKey());
+              oprot.writeString(_iter375.getKey());
               {
-                oprot.writeI32(_iter367.getValue().size());
-                for (String _iter368 : _iter367.getValue())
+                oprot.writeI32(_iter375.getValue().size());
+                for (String _iter376 : _iter375.getValue())
                 {
-                  oprot.writeString(_iter368);
+                  oprot.writeString(_iter376);
                 }
               }
             }
@@ -28237,24 +28293,24 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map369 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-            struct.success = new HashMap<String,List<String>>(2*_map369.size);
-            for (int _i370 = 0; _i370 < _map369.size; ++_i370)
+            org.apache.thrift.protocol.TMap _map377 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+            struct.success = new HashMap<String,List<String>>(2*_map377.size);
+            for (int _i378 = 0; _i378 < _map377.size; ++_i378)
             {
-              String _key371; // required
-              List<String> _val372; // required
-              _key371 = iprot.readString();
+              String _key379; // required
+              List<String> _val380; // required
+              _key379 = iprot.readString();
               {
-                org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-                _val372 = new ArrayList<String>(_list373.size);
-                for (int _i374 = 0; _i374 < _list373.size; ++_i374)
+                org.apache.thrift.protocol.TList _list381 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+                _val380 = new ArrayList<String>(_list381.size);
+                for (int _i382 = 0; _i382 < _list381.size; ++_i382)
                 {
-                  String _elem375; // optional
-                  _elem375 = iprot.readString();
-                  _val372.add(_elem375);
+                  String _elem383; // required
+                  _elem383 = iprot.readString();
+                  _val380.add(_elem383);
                 }
               }
-              struct.success.put(_key371, _val372);
+              struct.success.put(_key379, _val380);
             }
           }
           struct.setSuccessIsSet(true);
@@ -28915,14 +28971,14 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list376 = iprot.readListBegin();
-                  struct.success = new ArrayList<KsDef>(_list376.size);
-                  for (int _i377 = 0; _i377 < _list376.size; ++_i377)
+                  org.apache.thrift.protocol.TList _list384 = iprot.readListBegin();
+                  struct.success = new ArrayList<KsDef>(_list384.size);
+                  for (int _i385 = 0; _i385 < _list384.size; ++_i385)
                   {
-                    KsDef _elem378; // optional
-                    _elem378 = new KsDef();
-                    _elem378.read(iprot);
-                    struct.success.add(_elem378);
+                    KsDef _elem386; // required
+                    _elem386 = new KsDef();
+                    _elem386.read(iprot);
+                    struct.success.add(_elem386);
                   }
                   iprot.readListEnd();
                 }
@@ -28959,9 +29015,9 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (KsDef _iter379 : struct.success)
+            for (KsDef _iter387 : struct.success)
             {
-              _iter379.write(oprot);
+              _iter387.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -29000,9 +29056,9 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (KsDef _iter380 : struct.success)
+            for (KsDef _iter388 : struct.success)
             {
-              _iter380.write(oprot);
+              _iter388.write(oprot);
             }
           }
         }
@@ -29017,14 +29073,14 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list381 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<KsDef>(_list381.size);
-            for (int _i382 = 0; _i382 < _list381.size; ++_i382)
+            org.apache.thrift.protocol.TList _list389 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<KsDef>(_list389.size);
+            for (int _i390 = 0; _i390 < _list389.size; ++_i390)
             {
-              KsDef _elem383; // optional
-              _elem383 = new KsDef();
-              _elem383.read(iprot);
-              struct.success.add(_elem383);
+              KsDef _elem391; // required
+              _elem391 = new KsDef();
+              _elem391.read(iprot);
+              struct.success.add(_elem391);
             }
           }
           struct.setSuccessIsSet(true);
@@ -31009,14 +31065,14 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list384 = iprot.readListBegin();
-                  struct.success = new ArrayList<TokenRange>(_list384.size);
-                  for (int _i385 = 0; _i385 < _list384.size; ++_i385)
+                  org.apache.thrift.protocol.TList _list392 = iprot.readListBegin();
+                  struct.success = new ArrayList<TokenRange>(_list392.size);
+                  for (int _i393 = 0; _i393 < _list392.size; ++_i393)
                   {
-                    TokenRange _elem386; // optional
-                    _elem386 = new TokenRange();
-                    _elem386.read(iprot);
-                    struct.success.add(_elem386);
+                    TokenRange _elem394; // required
+                    _elem394 = new TokenRange();
+                    _elem394.read(iprot);
+                    struct.success.add(_elem394);
                   }
                   iprot.readListEnd();
                 }
@@ -31053,9 +31109,9 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TokenRange _iter387 : struct.success)
+            for (TokenRange _iter395 : struct.success)
             {
-              _iter387.write(oprot);
+              _iter395.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -31094,9 +31150,9 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TokenRange _iter388 : struct.success)
+            for (TokenRange _iter396 : struct.success)
             {
-              _iter388.write(oprot);
+              _iter396.write(oprot);
             }
           }
         }
@@ -31111,14 +31167,14 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list389 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TokenRange>(_list389.size);
-            for (int _i390 = 0; _i390 < _list389.size; ++_i390)
+            org.apache.thrift.protocol.TList _list397 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TokenRange>(_list397.size);
+            for (int _i398 = 0; _i398 < _list397.size; ++_i398)
             {
-              TokenRange _elem391; // optional
-              _elem391 = new TokenRange();
-              _elem391.read(iprot);
-              struct.success.add(_elem391);
+              TokenRange _elem399; // required
+              _elem399 = new TokenRange();
+              _elem399.read(iprot);
+              struct.success.add(_elem399);
             }
           }
           struct.setSuccessIsSet(true);
@@ -31784,15 +31840,15 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map392 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,String>(2*_map392.size);
-                  for (int _i393 = 0; _i393 < _map392.size; ++_i393)
+                  org.apache.thrift.protocol.TMap _map400 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,String>(2*_map400.size);
+                  for (int _i401 = 0; _i401 < _map400.size; ++_i401)
                   {
-                    String _key394; // required
-                    String _val395; // required
-                    _key394 = iprot.readString();
-                    _val395 = iprot.readString();
-                    struct.success.put(_key394, _val395);
+                    String _key402; // required
+                    String _val403; // required
+                    _key402 = iprot.readString();
+                    _val403 = iprot.readString();
+                    struct.success.put(_key402, _val403);
                   }
                   iprot.readMapEnd();
                 }
@@ -31829,10 +31885,10 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (Map.Entry<String, String> _iter396 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter404 : struct.success.entrySet())
             {
-              oprot.writeString(_iter396.getKey());
-              oprot.writeString(_iter396.getValue());
+              oprot.writeString(_iter404.getKey());
+              oprot.writeString(_iter404.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -31871,10 +31927,10 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, String> _iter397 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter405 : struct.success.entrySet())
             {
-              oprot.writeString(_iter397.getKey());
-              oprot.writeString(_iter397.getValue());
+              oprot.writeString(_iter405.getKey());
+              oprot.writeString(_iter405.getValue());
             }
           }
         }
@@ -31889,15 +31945,15 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map398 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new HashMap<String,String>(2*_map398.size);
-            for (int _i399 = 0; _i399 < _map398.size; ++_i399)
+            org.apache.thrift.protocol.TMap _map406 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new HashMap<String,String>(2*_map406.size);
+            for (int _i407 = 0; _i407 < _map406.size; ++_i407)
             {
-              String _key400; // required
-              String _val401; // required
-              _key400 = iprot.readString();
-              _val401 = iprot.readString();
-              struct.success.put(_key400, _val401);
+              String _key408; // required
+              String _val409; // required
+              _key408 = iprot.readString();
+              _val409 = iprot.readString();
+              struct.success.put(_key408, _val409);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35118,13 +35174,13 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list402 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list402.size);
-                  for (int _i403 = 0; _i403 < _list402.size; ++_i403)
+                  org.apache.thrift.protocol.TList _list410 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list410.size);
+                  for (int _i411 = 0; _i411 < _list410.size; ++_i411)
                   {
-                    String _elem404; // optional
-                    _elem404 = iprot.readString();
-                    struct.success.add(_elem404);
+                    String _elem412; // required
+                    _elem412 = iprot.readString();
+                    struct.success.add(_elem412);
                   }
                   iprot.readListEnd();
                 }
@@ -35161,9 +35217,9 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter405 : struct.success)
+            for (String _iter413 : struct.success)
             {
-              oprot.writeString(_iter405);
+              oprot.writeString(_iter413);
             }
             oprot.writeListEnd();
           }
@@ -35202,9 +35258,9 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter406 : struct.success)
+            for (String _iter414 : struct.success)
             {
-              oprot.writeString(_iter406);
+              oprot.writeString(_iter414);
             }
           }
         }
@@ -35219,13 +35275,13 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list407.size);
-            for (int _i408 = 0; _i408 < _list407.size; ++_i408)
+            org.apache.thrift.protocol.TList _list415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list415.size);
+            for (int _i416 = 0; _i416 < _list415.size; ++_i416)
             {
-              String _elem409; // optional
-              _elem409 = iprot.readString();
-              struct.success.add(_elem409);
+              String _elem417; // required
+              _elem417 = iprot.readString();
+              struct.success.add(_elem417);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36914,14 +36970,14 @@ public class Cassandra {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list410 = iprot.readListBegin();
-                  struct.success = new ArrayList<CfSplit>(_list410.size);
-                  for (int _i411 = 0; _i411 < _list410.size; ++_i411)
+                  org.apache.thrift.protocol.TList _list418 = iprot.readListBegin();
+                  struct.success = new ArrayList<CfSplit>(_list418.size);
+                  for (int _i419 = 0; _i419 < _list418.size; ++_i419)
                   {
-                    CfSplit _elem412; // optional
-                    _elem412 = new CfSplit();
-                    _elem412.read(iprot);
-                    struct.success.add(_elem412);
+                    CfSplit _elem420; // required
+                    _elem420 = new CfSplit();
+                    _elem420.read(iprot);
+                    struct.success.add(_elem420);
                   }
                   iprot.readListEnd();
                 }
@@ -36958,9 +37014,9 @@ public class Cassandra {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (CfSplit _iter413 : struct.success)
+            for (CfSplit _iter421 : struct.success)
             {
-              _iter413.write(oprot);
+              _iter421.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36999,9 +37055,9 @@ public class Cassandra {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (CfSplit _iter414 : struct.success)
+            for (CfSplit _iter422 : struct.success)
             {
-              _iter414.write(oprot);
+              _iter422.write(oprot);
             }
           }
         }
@@ -37016,14 +37072,14 @@ public class Cassandra {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<CfSplit>(_list415.size);
-            for (int _i416 = 0; _i416 < _list415.size; ++_i416)
+            org.apache.thrift.protocol.TList _list423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<CfSplit>(_list423.size);
+            for (int _i424 = 0; _i424 < _list423.size; ++_i424)
             {
-              CfSplit _elem417; // optional
-              _elem417 = new CfSplit();
-              _elem417.read(iprot);
-              struct.success.add(_elem417);
+              CfSplit _elem425; // required
+              _elem425 = new CfSplit();
+              _elem425.read(iprot);
+              struct.success.add(_elem425);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47623,13 +47679,13 @@ public class Cassandra {
             case 2: // VALUES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list418 = iprot.readListBegin();
-                  struct.values = new ArrayList<ByteBuffer>(_list418.size);
-                  for (int _i419 = 0; _i419 < _list418.size; ++_i419)
+                  org.apache.thrift.protocol.TList _list426 = iprot.readListBegin();
+                  struct.values = new ArrayList<ByteBuffer>(_list426.size);
+                  for (int _i427 = 0; _i427 < _list426.size; ++_i427)
                   {
-                    ByteBuffer _elem420; // optional
-                    _elem420 = iprot.readBinary();
-                    struct.values.add(_elem420);
+                    ByteBuffer _elem428; // required
+                    _elem428 = iprot.readBinary();
+                    struct.values.add(_elem428);
                   }
                   iprot.readListEnd();
                 }
@@ -47663,9 +47719,9 @@ public class Cassandra {
           oprot.writeFieldBegin(VALUES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-            for (ByteBuffer _iter421 : struct.values)
+            for (ByteBuffer _iter429 : struct.values)
             {
-              oprot.writeBinary(_iter421);
+              oprot.writeBinary(_iter429);
             }
             oprot.writeListEnd();
           }
@@ -47691,9 +47747,9 @@ public class Cassandra {
         oprot.writeI32(struct.itemId);
         {
           oprot.writeI32(struct.values.size());
-          for (ByteBuffer _iter422 : struct.values)
+          for (ByteBuffer _iter430 : struct.values)
           {
-            oprot.writeBinary(_iter422);
+            oprot.writeBinary(_iter430);
           }
         }
       }
@@ -47704,13 +47760,13 @@ public class Cassandra {
         struct.itemId = iprot.readI32();
         struct.setItemIdIsSet(true);
         {
-          org.apache.thrift.protocol.TList _list423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<ByteBuffer>(_list423.size);
-          for (int _i424 = 0; _i424 < _list423.size; ++_i424)
+          org.apache.thrift.protocol.TList _list431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<ByteBuffer>(_list431.size);
+          for (int _i432 = 0; _i432 < _list431.size; ++_i432)
           {
-            ByteBuffer _elem425; // optional
-            _elem425 = iprot.readBinary();
-            struct.values.add(_elem425);
+            ByteBuffer _elem433; // required
+            _elem433 = iprot.readBinary();
+            struct.values.add(_elem433);
           }
         }
         struct.setValuesIsSet(true);
@@ -49026,13 +49082,13 @@ public class Cassandra {
             case 2: // VALUES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list426 = iprot.readListBegin();
-                  struct.values = new ArrayList<ByteBuffer>(_list426.size);
-                  for (int _i427 = 0; _i427 < _list426.size; ++_i427)
+                  org.apache.thrift.protocol.TList _list434 = iprot.readListBegin();
+                  struct.values = new ArrayList<ByteBuffer>(_list434.size);
+                  for (int _i435 = 0; _i435 < _list434.size; ++_i435)
                   {
-                    ByteBuffer _elem428; // optional
-                    _elem428 = iprot.readBinary();
-                    struct.values.add(_elem428);
+                    ByteBuffer _elem436; // required
+                    _elem436 = iprot.readBinary();
+                    struct.values.add(_elem436);
                   }
                   iprot.readListEnd();
                 }
@@ -49074,9 +49130,9 @@ public class Cassandra {
           oprot.writeFieldBegin(VALUES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-            for (ByteBuffer _iter429 : struct.values)
+            for (ByteBuffer _iter437 : struct.values)
             {
-              oprot.writeBinary(_iter429);
+              oprot.writeBinary(_iter437);
             }
             oprot.writeListEnd();
           }
@@ -49107,9 +49163,9 @@ public class Cassandra {
         oprot.writeI32(struct.itemId);
         {
           oprot.writeI32(struct.values.size());
-          for (ByteBuffer _iter430 : struct.values)
+          for (ByteBuffer _iter438 : struct.values)
           {
-            oprot.writeBinary(_iter430);
+            oprot.writeBinary(_iter438);
           }
         }
         oprot.writeI32(struct.consistency.getValue());
@@ -49121,13 +49177,13 @@ public class Cassandra {
         struct.itemId = iprot.readI32();
         struct.setItemIdIsSet(true);
         {
-          org.apache.thrift.protocol.TList _list431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<ByteBuffer>(_list431.size);
-          for (int _i432 = 0; _i432 < _list431.size; ++_i432)
+          org.apache.thrift.protocol.TList _list439 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<ByteBuffer>(_list439.size);
+          for (int _i440 = 0; _i440 < _list439.size; ++_i440)
           {
-            ByteBuffer _elem433; // optional
-            _elem433 = iprot.readBinary();
-            struct.values.add(_elem433);
+            ByteBuffer _elem441; // required
+            _elem441 = iprot.readBinary();
+            struct.values.add(_elem441);
           }
         }
         struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
index 8943afb..0021366 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
@@ -3601,7 +3601,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
                 struct.column_metadata = new ArrayList<ColumnDef>(_list92.size);
                 for (int _i93 = 0; _i93 < _list92.size; ++_i93)
                 {
-                  ColumnDef _elem94; // optional
+                  ColumnDef _elem94; // required
                   _elem94 = new ColumnDef();
                   _elem94.read(iprot);
                   struct.column_metadata.add(_elem94);
@@ -4482,7 +4482,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           struct.column_metadata = new ArrayList<ColumnDef>(_list121.size);
           for (int _i122 = 0; _i122 < _list121.size; ++_i122)
           {
-            ColumnDef _elem123; // optional
+            ColumnDef _elem123; // required
             _elem123 = new ColumnDef();
             _elem123.read(iprot);
             struct.column_metadata.add(_elem123);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterSuperColumn.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterSuperColumn.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterSuperColumn.java
index f9bccfe..ea719cc 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterSuperColumn.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterSuperColumn.java
@@ -479,7 +479,7 @@ public class CounterSuperColumn implements org.apache.thrift.TBase<CounterSuperC
                 struct.columns = new ArrayList<CounterColumn>(_list8.size);
                 for (int _i9 = 0; _i9 < _list8.size; ++_i9)
                 {
-                  CounterColumn _elem10; // optional
+                  CounterColumn _elem10; // required
                   _elem10 = new CounterColumn();
                   _elem10.read(iprot);
                   struct.columns.add(_elem10);
@@ -560,7 +560,7 @@ public class CounterSuperColumn implements org.apache.thrift.TBase<CounterSuperC
         struct.columns = new ArrayList<CounterColumn>(_list13.size);
         for (int _i14 = 0; _i14 < _list13.size; ++_i14)
         {
-          CounterColumn _elem15; // optional
+          CounterColumn _elem15; // required
           _elem15 = new CounterColumn();
           _elem15.read(iprot);
           struct.columns.add(_elem15);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlPreparedResult.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlPreparedResult.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlPreparedResult.java
index 3f137de..c3f232f 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlPreparedResult.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlPreparedResult.java
@@ -653,7 +653,7 @@ public class CqlPreparedResult implements org.apache.thrift.TBase<CqlPreparedRes
                 struct.variable_types = new ArrayList<String>(_list194.size);
                 for (int _i195 = 0; _i195 < _list194.size; ++_i195)
                 {
-                  String _elem196; // optional
+                  String _elem196; // required
                   _elem196 = iprot.readString();
                   struct.variable_types.add(_elem196);
                 }
@@ -671,7 +671,7 @@ public class CqlPreparedResult implements org.apache.thrift.TBase<CqlPreparedRes
                 struct.variable_names = new ArrayList<String>(_list197.size);
                 for (int _i198 = 0; _i198 < _list197.size; ++_i198)
                 {
-                  String _elem199; // optional
+                  String _elem199; // required
                   _elem199 = iprot.readString();
                   struct.variable_names.add(_elem199);
                 }
@@ -798,7 +798,7 @@ public class CqlPreparedResult implements org.apache.thrift.TBase<CqlPreparedRes
           struct.variable_types = new ArrayList<String>(_list204.size);
           for (int _i205 = 0; _i205 < _list204.size; ++_i205)
           {
-            String _elem206; // optional
+            String _elem206; // required
             _elem206 = iprot.readString();
             struct.variable_types.add(_elem206);
           }
@@ -811,7 +811,7 @@ public class CqlPreparedResult implements org.apache.thrift.TBase<CqlPreparedRes
           struct.variable_names = new ArrayList<String>(_list207.size);
           for (int _i208 = 0; _i208 < _list207.size; ++_i208)
           {
-            String _elem209; // optional
+            String _elem209; // required
             _elem209 = iprot.readString();
             struct.variable_names.add(_elem209);
           }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlResult.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlResult.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlResult.java
index 5867b44..b989f7e 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlResult.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlResult.java
@@ -648,7 +648,7 @@ public class CqlResult implements org.apache.thrift.TBase<CqlResult, CqlResult._
                 struct.rows = new ArrayList<CqlRow>(_list186.size);
                 for (int _i187 = 0; _i187 < _list186.size; ++_i187)
                 {
-                  CqlRow _elem188; // optional
+                  CqlRow _elem188; // required
                   _elem188 = new CqlRow();
                   _elem188.read(iprot);
                   struct.rows.add(_elem188);
@@ -781,7 +781,7 @@ public class CqlResult implements org.apache.thrift.TBase<CqlResult, CqlResult._
           struct.rows = new ArrayList<CqlRow>(_list191.size);
           for (int _i192 = 0; _i192 < _list191.size; ++_i192)
           {
-            CqlRow _elem193; // optional
+            CqlRow _elem193; // required
             _elem193 = new CqlRow();
             _elem193.read(iprot);
             struct.rows.add(_elem193);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlRow.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlRow.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlRow.java
index d495530..469fc9a 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlRow.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CqlRow.java
@@ -482,7 +482,7 @@ public class CqlRow implements org.apache.thrift.TBase<CqlRow, CqlRow._Fields>,
                 struct.columns = new ArrayList<Column>(_list158.size);
                 for (int _i159 = 0; _i159 < _list158.size; ++_i159)
                 {
-                  Column _elem160; // optional
+                  Column _elem160; // required
                   _elem160 = new Column();
                   _elem160.read(iprot);
                   struct.columns.add(_elem160);
@@ -563,7 +563,7 @@ public class CqlRow implements org.apache.thrift.TBase<CqlRow, CqlRow._Fields>,
         struct.columns = new ArrayList<Column>(_list163.size);
         for (int _i164 = 0; _i164 < _list163.size; ++_i164)
         {
-          Column _elem165; // optional
+          Column _elem165; // required
           _elem165 = new Column();
           _elem165.read(iprot);
           struct.columns.add(_elem165);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/IndexClause.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/IndexClause.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/IndexClause.java
index 3e1058f..499a081 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/IndexClause.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/IndexClause.java
@@ -559,7 +559,7 @@ public class IndexClause implements org.apache.thrift.TBase<IndexClause, IndexCl
                 struct.expressions = new ArrayList<IndexExpression>(_list24.size);
                 for (int _i25 = 0; _i25 < _list24.size; ++_i25)
                 {
-                  IndexExpression _elem26; // optional
+                  IndexExpression _elem26; // required
                   _elem26 = new IndexExpression();
                   _elem26.read(iprot);
                   struct.expressions.add(_elem26);
@@ -661,7 +661,7 @@ public class IndexClause implements org.apache.thrift.TBase<IndexClause, IndexCl
         struct.expressions = new ArrayList<IndexExpression>(_list29.size);
         for (int _i30 = 0; _i30 < _list29.size; ++_i30)
         {
-          IndexExpression _elem31; // optional
+          IndexExpression _elem31; // required
           _elem31 = new IndexExpression();
           _elem31.read(iprot);
           struct.expressions.add(_elem31);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/KeyRange.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/KeyRange.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/KeyRange.java
index 152444e..243327d 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/KeyRange.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/KeyRange.java
@@ -848,7 +848,7 @@ public class KeyRange implements org.apache.thrift.TBase<KeyRange, KeyRange._Fie
                 struct.row_filter = new ArrayList<IndexExpression>(_list32.size);
                 for (int _i33 = 0; _i33 < _list32.size; ++_i33)
                 {
-                  IndexExpression _elem34; // optional
+                  IndexExpression _elem34; // required
                   _elem34 = new IndexExpression();
                   _elem34.read(iprot);
                   struct.row_filter.add(_elem34);
@@ -1017,7 +1017,7 @@ public class KeyRange implements org.apache.thrift.TBase<KeyRange, KeyRange._Fie
           struct.row_filter = new ArrayList<IndexExpression>(_list37.size);
           for (int _i38 = 0; _i38 < _list37.size; ++_i38)
           {
-            IndexExpression _elem39; // optional
+            IndexExpression _elem39; // required
             _elem39 = new IndexExpression();
             _elem39.read(iprot);
             struct.row_filter.add(_elem39);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ba5cf12/interface/thrift/gen-java/org/apache/cassandra/thrift/KeySlice.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/KeySlice.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/KeySlice.java
index 07a53d4..e7de670 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/KeySlice.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/KeySlice.java
@@ -486,7 +486,7 @@ public class KeySlice implements org.apache.thrift.TBase<KeySlice, KeySlice._Fie
                 struct.columns = new ArrayList<ColumnOrSuperColumn>(_list40.size);
                 for (int _i41 = 0; _i41 < _list40.size; ++_i41)
                 {
-                  ColumnOrSuperColumn _elem42; // optional
+                  ColumnOrSuperColumn _elem42; // required
                   _elem42 = new ColumnOrSuperColumn();
                   _elem42.read(iprot);
                   struct.columns.add(_elem42);
@@ -567,7 +567,7 @@ public class KeySlice implements org.apache.thrift.TBase<KeySlice, KeySlice._Fie
         struct.columns = new ArrayList<ColumnOrSuperColumn>(_list45.size);
         for (int _i46 = 0; _i46 < _list45.size; ++_i46)
         {
-          ColumnOrSuperColumn _elem47; // optional
+          ColumnOrSuperColumn _elem47; // required
           _elem47 = new ColumnOrSuperColumn();
           _elem47.read(iprot);
           struct.columns.add(_elem47);