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
         {

Reply via email to