You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/10/31 20:04:16 UTC

[GitHub] [cassandra] maedhroz opened a new pull request, #1962: CQL support for CEP-15 multi-partition transactions

maedhroz opened a new pull request, #1962:
URL: https://github.com/apache/cassandra/pull/1962

   patch by Caleb Rackliffe; reviewed by ? for CASSANDRA-17719
   
   Co-authored-by: Caleb Rackliffe <ca...@gmail.com>
   Co-authored-by: Blake Eggleston <bd...@gmail.com>
   Co-authored-by: David Capwell <dc...@apache.org>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] aweisberg commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
aweisberg commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1044827942


##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -41,15 +48,13 @@
  */
 public abstract class Constants
 {
-    private static final Logger logger = LoggerFactory.getLogger(Constants.class);
-
     public enum Type
     {
         STRING
         {
             public AbstractType<?> getPreferedTypeFor(String text)
             {
-                 if(Charset.forName("US-ASCII").newEncoder().canEncode(text))
+                 if(StandardCharsets.US_ASCII.newEncoder().canEncode(text))

Review Comment:
   Good candidate for static import



##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +519,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()
+        {
+            return !(column.type instanceof CounterColumnType);
+        }
+
         public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
-            ByteBuffer bytes = t.bindAndGet(params.options);
-            if (bytes == null)
-                throw new InvalidRequestException("Invalid null value for counter increment");
-            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                return;
+            if (column.type instanceof CounterColumnType)
+            {
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for counter increment");

Review Comment:
   It's a decrement not increment?



##########
src/java/org/apache/cassandra/service/accord/AccordService.java:
##########
@@ -99,6 +110,39 @@ public static long nowInMicros()
         return TimeUnit.MILLISECONDS.toMicros(Clock.Global.currentTimeMillis());
     }
 
+    public TxnData coordinate(Txn txn)
+    {
+        try
+        {
+            Future<Result> future = node.coordinate(txn);
+            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
+            return (TxnData) result;
+        }
+        catch (ExecutionException e)
+        {
+            Throwable cause = e.getCause();
+            if (cause instanceof Timeout)
+                throw throwTimeout(txn);
+            throw new RuntimeException(cause);
+        }
+        catch (InterruptedException e)
+        {
+            throw new UncheckedInterruptedException(e);
+        }
+        catch (TimeoutException e)
+        {
+            throw throwTimeout(txn);
+        }
+    }
+
+    private static RuntimeException throwTimeout(Txn txn)
+    {
+        // TODO: Consistency levels in these timeout exceptions have no semantics. Is there a better option?
+        throw txn.isWrite() ?
+              new WriteTimeoutException(WriteType.TRANSACTION, ConsistencyLevel.ANY, 0, 0) :

Review Comment:
   During migration and rollback these consistency levels will probably mean something so we should echo them back.



##########
src/java/org/apache/cassandra/service/accord/txn/TxnData.java:
##########
@@ -156,5 +164,43 @@ public long serializedSize(FilteredPartition partition, int version)
         }
     };
 
-    public static final IVersionedSerializer<AccordData> serializer = new Serializer<>(AccordData::new);
+    public static final IVersionedSerializer<TxnData> serializer = new IVersionedSerializer<TxnData>()
+    {
+        @Override
+        public void serialize(TxnData data, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeInt(data.data.size());

Review Comment:
   vint?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceValue.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class TxnReferenceValue
+{
+    private interface Serializer<T extends TxnReferenceValue>
+    {
+        void serialize(T t, DataOutputPlus out, int version) throws IOException;
+        T deserialize(DataInputPlus in, int version, Kind kind) throws IOException;
+        long serializedSize(T t, int version);
+    }
+
+    enum Kind
+    {
+        CONSTANT(Constant.serializer),
+        SUBSTITUTION(Substitution.serializer);
+
+        @SuppressWarnings("rawtypes")
+        final Serializer serializer;
+
+        Kind(Serializer<? extends TxnReferenceValue> serializer)
+        {
+            this.serializer = serializer;
+        }
+    }
+
+    protected abstract Kind kind();
+    abstract ByteBuffer compute(TxnData data, AbstractType<?> receiver);
+
+    public static class Constant extends TxnReferenceValue
+    {
+        private final ByteBuffer value;
+
+        public Constant(ByteBuffer value)
+        {
+            this.value = value;
+        }
+
+        public ByteBuffer getValue()
+        {
+            return value;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Constant constant = (Constant) o;
+            return value.equals(constant.value);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(value);
+        }
+
+        @Override
+        public String toString()
+        {
+            return ByteBufferUtil.bytesToHex(value);
+        }
+
+        @Override
+        public Kind kind()
+        {
+            return Kind.CONSTANT;
+        }
+
+        @Override
+        public ByteBuffer compute(TxnData data, AbstractType<?> receiver)
+        {
+            return value;
+        }
+
+        private static final Serializer<Constant> serializer = new Serializer<Constant>()
+        {
+            @Override
+            public void serialize(Constant constant, DataOutputPlus out, int version) throws IOException
+            {
+                ByteBufferUtil.writeWithVIntLength(constant.value, out);
+            }
+
+            @Override
+            public Constant deserialize(DataInputPlus in, int version, Kind kind) throws IOException
+            {
+                return new Constant(ByteBufferUtil.readWithVIntLength(in));
+            }
+
+            @Override
+            public long serializedSize(Constant constant, int version)
+            {
+                return ByteBufferUtil.serializedSizeWithVIntLength(constant.value);
+            }
+        };
+    }
+
+    public static class Substitution extends TxnReferenceValue
+    {
+        private final TxnReference reference;
+
+        public Substitution(TxnReference reference)
+        {
+            this.reference = reference;
+        }
+
+        @Override
+        public String toString()
+        {
+            return reference.toString();
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Substitution that = (Substitution) o;
+            return reference.equals(that.reference);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(reference);
+        }
+
+        @Override
+        public Kind kind()
+        {
+            return Kind.SUBSTITUTION;
+        }
+
+        @Override
+        public ByteBuffer compute(TxnData data, AbstractType<?> receiver)
+        {
+            return reference.toByteBuffer(data, receiver);
+        }
+
+        private static final Serializer<Substitution> serializer = new Serializer<Substitution>()
+        {
+            @Override
+            public void serialize(Substitution substitution, DataOutputPlus out, int version) throws IOException
+            {
+                TxnReference.serializer.serialize(substitution.reference, out, version);
+            }
+
+            @Override
+            public Substitution deserialize(DataInputPlus in, int version, Kind kind) throws IOException
+            {
+                return new Substitution(TxnReference.serializer.deserialize(in, version));
+            }
+
+            @Override
+            public long serializedSize(Substitution substitution, int version)
+            {
+                return TxnReference.serializer.serializedSize(substitution.reference, version);
+            }
+        };
+    }
+
+    static final IVersionedSerializer<TxnReferenceValue> serializer = new IVersionedSerializer<TxnReferenceValue>()
+    {
+        @SuppressWarnings("unchecked")
+        @Override
+        public void serialize(TxnReferenceValue value, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeInt(value.kind().ordinal());

Review Comment:
   vint?



##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +519,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()

Review Comment:
   Comment on `Operation.requiresRead()` is doesn't seem to be valid anymore.



##########
src/java/org/apache/cassandra/cql3/Operations.java:
##########
@@ -138,9 +171,35 @@ public Iterator<Operation> iterator()
         return Iterators.concat(staticOperations.iterator(), regularOperations.iterator());
     }
 
+

Review Comment:
   Extra line break



##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -83,6 +100,13 @@ public void collectMarkerSpecification(VariableSpecifications boundNames)
             t.collectMarkerSpecification(boundNames);
     }
 
+    protected ByteBuffer getCurrentCellBuffer(DecoratedKey key, UpdateParameters params)
+    {
+        Row currentRow = params.getPrefetchedRow(key, column.isStatic() ? Clustering.STATIC_CLUSTERING : params.currentClustering());

Review Comment:
   Comment on `UpdateParameters.prefetchedRows` is also wrong now.



##########
src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserializeCqlCollectionAsTerm;
+import static org.apache.cassandra.utils.CollectionSerializers.deserializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializedListSize;
+
+public abstract class TxnCondition
+{
+    private interface ConditionSerializer<T extends TxnCondition>
+    {
+        void serialize(T condition, DataOutputPlus out, int version) throws IOException;
+        T deserialize(DataInputPlus in, int version, Kind kind) throws IOException;
+        long serializedSize(T condition, int version);
+    }
+
+    public enum Kind
+    {
+        NONE("n/a", null),
+        AND("AND", null),
+        OR("OR", null),
+        IS_NOT_NULL("IS NOT NULL", null),
+        IS_NULL("IS NULL", null),
+        EQUAL("=", Operator.EQ),
+        NOT_EQUAL("!=", Operator.NEQ),
+        GREATER_THAN(">", Operator.GT),
+        GREATER_THAN_OR_EQUAL(">=", Operator.GTE),
+        LESS_THAN("<", Operator.LT),
+        LESS_THAN_OR_EQUAL("<=", Operator.LTE);
+
+        private final String symbol;
+        private final Operator operator;
+
+        Kind(String symbol, Operator operator)
+        {
+            this.symbol = symbol;
+            this.operator = operator;
+        }
+
+        @SuppressWarnings("rawtypes")
+        private ConditionSerializer serializer()
+        {
+            switch (this)
+            {
+                case IS_NOT_NULL:
+                case IS_NULL:
+                    return Exists.serializer;
+                case EQUAL:
+                case NOT_EQUAL:
+                case LESS_THAN:
+                case LESS_THAN_OR_EQUAL:
+                case GREATER_THAN:
+                case GREATER_THAN_OR_EQUAL:
+                    return Value.serializer;
+                case AND:
+                case OR:
+                    return BooleanGroup.serializer;
+                case NONE:
+                    return None.serializer;
+                default:
+                    throw new IllegalArgumentException("No serializer exists for kind " + this);
+            }
+        }
+    }
+
+    protected final Kind kind;
+
+    public TxnCondition(Kind kind)
+    {
+        this.kind = kind;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnCondition condition = (TxnCondition) o;
+        return kind == condition.kind;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(kind);
+    }
+
+    public Kind kind()
+    {
+        return kind;
+    }
+
+    public abstract boolean applies(TxnData data);
+
+    private static class None extends TxnCondition
+    {
+        private static final None instance = new None();
+
+        private None()
+        {
+            super(Kind.NONE);
+        }
+
+        @Override
+        public String toString()
+        {
+            return kind.toString();
+        }
+
+        @Override
+        public boolean applies(TxnData data)
+        {
+            return true;
+        }
+
+        private static final ConditionSerializer<None> serializer = new ConditionSerializer<None>()
+        {
+            @Override
+            public void serialize(None condition, DataOutputPlus out, int version) {}
+            @Override
+            public None deserialize(DataInputPlus in, int version, Kind kind) { return instance; }
+            @Override
+            public long serializedSize(None condition, int version) { return 0; }
+        };
+    }
+
+    public static TxnCondition none()
+    {
+        return None.instance;
+    }
+
+    public static class Exists extends TxnCondition
+    {
+        private static final Set<Kind> KINDS = ImmutableSet.of(Kind.IS_NOT_NULL, Kind.IS_NULL);
+
+        public final TxnReference reference;
+
+        public Exists(TxnReference reference, Kind kind)
+        {
+            super(kind);
+            Preconditions.checkArgument(KINDS.contains(kind), "Kind " + kind + " cannot be used with an existence condition");
+            this.reference = reference;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            Exists exists = (Exists) o;
+            return reference.equals(exists.reference);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), reference);
+        }
+
+        @Override
+        public String toString()
+        {
+            return reference.toString() + ' ' + kind.toString();
+        }
+
+        @Override
+        public boolean applies(TxnData data)
+        {
+            FilteredPartition partition = reference.getPartition(data);
+            boolean exists = partition != null && !partition.isEmpty();
+
+            Row row = null;
+            if (exists)
+            {
+                row = reference.getRow(partition);
+                exists = row != null && !row.isEmpty();
+            }
+
+            if (exists && reference.selectsColumn())
+            {
+                ColumnData columnData = reference.getColumnData(row);
+
+                if (columnData == null)
+                {
+                    exists = false;
+                }
+                else if (columnData.column().isComplex())
+                {
+                    if (reference.isElementSelection() || reference.isFieldSelection())
+                    {
+                        Cell<?> cell = (Cell<?>) columnData;
+                        exists = !cell.isTombstone();
+                    }
+                    else
+                    {
+                        // TODO: Is this even necessary, given the partition is already filtered?
+                        if (!((ComplexColumnData) columnData).complexDeletion().isLive())
+                            exists = false;
+                    }
+                }
+                else if (reference.isElementSelection())
+                {
+                    // This is frozen, so check if the Cell is a tombstone and that the element is present.
+                    Cell<?> cell = (Cell<?>) columnData;
+                    ByteBuffer element = reference.getFrozenCollectionElement(cell);
+                    exists = element != null && !cell.isTombstone();
+                }
+                else if (reference.isFieldSelection())
+                {
+                    // This is frozen, so check if the Cell is a tombstone and that the field is present.
+                    Cell<?> cell = (Cell<?>) columnData;
+                    ByteBuffer fieldValue = reference.getFrozenFieldValue(cell);
+                    exists = fieldValue != null && !cell.isTombstone();
+                }
+                else
+                {
+                    Cell<?> cell = (Cell<?>) columnData;
+                    exists = !cell.isTombstone();
+                }
+            }
+
+            switch (kind())
+            {
+                case IS_NOT_NULL:
+                    return exists;
+                case IS_NULL:
+                    return !exists;
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+
+        private static final ConditionSerializer<Exists> serializer = new ConditionSerializer<Exists>()
+        {
+            @Override
+            public void serialize(Exists condition, DataOutputPlus out, int version) throws IOException
+            {
+                TxnReference.serializer.serialize(condition.reference, out, version);
+            }
+
+            @Override
+            public Exists deserialize(DataInputPlus in, int version, Kind kind) throws IOException
+            {
+                return new Exists(TxnReference.serializer.deserialize(in, version), kind);
+            }
+
+            @Override
+            public long serializedSize(Exists condition, int version)
+            {
+                return TxnReference.serializer.serializedSize(condition.reference, version);
+            }
+        };
+    }
+
+    public static class Value extends TxnCondition
+    {
+        private static final Set<Kind> KINDS = ImmutableSet.of(Kind.EQUAL, Kind.NOT_EQUAL,
+                                                               Kind.GREATER_THAN, Kind.GREATER_THAN_OR_EQUAL,
+                                                               Kind.LESS_THAN, Kind.LESS_THAN_OR_EQUAL);
+
+        private final TxnReference reference;
+        private final ByteBuffer value;
+        private final ProtocolVersion version;
+
+        public Value(TxnReference reference, Kind kind, ByteBuffer value, ProtocolVersion version)
+        {
+            super(kind);
+            Preconditions.checkArgument(KINDS.contains(kind), "Kind " + kind + " cannot be used with a value condition");
+            Preconditions.checkArgument(reference.selectsColumn(), "Reference " + reference + " does not select a column");
+            this.reference = reference;
+            this.value = value;
+            this.version = version;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            Value value1 = (Value) o;
+            return reference.equals(value1.reference) && value.equals(value1.value);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), reference, value);
+        }
+
+        @Override
+        public String toString()
+        {
+            return reference.toString() + ' ' + kind.symbol + " 0x" + ByteBufferUtil.bytesToHex(value);
+        }
+
+        @Override
+        public boolean applies(TxnData data)
+        {
+            ColumnMetadata column = reference.column();
+
+            if (column.isPartitionKey())
+                return ColumnCondition.Bound.compareWithOperator(kind.operator, column.type, value, reference.getPartitionKey(data));
+            else if (column.isClusteringColumn())
+                return ColumnCondition.Bound.compareWithOperator(kind.operator, column.type, value, reference.getClusteringKey(data));
+
+            if (column.isComplex())
+            {
+                AbstractType<?> type = column.type;
+                Row row = reference.getRow(data);
+                if (row == null) return false;
+
+                if (type.isCollection())
+                {
+                    if (reference.selectsPath())
+                    {
+                        return new ColumnCondition.ElementAccessBound(column, reference.path().get(0), kind.operator, Collections.singletonList(value)).appliesTo(row);
+                    }
+                    else
+                    {
+                        Term.Terminal term = deserializeCqlCollectionAsTerm(value, type, version);
+                        return ColumnCondition.MultiCellCollectionBound.appliesTo(column, kind.operator, Collections.singletonList(term), row);
+                    }
+                }
+                else if (type.isUDT())
+                {
+                    if (reference.isFieldSelection())
+                    {
+                        Cell<?> cell = (Cell<?>) reference.getColumnData(data);
+                        if (cell == null) return false;
+                        return ColumnCondition.Bound.compareWithOperator(kind.operator, reference.getFieldSelectionType(), value, cell.buffer());
+                    }
+
+                    return new ColumnCondition.MultiCellUdtBound(column, kind.operator, Collections.singletonList(value), version).appliesTo(row);
+                }
+
+                throw new UnsupportedOperationException("Unsupported complex type: " + type);
+            }
+
+            Cell<?> cell = (Cell<?>) reference.getColumnData(data);
+            if (cell == null) return false;
+
+            if (reference.isElementSelection())
+            {
+                // Frozen...otherwise we would have fallen into the complex logic above.
+                ByteBuffer element = reference.getFrozenCollectionElement(cell);
+                AbstractType<?> comparator = ((CollectionType<?>) column.type).valueComparator();
+                return ColumnCondition.Bound.compareWithOperator(kind.operator, comparator, value, element);
+            }
+            else if (reference.isFieldSelection())
+            {
+                ByteBuffer fieldValue = reference.getFrozenFieldValue(cell);
+                return ColumnCondition.Bound.compareWithOperator(kind.operator, reference.getFieldSelectionType(), value, fieldValue);
+            }
+
+            return ColumnCondition.Bound.compareWithOperator(kind.operator, column.type, value, cell.buffer());
+        }
+
+        private static final ConditionSerializer<Value> serializer = new ConditionSerializer<Value>()
+        {
+            @Override
+            public void serialize(Value condition, DataOutputPlus out, int version) throws IOException
+            {
+                TxnReference.serializer.serialize(condition.reference, out, version);
+                ByteBufferUtil.writeWithVIntLength(condition.value, out);
+                out.writeUTF(condition.version.name());
+            }
+
+            @Override
+            public Value deserialize(DataInputPlus in, int version, Kind kind) throws IOException
+            {
+                TxnReference reference = TxnReference.serializer.deserialize(in, version);
+                ByteBuffer value = ByteBufferUtil.readWithVIntLength(in);
+                ProtocolVersion protocolVersion = ProtocolVersion.valueOf(in.readUTF());
+                return new Value(reference, kind, value, protocolVersion);
+            }
+
+            @Override
+            public long serializedSize(Value condition, int version)
+            {
+                long size = 0;
+                size += TxnReference.serializer.serializedSize(condition.reference, version);
+                size += ByteBufferUtil.serializedSizeWithVIntLength(condition.value);
+                size += TypeSizes.sizeof(condition.version.name());
+                return size;
+            }
+        };
+    }
+
+    public static class BooleanGroup extends TxnCondition
+    {
+        private static final Set<Kind> KINDS = ImmutableSet.of(Kind.AND, Kind.OR);
+
+        public final List<TxnCondition> conditions;
+
+        public BooleanGroup(Kind kind, List<TxnCondition> conditions)
+        {
+            super(kind);
+            Preconditions.checkArgument(KINDS.contains(kind), "Kind " + kind + " cannot be used at the root of a boolean condition");
+            this.conditions = conditions;
+        }
+
+        @Override
+        public String toString()
+        {
+            return '(' + conditions.stream().map(Objects::toString).reduce((a, b) -> a + ' ' + kind.symbol  + ' ' + b).orElse("") + ')';
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            if (!super.equals(o)) return false;
+            BooleanGroup that = (BooleanGroup) o;
+            return Objects.equals(conditions, that.conditions);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(super.hashCode(), conditions);
+        }
+
+        @Override
+        public boolean applies(TxnData data)
+        {
+            switch (kind())
+            {
+                case AND:
+                    return Iterables.all(conditions, c -> c.applies(data));
+                case OR:
+                    return Iterables.any(conditions, c -> c.applies(data));
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+
+        private static final ConditionSerializer<BooleanGroup> serializer = new ConditionSerializer<BooleanGroup>()
+        {
+            @Override
+            public void serialize(BooleanGroup condition, DataOutputPlus out, int version) throws IOException
+            {
+                serializeList(condition.conditions, out, version, TxnCondition.serializer);
+            }
+
+            @Override
+            public BooleanGroup deserialize(DataInputPlus in, int version, Kind kind) throws IOException
+            {
+                return new BooleanGroup(kind, deserializeList(in, version, TxnCondition.serializer));
+            }
+
+            @Override
+            public long serializedSize(BooleanGroup condition, int version)
+            {
+                return serializedListSize(condition.conditions, version, TxnCondition.serializer);
+            }
+        };
+    }
+
+    public static final IVersionedSerializer<TxnCondition> serializer = new IVersionedSerializer<TxnCondition>()
+    {
+        @SuppressWarnings("unchecked")
+        @Override
+        public void serialize(TxnCondition condition, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeInt(condition.kind.ordinal());

Review Comment:
   vint?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039935959


##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            int n = readCollectionSize(collection, ByteBufferAccessor.instance, ProtocolVersion.V3);

Review Comment:
   currently this isn't an issue, but hard coding `V3` feels buggy... if we get a v6 with changes we then try to read as v3?  Don't we need to know how it was written?



##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            int n = readCollectionSize(collection, ByteBufferAccessor.instance, ProtocolVersion.V3);
+            int offset = sizeOfCollectionSize(n, ProtocolVersion.V3);

Review Comment:
   for readability a doc saying why is good, as this method I think causes more confusion (at least it did for me)
   
   ```
   // the offset starts after the size, which is after the sizeOf the collection size
   ```



##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+
+        throw new UnsupportedOperationException("Unsupported collection type: " + type);
+    }
+
+    public static final IVersionedSerializer<PartitionUpdate> partitionUpdateSerializer = new IVersionedSerializer<PartitionUpdate>()
+    {
+        @Override
+        public void serialize(PartitionUpdate upd, DataOutputPlus out, int version) throws IOException
+        {
+            PartitionUpdate.serializer.serialize(upd, out, version);
+        }
+
+        @Override
+        public PartitionUpdate deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return PartitionUpdate.serializer.deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE);
+        }
+
+        @Override
+        public long serializedSize(PartitionUpdate upd, int version)
+        {
+            return PartitionUpdate.serializer.serializedSize(upd, version);
+        }
+    };
+
+    public static final IVersionedSerializer<SinglePartitionReadCommand> singlePartitionReadCommandSerializer = new IVersionedSerializer<SinglePartitionReadCommand>()
+    {
+        @Override
+        public void serialize(SinglePartitionReadCommand command, DataOutputPlus out, int version) throws IOException
+        {
+            SinglePartitionReadCommand.serializer.serialize(command, out, version);
+        }
+
+        @Override
+        public SinglePartitionReadCommand deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return (SinglePartitionReadCommand) SinglePartitionReadCommand.serializer.deserialize(in, version);
+        }
+
+        @Override
+        public long serializedSize(SinglePartitionReadCommand command, int version)
+        {
+            return SinglePartitionReadCommand.serializer.serializedSize(command, version);
+        }
+    };
+
+    public static final IVersionedSerializer<ColumnMetadata> columnMetadataSerializer = new IVersionedSerializer<ColumnMetadata>()

Review Comment:
   should this move to `ColumnMetadata`?



##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+
+        throw new UnsupportedOperationException("Unsupported collection type: " + type);
+    }
+
+    public static final IVersionedSerializer<PartitionUpdate> partitionUpdateSerializer = new IVersionedSerializer<PartitionUpdate>()
+    {
+        @Override
+        public void serialize(PartitionUpdate upd, DataOutputPlus out, int version) throws IOException
+        {
+            PartitionUpdate.serializer.serialize(upd, out, version);
+        }
+
+        @Override
+        public PartitionUpdate deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return PartitionUpdate.serializer.deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE);
+        }
+
+        @Override
+        public long serializedSize(PartitionUpdate upd, int version)
+        {
+            return PartitionUpdate.serializer.serializedSize(upd, version);
+        }
+    };
+
+    public static final IVersionedSerializer<SinglePartitionReadCommand> singlePartitionReadCommandSerializer = new IVersionedSerializer<SinglePartitionReadCommand>()
+    {
+        @Override
+        public void serialize(SinglePartitionReadCommand command, DataOutputPlus out, int version) throws IOException
+        {
+            SinglePartitionReadCommand.serializer.serialize(command, out, version);
+        }
+
+        @Override
+        public SinglePartitionReadCommand deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return (SinglePartitionReadCommand) SinglePartitionReadCommand.serializer.deserialize(in, version);
+        }
+
+        @Override
+        public long serializedSize(SinglePartitionReadCommand command, int version)
+        {
+            return SinglePartitionReadCommand.serializer.serializedSize(command, version);
+        }
+    };

Review Comment:
   doesn't look needed as `org.apache.cassandra.db.ReadCommand#serializer` already impl `IVersionedSerializer`; only difference is deserialize that returns `ReadCommand` vs casting to `SinglePartitionReadCommand`



##########
src/java/org/apache/cassandra/service/accord/txn/AccordUpdateParameters.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+
+public class AccordUpdateParameters
+{
+    private final TxnData data;
+    private final QueryOptions options;
+
+    public AccordUpdateParameters(TxnData data, QueryOptions options)
+    {
+        this.data = data;
+        this.options = options;
+    }
+
+    public TxnData getData()
+    {
+        return data;
+    }
+
+    public UpdateParameters updateParameters(TableMetadata metadata, int rowIndex)
+    {
+        // This is currently only used by Guardrails, but this logically have issues with Accord as drifts in config
+        // values could cause unexpected issues in Accord. (ex. some nodes reject writes while others accept)
+        // For the time being, guardrails are disabled for Accord queries.
+        ClientState disabledGuardrails = null;
+
+        // What we use here doesn't matter as they get replaced before actually performing the write.
+        // see org.apache.cassandra.service.accord.txn.TxnWrite.Update.write
+        int nowInSeconds = 42;
+        long timestamp = nowInSeconds;
+
+        // TODO: How should Accord work with TTL?
+        int ttl = metadata.params.defaultTimeToLive;
+        return new UpdateParameters(metadata,
+                                    disabledGuardrails,
+                                    options,
+                                    timestamp,
+                                    nowInSeconds,
+                                    ttl,
+                                    prefetchRow(metadata, rowIndex));
+    }
+
+    private Map<DecoratedKey, Partition> prefetchRow(TableMetadata metadata, int index)
+    {
+        for (Map.Entry<TxnDataName, FilteredPartition> e : data.entrySet())

Review Comment:
   Should prob just read `AUTO_READ` rather than rely on `isFor` to filter



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;

Review Comment:
   in java all statics are cap; we don't override this in C* (though we are not consistent in C* on this)



##########
src/java/org/apache/cassandra/service/StorageServiceMBean.java:
##########
@@ -630,6 +632,9 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion,
     public void setTruncateRpcTimeout(long value);
     public long getTruncateRpcTimeout();
 
+    public void setTransactionTimeout(long value);
+    public long getTransactionTimeout();

Review Comment:
   should use `String`



##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -533,6 +551,35 @@ else if (obj instanceof byte[])
             return ByteBuffer.wrap((byte[]) obj);
         else if (obj instanceof ByteBuffer)
             return (ByteBuffer) obj;
+        else if (obj instanceof List)
+        {
+            List<?> list = (List<?>) obj;
+            // convert subtypes to BB
+            List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList());
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return ListType.getInstance(BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Map)
+        {
+            Map<?, ?> map = (Map<?, ?>) obj;
+            // convert subtypes to BB
+            Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
+            for (Map.Entry<?, ?> e : map.entrySet())
+                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));

Review Comment:
   yep, that's about it; we know `key` .equals/.hashCode don't conflict but the serialized version "could" invalidate that, so the check you added would detect that case.



##########
src/java/org/apache/cassandra/service/accord/txn/AbstractKeySorted.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+
+import accord.primitives.Keys;
+import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
+
+/**
+ * Immutable collection of items, sorted first by their partition key
+ */
+public abstract class AbstractKeySorted<T> implements Iterable<T>
+{
+    public static final String ITEMS_OUT_OF_ORDER_MESSAGE = "Items are out of order ([%s] %s >= [%s] %s)";
+
+    protected final Keys itemKeys;
+    protected final T[] items;
+
+    // items are expected to be sorted
+    public AbstractKeySorted(T[] items)
+    {
+        this.items = items;
+        this.itemKeys = extractItemKeys();
+    }
+
+    public AbstractKeySorted(List<T> items)
+    {
+        T[] arr = newArray(items.size());
+        items.toArray(arr);
+        Arrays.sort(arr, this::compare);
+        this.items = arr;
+        this.itemKeys = extractItemKeys();
+    }
+
+    private Keys extractItemKeys()
+    {
+        SortedSet<PartitionKey> keysSet = new TreeSet<>(AccordRoutingKey::compareKeys);
+        forEach(i -> keysSet.add(getKey(i)));
+        return new Keys(keysSet);
+    }
+
+    @Override
+    public Iterator<T> iterator()
+    {
+        return Iterators.forArray(items);
+    }
+
+    @Override
+    public String toString()
+    {
+        return getClass().getSimpleName() + Arrays.stream(items)
+                                                  .map(Objects::toString)
+                                                  .collect(Collectors.joining(", ", "{", "}"));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        AbstractKeySorted<?> that = (AbstractKeySorted<?>) o;
+        return Arrays.equals(items, that.items);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Arrays.hashCode(items);
+    }
+
+    @VisibleForTesting
+    public Keys keys()
+    {
+        return itemKeys;
+    }
+
+    /**
+     * Compare the non-key component of items (since this class handles sorting by key)
+     */
+    abstract int compareNonKeyFields(T left, T right);
+
+    abstract PartitionKey getKey(T item);
+    abstract T[] newArray(int size);
+
+    private int compare(T left, T right)
+    {
+        int cmp = getKey(left).compareTo(getKey(right));
+        return cmp != 0 ? cmp : compareNonKeyFields(left, right);
+    }
+
+    @VisibleForTesting
+    void validateOrder()
+    {
+        for (int i=1; i< items.length; i++)
+        {
+            T prev = items[i-1];
+            T next = items[i];
+            if (getKey(prev).compareTo(getKey(next)) > 0)
+                throw new IllegalStateException(String.format(ITEMS_OUT_OF_ORDER_MESSAGE, i-1, prev, i, next));
+            if (compare(prev, next) >= 0)
+                throw new IllegalStateException(String.format("Items are out of order ([%s] %s >= [%s] %s)", i-1, prev, i, next));
+        }
+    }
+
+    public int size()
+    {
+        return items.length;
+    }
+
+    public void forEachWithKey(PartitionKey key, Consumer<T> consumer)
+    {
+        for (int i = firstPossibleKeyIdx(key); i < items.length && getKey(items[i]).equals(key); i++)
+            consumer.accept(items[i]);
+    }
+
+    private int firstPossibleKeyIdx(PartitionKey key)
+    {
+        int idx = Arrays.binarySearch(items, key, (l, r) -> {
+            PartitionKey lk = getKey((T) l);
+            PartitionKey rk = (PartitionKey) r;

Review Comment:
   I had to read the code as this felt wrong... `r` is `key`...



##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            int n = readCollectionSize(collection, ByteBufferAccessor.instance, ProtocolVersion.V3);
+            int offset = sizeOfCollectionSize(n, ProtocolVersion.V3);
+            int idx = ByteBufferUtil.toInt(index);
+
+            Preconditions.checkElementIndex(idx, n);
+
+            for (int i = 0; i <= idx; i++)
+            {
+                ByteBuffer value = readValue(collection, ByteBufferAccessor.instance, offset, ProtocolVersion.V3);

Review Comment:
   pushed a change to avoid alloc and use `skip`; see ef3e2f0ea9166011c6e3bba0aa41cf599bbc0842



##########
src/java/org/apache/cassandra/service/accord/txn/AbstractKeySorted.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+
+import accord.primitives.Keys;
+import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
+
+/**
+ * Immutable collection of items, sorted first by their partition key
+ */
+public abstract class AbstractKeySorted<T> implements Iterable<T>
+{
+    public static final String ITEMS_OUT_OF_ORDER_MESSAGE = "Items are out of order ([%s] %s >= [%s] %s)";
+
+    protected final Keys itemKeys;
+    protected final T[] items;
+
+    // items are expected to be sorted
+    public AbstractKeySorted(T[] items)

Review Comment:
   unsafe, we don't know this is sorted; can call ``validateOrder or just sort like we do for `List`



##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+

Review Comment:
   shouldn't hard code the version, this could bite us in the future.



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1192,46 +1192,52 @@ public static void applySeedProvider()
     @VisibleForTesting
     static void checkForLowestAcceptedTimeouts(Config conf)
     {
-        if(conf.read_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.read_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("read_request_timeout", conf.read_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.read_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.range_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.range_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("range_request_timeout", conf.range_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.range_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("request_timeout", conf.request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.write_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.write_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("write_request_timeout", conf.write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.write_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.cas_contention_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.cas_contention_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("cas_contention_timeout", conf.cas_contention_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.cas_contention_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.counter_write_request_timeout.toMilliseconds()< LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.counter_write_request_timeout.toMilliseconds()< LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("counter_write_request_timeout", conf.counter_write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.counter_write_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
-        if(conf.truncate_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.truncate_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("truncate_request_timeout", conf.truncate_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.truncate_request_timeout = LOWEST_ACCEPTED_TIMEOUT;
         }
+
+        if (conf.transaction_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        {
+            logInfo("transaction_timeout", conf.transaction_timeout, LOWEST_ACCEPTED_TIMEOUT);
+            conf.transaction_timeout = LOWEST_ACCEPTED_TIMEOUT;
+        }

Review Comment:
   never saw this before... :sigh:   this is hard to deal with with DD as we need to duplicate the effort for non-yaml paths!  =(
   
   Your patch is fine; not asking you to do anything... just sad



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -1522,6 +1587,17 @@ public long getTruncateRpcTimeout()
         return DatabaseDescriptor.getTruncateRpcTimeout(MILLISECONDS);
     }
 
+    public void setTransactionTimeout(long value)
+    {
+        DatabaseDescriptor.setTransactionTimeout(value);
+        logger.info("set transaction timeout to {} ms", value);
+    }
+
+    public long getTransactionTimeout()
+    {
+        return DatabaseDescriptor.getTransactionTimeout(MILLISECONDS);
+    }

Review Comment:
   we should use `String` when talking to users as that's the API they know; we did this for most of the other configs as well.



##########
src/java/org/apache/cassandra/service/accord/txn/TxnAppliedQuery.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+import accord.api.Data;
+import accord.api.Query;
+import accord.api.Read;
+import accord.api.Result;
+import accord.api.Update;
+import accord.primitives.TxnId;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserialize;
+import static org.apache.cassandra.service.accord.AccordSerializers.serialize;
+
+// TODO: This is currently unused, but we might want to use it to support returning the condition result.
+public class TxnAppliedQuery implements Query
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new TxnAppliedQuery(ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    public static class Applied implements Result
+    {
+        public static final Applied TRUE = new Applied();
+        public static final Applied FALSE = new Applied();
+        private static final long SIZE = ObjectSizes.measure(TRUE);
+
+        private Applied() {}
+
+        public boolean wasApplied()
+        {
+            return this == TRUE;
+        }
+
+        public static Applied valueOf(boolean b)
+        {
+            return b ? TRUE : FALSE;
+        }
+
+        public long estimatedSizeOnHeap()
+        {
+            return SIZE;
+        }
+
+        public static final IVersionedSerializer<Applied> serializer = new IVersionedSerializer<Applied>()
+        {
+            @Override
+            public void serialize(Applied applied, DataOutputPlus out, int version) throws IOException
+            {
+                out.writeBoolean(applied.wasApplied());
+            }
+
+            @Override
+            public Applied deserialize(DataInputPlus in, int version) throws IOException
+            {
+                return Applied.valueOf(in.readBoolean());
+            }
+
+            @Override
+            public long serializedSize(Applied applied, int version)
+            {
+                return TypeSizes.BOOL_SIZE;
+            }
+        };
+    }
+    private final ByteBuffer serializedCondition;
+
+    public TxnAppliedQuery(TxnCondition condition)
+    {
+        this.serializedCondition = serialize(condition, TxnCondition.serializer);
+    }
+
+    public TxnAppliedQuery(ByteBuffer serializedCondition)
+    {
+        this.serializedCondition = serializedCondition;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnAppliedQuery query = (TxnAppliedQuery) o;
+        return Objects.equals(serializedCondition, query.serializedCondition);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(serializedCondition);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "TxnAppliedQuery{serializedCondition=" + deserialize(serializedCondition, TxnCondition.serializer) + '}';
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        return EMPTY_SIZE + ByteBufferUtil.estimatedSizeOnHeap(serializedCondition);
+    }
+
+    @Override
+    public Result compute(TxnId txnId, Data data, @Nullable Read read, @Nullable Update update)
+    {
+        TxnCondition condition = deserialize(serializedCondition, TxnCondition.serializer);
+        return condition.applies((TxnData) data) ? Applied.TRUE : Applied.FALSE;

Review Comment:
   can the `Update` add the result?  If we will include in the returned `TxnData` then we could add a new TxnDataName.CONDITION_MET or something like that...  only concern with that is what happens when we add multi condition support



##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   we use `int32` in other contexts, maybe `readUnsignedVInt32`?



##########
src/java/org/apache/cassandra/service/accord/txn/AccordUpdateParameters.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+
+public class AccordUpdateParameters
+{
+    private final TxnData data;
+    private final QueryOptions options;
+
+    public AccordUpdateParameters(TxnData data, QueryOptions options)
+    {
+        this.data = data;
+        this.options = options;
+    }
+
+    public TxnData getData()
+    {
+        return data;
+    }
+
+    public UpdateParameters updateParameters(TableMetadata metadata, int rowIndex)
+    {
+        // This is currently only used by Guardrails, but this logically have issues with Accord as drifts in config
+        // values could cause unexpected issues in Accord. (ex. some nodes reject writes while others accept)
+        // For the time being, guardrails are disabled for Accord queries.
+        ClientState disabledGuardrails = null;
+
+        // What we use here doesn't matter as they get replaced before actually performing the write.
+        // see org.apache.cassandra.service.accord.txn.TxnWrite.Update.write
+        int nowInSeconds = 42;
+        long timestamp = nowInSeconds;
+
+        // TODO: How should Accord work with TTL?
+        int ttl = metadata.params.defaultTimeToLive;
+        return new UpdateParameters(metadata,
+                                    disabledGuardrails,
+                                    options,
+                                    timestamp,
+                                    nowInSeconds,
+                                    ttl,
+                                    prefetchRow(metadata, rowIndex));
+    }
+
+    private Map<DecoratedKey, Partition> prefetchRow(TableMetadata metadata, int index)

Review Comment:
   `index` is the `write` index, so this implies that our `AUTO_READ` are `table + write_ index`?  This can be a problem if you do an update to the same partition multiple times; we could read once but read multiple
   
   ```
   -- does this really read `pk=0` twice?
   UPDATE table SET a += 1 WHERE pk=0;
   UPDATE table SET b -= 1 WHERE pk=0;
   ```



##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   > Do we want to audit/change all of those in this PR
   
   To avoid conflicts with rebasing on trunk im in favor of only updating what accord touches



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }

Review Comment:
   we need to serialize, so we need to make sure we can send cross nodes; `Object[]` could work, but serializer would be more annoying
   
   cool with that if you feel best



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040085314


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1855,6 +1855,11 @@ public static long getCasContentionTimeout(TimeUnit unit)
         return conf.cas_contention_timeout.to(unit);
     }
 
+    public static long getTransactionTimeout(TimeUnit unit)

Review Comment:
   Sure, can add it. While I'm at it, can also add this to `DD#checkForLowestAcceptedTimeouts()`...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040145165


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+                return executeWithRetry(cluster, check, boundValues);

Review Comment:
   Also, at some point in the future, I think this pre-emption checking might become unnecessary anyway. (i.e. Unified Log might make it unnecessary.) CC @bdeggleston 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040117215


##########
src/java/org/apache/cassandra/utils/CollectionSerializers.java:
##########
@@ -19,41 +19,43 @@
 package org.apache.cassandra.utils;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.RandomAccess;
 import java.util.Set;
 import java.util.function.IntFunction;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public class CollectionSerializer
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class CollectionSerializers
 {
 
-    public static <V> void serializeCollection(IVersionedSerializer<V> valueSerializer, Collection<V> values, DataOutputPlus out, int version) throws IOException
+    public static <V> void serializeCollection(Collection<V> values, DataOutputPlus out, int version, IVersionedSerializer<V> valueSerializer) throws IOException
     {
         out.writeUnsignedVInt(values.size());
         for (V value : values)
             valueSerializer.serialize(value, out, version);
     }
 
-    public static <V, L extends List<V> & RandomAccess> void serializeList(IVersionedSerializer<V> valueSerializer, L values, DataOutputPlus out, int version) throws IOException

Review Comment:
   (I agree we shouldn’t simply invoke the collection variant)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041810968


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)

Review Comment:
   Do we want to subject our select, if it exists, to the Guardrails system?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039946762


##########
src/antlr/Parser.g:
##########
@@ -73,6 +80,19 @@ options {
         return marker;
     }
 
+    public RowDataReference.Raw newRowDataReference(Selectable.RawIdentifier tuple, Selectable.Raw selectable)
+    {
+        if (!isParsingTxn)
+            throw new IllegalStateException();

Review Comment:
   I should also probably make this a `SyntaxException` while I'm at it...how about this?
   
   ```
   throw new SyntaxException("Cannot create a row data reference unless parsing a transaction");
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz closed pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz closed pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)
URL: https://github.com/apache/cassandra/pull/1962


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042513395


##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+

Review Comment:
   I should have just used V3 here, since that's what everything else does w/ collections. Given that's the case, it seems like we should just introduce a "default collection serialization version" or something somewhere, but I'd need to see how many more files this patch would touch...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041496658


##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   Looks like this is being handled in [CASSANDRA-18099](https://issues.apache.org/jira/browse/CASSANDRA-18099)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042720633


##########
src/antlr/Parser.g:
##########
@@ -651,6 +717,102 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     | d=deleteStatement  { $statement = d; }
     ;
 
+/**
+ * ex. conditional update returning pre-update values
+ *
+ * BEGIN TRANSACTION
+ *   LET row1 = (SELECT * FROM <table> WHERE k=1 AND c=2);
+ *   LET row2 = (SELECT * FROM <table> WHERE k=2 AND c=2);
+ *   SELECT row1.v, row2.v;
+ *   IF row1.v = 3 AND row2.v = 4 THEN
+ *     UPDATE <table> SET v = row1.v + 1 WHERE k = 1 AND c = 2;
+ *   END IF
+ * COMMIT TRANSACTION
+ *
+ * ex. read-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   SELECT * FROM <table> WHERE k=1 AND c=2;
+ * COMMIT TRANSACTION
+ *
+ * ex. write-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   INSERT INTO <table> (k, c, v) VALUES (0, 0, 1);
+ * COMMIT TRANSACTION
+ */
+batchTxnStatement returns [TransactionStatement.Parsed expr]
+    @init {
+        isParsingTxn = true;
+        List<SelectStatement.RawStatement> assignments = new ArrayList<>();
+        SelectStatement.RawStatement select = null;
+        List<RowDataReference.Raw> returning = null;
+        List<ModificationStatement.Parsed> updates = new ArrayList<>();
+    }
+    : K_BEGIN K_TRANSACTION
+      ( let=letStatement ';' { assignments.add(let); })*
+      ( ( (selectStatement) => s=selectStatement ';' { select = s; }) | ( K_SELECT drs=rowDataReferences ';' { returning = drs; }) )?
+      ( K_IF conditions=txnConditions K_THEN { isTxnConditional = true; } )?
+      ( upd=batchStatementObjective ';' { updates.add(upd); } )*
+      ( {!isTxnConditional}? (K_COMMIT K_TRANSACTION) | {isTxnConditional}? (K_END K_IF K_COMMIT K_TRANSACTION))
+    {
+        $expr = new TransactionStatement.Parsed(assignments, select, returning, updates, conditions, references);
+    }
+    ;
+    finally { isParsingTxn = false; }
+
+rowDataReferences returns [List<RowDataReference.Raw> refs]
+    : r1=rowDataReference { refs = new ArrayList<RowDataReference.Raw>(); refs.add(r1); } (',' rN=rowDataReference { refs.add(rN); })*
+    ;
+
+rowDataReference returns [RowDataReference.Raw rawRef]
+    @init { Selectable.RawIdentifier tuple = null; Selectable.Raw selectable = null; }
+    @after { $rawRef = newRowDataReference(tuple, selectable); }
+    : t=sident ('.' s=referenceSelection)? { tuple = t; selectable = s; }
+    ;
+
+referenceSelection returns [Selectable.Raw s]
+    : g=referenceSelectionWithoutField m=selectorModifier[g] {$s = m;}
+    ;
+
+referenceSelectionWithoutField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : sn=sident  { tmp=sn; }
+    | (selectionTypeHint)=> h=selectionTypeHint { tmp=h; }
+    | t=selectionTupleOrNestedSelector { tmp=t; }
+    | l=selectionList { tmp=l; }
+    | m=selectionMapOrSet { tmp=m; }
+    // UDTs are equivalent to maps from the syntax point of view, so the final decision will be done in Selectable.WithMapOrUdt
+    ;
+
+txnConditions returns [List<ConditionStatement.Raw> conditions]
+    @init { conditions = new ArrayList<ConditionStatement.Raw>(); }
+    : txnColumnCondition[conditions] ( K_AND txnColumnCondition[conditions] )*

Review Comment:
   spoke in slack; pushed out of v1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042488468


##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -160,18 +169,30 @@ protected ModificationStatement prepareInternal(TableMetadata metadata,
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
 
-                Term.Raw value = columnValues.get(i);
+                Object value = columnValues.get(i);
 
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, value));
+                    checkTrue(value instanceof Term.Raw, "value references can't be used with primary key columns");
+                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, (Term.Raw) value));
                 }
-                else
+                else if (value instanceof ReferenceValue.Raw)
+                {
+                    ReferenceValue.Raw raw = (ReferenceValue.Raw) value;
+                    ReferenceValue referenceValue = raw.prepare(def, bindVariables);
+                    ReferenceOperation operation = new ReferenceOperation(def, TxnReferenceOperation.Kind.setterFor(def), null, null, referenceValue);
+                    operations.add(def, operation);
+                }
+                else if (value instanceof Term.Raw)
                 {
-                    Operation operation = new Operation.SetValue(value).prepare(metadata, def, !conditions.isEmpty());
+                    Operation operation = new Operation.SetValue((Term.Raw) value).prepare(metadata, def, !conditions.isEmpty());
                     operation.collectMarkerSpecification(bindVariables);
                     operations.add(operation);
                 }
+                else
+                {
+                    throw new IllegalStateException();

Review Comment:
   Might not even be necessary if we switch to `Term.Raw` above...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041821810


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));

Review Comment:
   Yeah, `getOnlyElement()` certainly doesn't have a helpful message if somehow we have multiple queries. What if we catch that case and throw up a more txn-specific error message?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041520301


##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -365,17 +389,23 @@ public Substraction(Term.Raw value)
         }
 
         public Operation prepare(TableMetadata metadata, ColumnMetadata receiver, boolean canReadExistingState) throws InvalidRequestException
-        {
+        {   
             if (!(receiver.type instanceof CollectionType))
             {
-                if (!(receiver.type instanceof CounterColumnType))
+                if (canReadExistingState)
+                {
+                    if (!(receiver.type instanceof NumberType))

Review Comment:
   Pulling up to the outer `if` slightly changes the behavior though. If the inner `if` fails, you need to fall through to `return`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042737283


##########
src/java/org/apache/cassandra/cql3/transactions/ConditionStatement.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+
+public class ConditionStatement
+{
+    public enum Kind
+    {
+        IS_NOT_NULL(TxnCondition.Kind.IS_NOT_NULL),
+        IS_NULL(TxnCondition.Kind.IS_NULL),
+        EQ(TxnCondition.Kind.EQUAL),
+        NEQ(TxnCondition.Kind.NOT_EQUAL),
+        GT(TxnCondition.Kind.GREATER_THAN),
+        GTE(TxnCondition.Kind.GREATER_THAN_OR_EQUAL),
+        LT(TxnCondition.Kind.LESS_THAN),
+        LTE(TxnCondition.Kind.LESS_THAN_OR_EQUAL);
+        
+        // TODO: Support for IN, CONTAINS, CONTAINS KEY
+
+        private final TxnCondition.Kind kind;
+        
+        Kind(TxnCondition.Kind kind)
+        {
+            this.kind = kind;
+        }
+
+        TxnCondition.Kind toTxnKind()
+        {
+            return kind;
+        }
+    }
+
+    private final RowDataReference reference;
+    private final Kind kind;
+    private final Term value;
+
+    public ConditionStatement(RowDataReference reference, Kind kind, Term value)
+    {
+        this.reference = reference;
+        this.kind = kind;
+        this.value = value;
+    }
+
+    public static class Raw
+    {
+        private final RowDataReference.Raw reference;
+        private final Kind kind;
+        private final Term.Raw value;
+
+        public Raw(RowDataReference.Raw reference, Kind kind, Term.Raw value)
+        {
+            Preconditions.checkArgument(reference != null);
+            Preconditions.checkArgument((value == null) == (kind == Kind.IS_NOT_NULL || kind == Kind.IS_NULL));
+            this.reference = reference;
+            this.kind = kind;
+            this.value = value;
+        }
+
+        public ConditionStatement prepare(String keyspace, VariableSpecifications bindVariables)
+        {
+            RowDataReference preparedReference = reference.prepareAsReceiver();
+            preparedReference.collectMarkerSpecification(bindVariables);
+            Term preparedValue = null;
+
+            if (value != null)
+            {
+                ColumnSpecification receiver = preparedReference.column();
+                
+                if (preparedReference.isElementSelection())
+                {
+                    switch (((CollectionType<?>) receiver.type).kind)
+                    {
+                        case LIST:
+                            receiver = Lists.valueSpecOf(receiver);
+                            break;
+                        case MAP:
+                            receiver = Maps.valueSpecOf(receiver);
+                            break;
+                        case SET:
+                            throw new InvalidRequestException(String.format("Invalid operation %s = %s for set column %s",

Review Comment:
   should have a default?  this is exhaustive but things can always change



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,442 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()

Review Comment:
   this is dead code



##########
src/java/org/apache/cassandra/service/accord/txn/TxnAppliedQuery.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+import accord.api.Data;
+import accord.api.Query;
+import accord.api.Read;
+import accord.api.Result;
+import accord.api.Update;
+import accord.primitives.TxnId;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserialize;
+import static org.apache.cassandra.service.accord.AccordSerializers.serialize;
+
+// TODO: This is currently unused, but we might want to use it to support returning the condition result.
+public class TxnAppliedQuery implements Query

Review Comment:
   should we remove as Ariel has his own version?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserializeCqlCollectionAsTerm;
+import static org.apache.cassandra.utils.CollectionSerializers.deserializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializedListSize;
+
+public abstract class TxnCondition
+{
+    private interface ConditionSerializer<T extends TxnCondition>
+    {
+        void serialize(T condition, DataOutputPlus out, int version) throws IOException;
+        T deserialize(DataInputPlus in, int version, Kind kind) throws IOException;
+        long serializedSize(T condition, int version);
+    }
+
+    public enum Kind
+    {
+        NONE("n/a", null),
+        AND("AND", null),
+        OR("OR", null),
+        IS_NOT_NULL("IS NOT NULL", null),
+        IS_NULL("IS NULL", null),
+        EQUAL("=", Operator.EQ),
+        NOT_EQUAL("!=", Operator.NEQ),
+        GREATER_THAN(">", Operator.GT),
+        GREATER_THAN_OR_EQUAL(">=", Operator.GTE),
+        LESS_THAN("<", Operator.LT),
+        LESS_THAN_OR_EQUAL("<=", Operator.LTE);
+
+        private final String symbol;
+        private final Operator operator;
+
+        Kind(String symbol, Operator operator)
+        {
+            this.symbol = symbol;
+            this.operator = operator;
+        }
+
+        @SuppressWarnings("rawtypes")
+        private ConditionSerializer serializer()
+        {
+            switch (this)
+            {
+                case IS_NOT_NULL:
+                case IS_NULL:
+                    return Exists.serializer;
+                case EQUAL:
+                case NOT_EQUAL:
+                case LESS_THAN:
+                case LESS_THAN_OR_EQUAL:
+                case GREATER_THAN:
+                case GREATER_THAN_OR_EQUAL:
+                    return Value.serializer;
+                case AND:
+                case OR:
+                    return BooleanGroup.serializer;
+                case NONE:
+                    return None.serializer;
+                default:
+                    throw new IllegalArgumentException();

Review Comment:
   useful message plz!



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),

Review Comment:
   nit: `ListAppender`?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),
+        Putter((byte) 7, (column, keyOrIndex, field, value) -> new Maps.Putter(column, value)),
+        ListSetter((byte) 8, (column, keyOrIndex, field, value) -> new Lists.Setter(column, value)),
+        SetSetter((byte) 9, (column, keyOrIndex, field, value) -> new Sets.Setter(column, value)),
+        MapSetter((byte) 10, (column, keyOrIndex, field, value) -> new Maps.Setter(column, value)),
+        UserTypeSetter((byte) 11, (column, keyOrIndex, field, value) -> new UserTypes.Setter(column, value)),
+        ConstantSetter((byte) 12, (column, keyOrIndex, field, value) -> new Constants.Setter(column, value)),
+        Subtracter((byte) 13, (column, keyOrIndex, field, value) -> new Constants.Substracter(column, value)),

Review Comment:
   nit: ConstantSubtracter?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnCondition.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserializeCqlCollectionAsTerm;
+import static org.apache.cassandra.utils.CollectionSerializers.deserializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializeList;
+import static org.apache.cassandra.utils.CollectionSerializers.serializedListSize;
+
+public abstract class TxnCondition
+{
+    private interface ConditionSerializer<T extends TxnCondition>
+    {
+        void serialize(T condition, DataOutputPlus out, int version) throws IOException;
+        T deserialize(DataInputPlus in, int version, Kind kind) throws IOException;
+        long serializedSize(T condition, int version);
+    }
+
+    public enum Kind
+    {
+        NONE("n/a", null),
+        AND("AND", null),
+        OR("OR", null),
+        IS_NOT_NULL("IS NOT NULL", null),
+        IS_NULL("IS NULL", null),
+        EQUAL("=", Operator.EQ),
+        NOT_EQUAL("!=", Operator.NEQ),
+        GREATER_THAN(">", Operator.GT),
+        GREATER_THAN_OR_EQUAL(">=", Operator.GTE),
+        LESS_THAN("<", Operator.LT),
+        LESS_THAN_OR_EQUAL("<=", Operator.LTE);
+
+        private final String symbol;
+        private final Operator operator;
+
+        Kind(String symbol, Operator operator)
+        {
+            this.symbol = symbol;
+            this.operator = operator;
+        }
+
+        @SuppressWarnings("rawtypes")
+        private ConditionSerializer serializer()
+        {
+            switch (this)
+            {
+                case IS_NOT_NULL:
+                case IS_NULL:
+                    return Exists.serializer;
+                case EQUAL:
+                case NOT_EQUAL:
+                case LESS_THAN:
+                case LESS_THAN_OR_EQUAL:
+                case GREATER_THAN:
+                case GREATER_THAN_OR_EQUAL:
+                    return Value.serializer;
+                case AND:
+                case OR:
+                    return BooleanGroup.serializer;
+                case NONE:
+                    return None.serializer;
+                default:
+                    throw new IllegalArgumentException();
+            }
+        }
+    }
+
+    protected final Kind kind;
+
+    public TxnCondition(Kind kind)
+    {
+        this.kind = kind;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnCondition condition = (TxnCondition) o;
+        return kind == condition.kind;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(kind);
+    }
+
+    public Kind kind()
+    {
+        return kind;
+    }
+
+    public abstract boolean applies(TxnData data);
+
+    private static class None extends TxnCondition
+    {
+        private static final None instance = new None();
+
+        private None()
+        {
+            super(Kind.NONE);
+        }
+
+        @Override
+        public String toString()
+        {
+            return kind.toString();
+        }
+
+        @Override
+        public boolean applies(TxnData data)
+        {
+            return true;
+        }
+
+        private static final ConditionSerializer<None> serializer = new ConditionSerializer<None>()
+        {
+            @Override
+            public void serialize(None condition, DataOutputPlus out, int version) {}
+            @Override
+            public None deserialize(DataInputPlus in, int version, Kind kind) { return instance; }
+            @Override
+            public long serializedSize(None condition, int version) { return 0; }
+        };
+    }
+
+    public static TxnCondition none()
+    {
+        return None.instance;
+    }
+
+    public static class Exists extends TxnCondition
+    {
+        private static final Set<Kind> KINDS = ImmutableSet.of(Kind.IS_NOT_NULL, Kind.IS_NULL);
+
+        public final TxnReference reference;
+
+        public Exists(TxnReference reference, Kind kind)
+        {
+            super(kind);
+            Preconditions.checkArgument(KINDS.contains(kind));

Review Comment:
   useful error plz



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);

Review Comment:
   parser drops the `AND` and stores everything into an array, so this code matches parser...
   
   We spoke and `OR` and `NOT` are not part of v1, so this is fine for now.



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));

Review Comment:
   added
   
   ```
   if (selectQuery.queries.size() != 1)
               throw new IllegalArgumentException("When running within a transaction, select statements may only select a single partition; found " + selectQuery.queries.size() + " partitions");
   ```
   
   I still call `Iterables.getOnlyElement` but since we already validated `size=1` that should be fine



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));

Review Comment:
   for now im cool with us ignore... 



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));

Review Comment:
   that's where we would actually get the keyspace name from, correct?
   
   it gets tricky... 
   
   ```
   IF fancy(a)
   ```
   
   that should work IMO IFF `fancy` is in the same keyspace as `a`, so parser wouldn't know this, `prepare` would need to handle/solve



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),
+        Putter((byte) 7, (column, keyOrIndex, field, value) -> new Maps.Putter(column, value)),
+        ListSetter((byte) 8, (column, keyOrIndex, field, value) -> new Lists.Setter(column, value)),
+        SetSetter((byte) 9, (column, keyOrIndex, field, value) -> new Sets.Setter(column, value)),
+        MapSetter((byte) 10, (column, keyOrIndex, field, value) -> new Maps.Setter(column, value)),
+        UserTypeSetter((byte) 11, (column, keyOrIndex, field, value) -> new UserTypes.Setter(column, value)),
+        ConstantSetter((byte) 12, (column, keyOrIndex, field, value) -> new Constants.Setter(column, value)),
+        Subtracter((byte) 13, (column, keyOrIndex, field, value) -> new Constants.Substracter(column, value)),
+        SetterByKey((byte) 14, (column, keyOrIndex, field, value) -> new Maps.SetterByKey(column, keyOrIndex, value)),

Review Comment:
   nit: MapSetterByKey?



##########
src/java/org/apache/cassandra/cql3/transactions/RowDataReference.java:
##########
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.types.utils.Bytes;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+
+public class RowDataReference extends Term.NonTerminal
+{
+    public static final String CANNOT_FIND_TUPLE_MESSAGE = "Cannot resolve reference to tuple '%s'.";
+    public static final String COLUMN_NOT_IN_TUPLE_MESSAGE = "Column '%s' does not exist in tuple '%s'.";
+
+    private final TxnDataName selectName;
+    private final ColumnMetadata column;
+    private final Term elementPath;
+    private final CellPath fieldPath;
+    
+    public RowDataReference(TxnDataName selectName, ColumnMetadata column, Term elementPath, CellPath fieldPath)
+    {
+        Preconditions.checkArgument(elementPath == null || fieldPath == null, "Cannot specify both element and field paths");
+        
+        this.selectName = selectName;
+        this.column = column;
+        this.elementPath = elementPath;
+        this.fieldPath = fieldPath;
+    }
+
+    @Override
+    public void collectMarkerSpecification(VariableSpecifications boundNames)
+    {
+        if (elementPath != null)
+            elementPath.collectMarkerSpecification(boundNames);
+    }
+
+    @Override
+    public Terminal bind(QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean containsBindMarker()
+    {
+        return elementPath != null && elementPath.containsBindMarker();
+    }
+
+    @Override
+    public void addFunctionsTo(List<Function> functions)
+    {
+        throw new UnsupportedOperationException("Functions are not currently supported w/ reference terms.");
+    }
+
+    public ColumnMetadata toResultMetadata()
+    {
+        ColumnIdentifier fullName = getFullyQualifiedName();
+        ColumnMetadata forMetadata = column.withNewName(fullName);
+
+        if (isElementSelection())
+        {
+            if (forMetadata.type instanceof SetType)
+                forMetadata = forMetadata.withNewType(((SetType<?>) forMetadata.type).nameComparator());
+            else if (forMetadata.type instanceof MapType)

Review Comment:
   no list?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),

Review Comment:
   nit: ListPrepender?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),
+        Putter((byte) 7, (column, keyOrIndex, field, value) -> new Maps.Putter(column, value)),

Review Comment:
   nit: MapPutter?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),
+        Putter((byte) 7, (column, keyOrIndex, field, value) -> new Maps.Putter(column, value)),
+        ListSetter((byte) 8, (column, keyOrIndex, field, value) -> new Lists.Setter(column, value)),
+        SetSetter((byte) 9, (column, keyOrIndex, field, value) -> new Sets.Setter(column, value)),
+        MapSetter((byte) 10, (column, keyOrIndex, field, value) -> new Maps.Setter(column, value)),
+        UserTypeSetter((byte) 11, (column, keyOrIndex, field, value) -> new UserTypes.Setter(column, value)),
+        ConstantSetter((byte) 12, (column, keyOrIndex, field, value) -> new Constants.Setter(column, value)),
+        Subtracter((byte) 13, (column, keyOrIndex, field, value) -> new Constants.Substracter(column, value)),
+        SetterByKey((byte) 14, (column, keyOrIndex, field, value) -> new Maps.SetterByKey(column, keyOrIndex, value)),
+        SetterByIndex((byte) 15, (column, keyOrIndex, field, value) -> new Lists.SetterByIndex(column, keyOrIndex, value)),

Review Comment:
   nit: ListSetterByIndex?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));

Review Comment:
   we changed `org.apache.cassandra.cql3.statements.ModificationStatement#getTxnUpdate` to avoid calling `Iterables.getOnlyElement` for this reason... I think its cleaner to just check the size rather than try/catch
   
   ill put in my feedback branch



##########
src/java/org/apache/cassandra/service/accord/txn/TxnReferenceOperation.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.AccordSerializers;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.service.accord.AccordSerializers.columnMetadataSerializer;
+
+public class TxnReferenceOperation
+{
+    private static final Map<Class<? extends Operation>, Kind> operationKindMap = initOperationKindMap();
+    
+    private static Map<Class<? extends Operation>, Kind> initOperationKindMap()
+    {
+        Map<Class<? extends Operation>, Kind> temp = new HashMap<>();
+        temp.put(Sets.Adder.class, Kind.SetAdder);
+        temp.put(Constants.Adder.class, Kind.ConstantAdder);
+        temp.put(Lists.Appender.class, Kind.Appender);
+        temp.put(Sets.Discarder.class, Kind.SetDiscarder);
+        temp.put(Lists.Discarder.class, Kind.ListDiscarder);
+        temp.put(Lists.Prepender.class, Kind.Prepender);
+        temp.put(Maps.Putter.class, Kind.Putter);
+        temp.put(Lists.Setter.class, Kind.ListSetter);
+        temp.put(Sets.Setter.class, Kind.SetSetter);
+        temp.put(Maps.Setter.class, Kind.MapSetter);
+        temp.put(UserTypes.Setter.class, Kind.UserTypeSetter);
+        temp.put(Constants.Setter.class, Kind.ConstantSetter);
+        temp.put(Constants.Substracter.class, Kind.Subtracter);
+        temp.put(Maps.SetterByKey.class, Kind.SetterByKey);
+        temp.put(Lists.SetterByIndex.class, Kind.SetterByIndex);
+        temp.put(UserTypes.SetterByField.class, Kind.SetterByField);
+        return temp;
+    }
+
+    private interface ToOperation
+    {
+        Operation apply(ColumnMetadata column, Term keyOrIndex, FieldIdentifier field, Term value);
+    }
+
+    public enum Kind
+    {
+        SetAdder((byte) 1, (column, keyOrIndex, field, value) -> new Sets.Adder(column, value)),
+        ConstantAdder((byte) 2, (column, keyOrIndex, field, value) -> new Constants.Adder(column, value)),
+        Appender((byte) 3, (column, keyOrIndex, field, value) -> new Lists.Appender(column, value)),
+        SetDiscarder((byte) 4, (column, keyOrIndex, field, value) -> new Sets.Discarder(column, value)),
+        ListDiscarder((byte) 5, (column, keyOrIndex, field, value) -> new Lists.Discarder(column, value)),
+        Prepender((byte) 6, (column, keyOrIndex, field, value) -> new Lists.Prepender(column, value)),
+        Putter((byte) 7, (column, keyOrIndex, field, value) -> new Maps.Putter(column, value)),
+        ListSetter((byte) 8, (column, keyOrIndex, field, value) -> new Lists.Setter(column, value)),
+        SetSetter((byte) 9, (column, keyOrIndex, field, value) -> new Sets.Setter(column, value)),
+        MapSetter((byte) 10, (column, keyOrIndex, field, value) -> new Maps.Setter(column, value)),
+        UserTypeSetter((byte) 11, (column, keyOrIndex, field, value) -> new UserTypes.Setter(column, value)),
+        ConstantSetter((byte) 12, (column, keyOrIndex, field, value) -> new Constants.Setter(column, value)),
+        Subtracter((byte) 13, (column, keyOrIndex, field, value) -> new Constants.Substracter(column, value)),
+        SetterByKey((byte) 14, (column, keyOrIndex, field, value) -> new Maps.SetterByKey(column, keyOrIndex, value)),
+        SetterByIndex((byte) 15, (column, keyOrIndex, field, value) -> new Lists.SetterByIndex(column, keyOrIndex, value)),
+        SetterByField((byte) 16, (column, keyOrIndex, field, value) -> new UserTypes.SetterByField(column, field, value));

Review Comment:
   nit: UserTypeSetterByField?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));

Review Comment:
   since writes don't share the same auto read, this isn't an issue; this is because `partitionRead` includes the write `idx`; this does cause multiple reads to the same partition... so we could be smarter in some cases, but in cases where clustering don't match, we need to isolate still.



##########
src/java/org/apache/cassandra/service/accord/txn/AccordUpdateParameters.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+
+public class AccordUpdateParameters
+{
+    private final TxnData data;
+    private final QueryOptions options;
+
+    public AccordUpdateParameters(TxnData data, QueryOptions options)
+    {
+        this.data = data;
+        this.options = options;
+    }
+
+    public TxnData getData()
+    {
+        return data;
+    }
+
+    public UpdateParameters updateParameters(TableMetadata metadata, int rowIndex)
+    {
+        // This is currently only used by Guardrails, but this logically have issues with Accord as drifts in config
+        // values could cause unexpected issues in Accord. (ex. some nodes reject writes while others accept)
+        // For the time being, guardrails are disabled for Accord queries.
+        ClientState disabledGuardrails = null;
+
+        // What we use here doesn't matter as they get replaced before actually performing the write.
+        // see org.apache.cassandra.service.accord.txn.TxnWrite.Update.write
+        int nowInSeconds = 42;
+        long timestamp = nowInSeconds;
+
+        // TODO: How should Accord work with TTL?
+        int ttl = metadata.params.defaultTimeToLive;
+        return new UpdateParameters(metadata,
+                                    disabledGuardrails,
+                                    options,
+                                    timestamp,
+                                    nowInSeconds,
+                                    ttl,
+                                    prefetchRow(metadata, rowIndex));
+    }
+
+    private Map<DecoratedKey, Partition> prefetchRow(TableMetadata metadata, int index)

Review Comment:
   marking resolved.  this works now because of `index` but makes same row mutations do extra queries, which kinda sucks... correct > performance so cool with this for now.



##########
src/java/org/apache/cassandra/cql3/transactions/RowDataReference.java:
##########
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.types.utils.Bytes;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+
+public class RowDataReference extends Term.NonTerminal
+{
+    public static final String CANNOT_FIND_TUPLE_MESSAGE = "Cannot resolve reference to tuple '%s'.";
+    public static final String COLUMN_NOT_IN_TUPLE_MESSAGE = "Column '%s' does not exist in tuple '%s'.";
+
+    private final TxnDataName selectName;
+    private final ColumnMetadata column;
+    private final Term elementPath;
+    private final CellPath fieldPath;
+    
+    public RowDataReference(TxnDataName selectName, ColumnMetadata column, Term elementPath, CellPath fieldPath)
+    {
+        Preconditions.checkArgument(elementPath == null || fieldPath == null, "Cannot specify both element and field paths");
+        
+        this.selectName = selectName;
+        this.column = column;
+        this.elementPath = elementPath;
+        this.fieldPath = fieldPath;
+    }
+
+    @Override
+    public void collectMarkerSpecification(VariableSpecifications boundNames)
+    {
+        if (elementPath != null)
+            elementPath.collectMarkerSpecification(boundNames);
+    }
+
+    @Override
+    public Terminal bind(QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean containsBindMarker()
+    {
+        return elementPath != null && elementPath.containsBindMarker();
+    }
+
+    @Override
+    public void addFunctionsTo(List<Function> functions)
+    {
+        throw new UnsupportedOperationException("Functions are not currently supported w/ reference terms.");
+    }
+
+    public ColumnMetadata toResultMetadata()
+    {
+        ColumnIdentifier fullName = getFullyQualifiedName();
+        ColumnMetadata forMetadata = column.withNewName(fullName);
+
+        if (isElementSelection())
+        {
+            if (forMetadata.type instanceof SetType)
+                forMetadata = forMetadata.withNewType(((SetType<?>) forMetadata.type).nameComparator());
+            else if (forMetadata.type instanceof MapType)

Review Comment:
   fixed this in my feedback branch, this breaks return `SELECT`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042588968


##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+
+        throw new UnsupportedOperationException("Unsupported collection type: " + type);
+    }
+
+    public static final IVersionedSerializer<PartitionUpdate> partitionUpdateSerializer = new IVersionedSerializer<PartitionUpdate>()
+    {
+        @Override
+        public void serialize(PartitionUpdate upd, DataOutputPlus out, int version) throws IOException
+        {
+            PartitionUpdate.serializer.serialize(upd, out, version);
+        }
+
+        @Override
+        public PartitionUpdate deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return PartitionUpdate.serializer.deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE);
+        }
+
+        @Override
+        public long serializedSize(PartitionUpdate upd, int version)
+        {
+            return PartitionUpdate.serializer.serializedSize(upd, version);
+        }
+    };
+
+    public static final IVersionedSerializer<SinglePartitionReadCommand> singlePartitionReadCommandSerializer = new IVersionedSerializer<SinglePartitionReadCommand>()
+    {
+        @Override
+        public void serialize(SinglePartitionReadCommand command, DataOutputPlus out, int version) throws IOException
+        {
+            SinglePartitionReadCommand.serializer.serialize(command, out, version);
+        }
+
+        @Override
+        public SinglePartitionReadCommand deserialize(DataInputPlus in, int version) throws IOException
+        {
+            return (SinglePartitionReadCommand) SinglePartitionReadCommand.serializer.deserialize(in, version);
+        }
+
+        @Override
+        public long serializedSize(SinglePartitionReadCommand command, int version)
+        {
+            return SinglePartitionReadCommand.serializer.serializedSize(command, version);
+        }
+    };
+
+    public static final IVersionedSerializer<ColumnMetadata> columnMetadataSerializer = new IVersionedSerializer<ColumnMetadata>()

Review Comment:
   The only thing I'm a little worried about w/ `columnMetadataSerializer` and `tableMetadataSerializer` going into their respective classes is that they aren't "simple" in the sense that they rely on `Schema.instance.getTableMetadata()`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041851488


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));

Review Comment:
   The grammar doesn't support this at present, and if it did, that's where we would actually get the keyspace name from, correct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041776784


##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -386,7 +416,7 @@ else if (!(receiver.type.isMultiCell()))
                     ColumnSpecification vr = new ColumnSpecification(receiver.ksName,
                                                                      receiver.cfName,
                                                                      receiver.name,
-                                                                     SetType.getInstance(((MapType)receiver.type).getKeysType(), false));
+                                                                     SetType.getInstance(((MapType<?, ?>) receiver.type).getKeysType(), true));

Review Comment:
   I think you're right. No idea why it changed...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038390751


##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +517,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()

Review Comment:
   why does this say you changed this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041267724


##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   `readUnsignedVInt32` sounds like a winner



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040074418


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -1171,8 +1173,9 @@ public static void log(Config config)
             String value;
             try
             {
-                // Field.get() can throw NPE if the value of the field is null
-                value = field.get(config).toString();
+                // don't use exceptions for normal control flow!
+                Object obj = field.get(config);
+                value = obj != null ? obj.toString() : "null";

Review Comment:
   `Field#get()` throws an NPE, supposedly, if `obj` is `null` and we're looking at an instance `Field`. The `Config` `obj` passed in here should never be `null`, at least as I read it, and we now explicitly check `obj`, so removing the `NPE` from the `catch` block should be fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040128246


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))

Review Comment:
   Good call...fixing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041790210


##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -386,7 +416,7 @@ else if (!(receiver.type.isMultiCell()))
                     ColumnSpecification vr = new ColumnSpecification(receiver.ksName,
                                                                      receiver.cfName,
                                                                      receiver.name,
-                                                                     SetType.getInstance(((MapType)receiver.type).getKeysType(), false));
+                                                                     SetType.getInstance(((MapType<?, ?>) receiver.type).getKeysType(), true));

Review Comment:
   Short story: `testMultiCellMapSubtraction` will fail if you revert
   
   Long story: If we create a frozen `SetType`, `vr` will eventually make it to `testAssignment()` in `RowDataReference`, which will delegate to `AbstractType#testAssignment()`. There, if we need to, we'll freeze the receiver if necessary, but not do the opposite.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041799741


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()

Review Comment:
   Does `SimpleSelection` require a single table...and we may have more than one source table when building our result in `TransactionStatement#execute()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042513395


##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+

Review Comment:
   I should have just used V3 here, since that's what everything else does w/ collections. Given that's the case, it seems like we should just introduce a "default collection serialization version" or something somewhere, but I'd need to see how many more files this patch would touch...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1043719684


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }

Review Comment:
   can improve later, resolving



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038360502


##########
src/antlr/Cql.g:
##########
@@ -43,6 +43,7 @@ import Parser,Lexer;
     import org.apache.cassandra.cql3.selection.*;
     import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.cql3.statements.schema.*;
+    import org.apache.cassandra.cql3.transactions.*;

Review Comment:
   unused import?



##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +517,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()

Review Comment:
   why does this say you changed this?



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();

Review Comment:
   should this by try/finally?



##########
test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java:
##########
@@ -154,20 +154,34 @@ public static void processCommandResult(AccordCommandStore commandStore, Command
                                         throw new RuntimeException(e);
                                     }
                                 })
-                                .reduce(null, AccordData::merge);
+                                .reduce(null, TxnData::merge);
             Write write = txn.update().apply(readData);
             ((AccordCommand)command).setWrites(new Writes(command.executeAt(), txn.keys(), write));
             ((AccordCommand)command).setResult(txn.query().compute(command.txnId(), readData, txn.read(), txn.update()));
         }).get();
     }
 
+    public static Txn createTxn(String query)

Review Comment:
   this is very useful for cases like Simulator which benefits from bypassing TransactionStatement to directly access `TxnData` and all "hidden" state



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+                return executeWithRetry(cluster, check, boundValues);

Review Comment:
   should we add delays?  If so can use `org.apache.cassandra.utils.Retry#retryWithBackoffBlocking(int, java.util.function.Supplier<A>)`



##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -533,6 +551,35 @@ else if (obj instanceof byte[])
             return ByteBuffer.wrap((byte[]) obj);
         else if (obj instanceof ByteBuffer)
             return (ByteBuffer) obj;
+        else if (obj instanceof List)
+        {
+            List<?> list = (List<?>) obj;
+            // convert subtypes to BB
+            List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList());
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return ListType.getInstance(BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Map)
+        {
+            Map<?, ?> map = (Map<?, ?>) obj;
+            // convert subtypes to BB
+            Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
+            for (Map.Entry<?, ?> e : map.entrySet())
+                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));

Review Comment:
   for safety should we validate `put` returns `null`?



##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   since `checkedCast(in.readUnsignedVInt())` is the common case, should we add this to `DataInputPlus`?  `readCheckedUnsignedVInt`?



##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -533,6 +551,35 @@ else if (obj instanceof byte[])
             return ByteBuffer.wrap((byte[]) obj);
         else if (obj instanceof ByteBuffer)
             return (ByteBuffer) obj;
+        else if (obj instanceof List)
+        {
+            List<?> list = (List<?>) obj;
+            // convert subtypes to BB
+            List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList());
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return ListType.getInstance(BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Map)
+        {
+            Map<?, ?> map = (Map<?, ?>) obj;
+            // convert subtypes to BB
+            Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
+            for (Map.Entry<?, ?> e : map.entrySet())
+                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return MapType.getInstance(BytesType.instance, BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Set)
+        {
+            Set<?> set = (Set<?>) obj;
+            // convert subtypes to BB
+            Set<ByteBuffer> bbs = new LinkedHashSet<>();
+            set.forEach(o -> bbs.add(objectToBytes(o)));

Review Comment:
   for safety should we validate `add` returns `true`?



##########
src/java/org/apache/cassandra/service/accord/AccordService.java:
##########
@@ -99,6 +110,39 @@ public static long nowInMicros()
         return TimeUnit.MILLISECONDS.toMicros(Clock.Global.currentTimeMillis());
     }
 
+    public TxnData coordinate(Txn txn)
+    {
+        try
+        {
+            Future<Result> future = node.coordinate(txn);
+            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.SECONDS), TimeUnit.SECONDS);
+            return (TxnData) result;
+        }
+        catch (ExecutionException e)
+        {
+            Throwable cause = e.getCause();
+            if (cause instanceof Timeout)
+                throw throwTimeout(txn);
+            throw new RuntimeException(e);

Review Comment:
   this should throw `cause`?



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -1171,8 +1173,9 @@ public static void log(Config config)
             String value;
             try
             {
-                // Field.get() can throw NPE if the value of the field is null
-                value = field.get(config).toString();
+                // don't use exceptions for normal control flow!
+                Object obj = field.get(config);
+                value = obj != null ? obj.toString() : "null";

Review Comment:
   comment said that `field.get` throws NPE; is that true?  it should return null... should we also remove the `NPE` catch?



##########
src/java/org/apache/cassandra/audit/AuditLogEntryType.java:
##########
@@ -60,6 +60,9 @@
     CREATE_ROLE(AuditLogEntryCategory.DCL),
     USE_KEYSPACE(AuditLogEntryCategory.OTHER),
     DESCRIBE(AuditLogEntryCategory.OTHER),
+    
+    // TODO: Is DML the most appropriate classification, given a transaction can read, write, or both?
+    TRANSACTION(AuditLogEntryCategory.DML),

Review Comment:
   Maybe best to add a new one?  Maybe `TRANSACTION` like we did for `WriteType`?



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -1855,6 +1855,11 @@ public static long getCasContentionTimeout(TimeUnit unit)
         return conf.cas_contention_timeout.to(unit);
     }
 
+    public static long getTransactionTimeout(TimeUnit unit)

Review Comment:
   no setter?



##########
src/antlr/Parser.g:
##########
@@ -73,6 +80,19 @@ options {
         return marker;
     }
 
+    public RowDataReference.Raw newRowDataReference(Selectable.RawIdentifier tuple, Selectable.Raw selectable)
+    {
+        if (!isParsingTxn)
+            throw new IllegalStateException();

Review Comment:
   can we get a useful error msg?



##########
src/java/org/apache/cassandra/utils/CollectionSerializers.java:
##########
@@ -19,41 +19,43 @@
 package org.apache.cassandra.utils;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.RandomAccess;
 import java.util.Set;
 import java.util.function.IntFunction;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public class CollectionSerializer
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class CollectionSerializers
 {
 
-    public static <V> void serializeCollection(IVersionedSerializer<V> valueSerializer, Collection<V> values, DataOutputPlus out, int version) throws IOException
+    public static <V> void serializeCollection(Collection<V> values, DataOutputPlus out, int version, IVersionedSerializer<V> valueSerializer) throws IOException
     {
         out.writeUnsignedVInt(values.size());
         for (V value : values)
             valueSerializer.serialize(value, out, version);
     }
 
-    public static <V, L extends List<V> & RandomAccess> void serializeList(IVersionedSerializer<V> valueSerializer, L values, DataOutputPlus out, int version) throws IOException

Review Comment:
   if you remove `& RandomAccess` shouldn't you just call `serializeCollection` then to avoid copy/paste?  `& RandomAccess` looks to be there to know its safe to do `.get` to avoid the memory overhead of iterators



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;

Review Comment:
   nit: all caps



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+                return executeWithRetry(cluster, check, boundValues);
+
+            throw t;
+        }
+    }
+
+    public static class ByteBuddyHelper

Review Comment:
   should this be renamed to be more explicit?  Maybe something like `EnforceUpdateDoesNotPerformRead`?



##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))

Review Comment:
   FYI there is a `AssertionUtils` class to make working with this easier...
   
   ```
   if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
   ```
   This was written to work with Assert4j but can directly call `matches`
   
   The main reason to prefer these methods is that they try to do the type checks and not just the name check, so `class FluffyKitten extends Preempted` will be detected as well.
   



##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -60,6 +60,23 @@
     private static final Constraint DNS_DOMAIN_PART_CONSTRAINT = Constraint.between(0, DNS_DOMAIN_PART_DOMAIN.length - 1).withNoShrinkPoint();
 
     public static final Gen<String> IDENTIFIER_GEN = Generators.regexWord(SourceDSL.integers().between(1, 50));
+    public static final Gen<String> SYMBOL_GEN = filter(symbolGen(SourceDSL.integers().between(1, 48)), Generators::thisBugIsBroughtToYouByTheLetterP);
+    private static boolean thisBugIsBroughtToYouByTheLetterP(String value)
+    {
+        // In Lexer.g DURATION is before IDENT and Duration allows the following to parsse: P, and PT
+        // This causes an issue for cases that use IDENT as P and PT will not match as they matched DURATION already
+        // to avoid these cases, this function will be used to filter them out so only "valid" symbols are returned
+        // see CASSANDRA-17919
+        return !("P".equals(value) || "PT".equals(value));
+    }
+    private static final char CHAR_UNDERSCORE = 95;
+    public static Gen<String> symbolGen(Gen<Integer> size)
+    {
+        char[] domain = new char[LETTER_OR_DIGIT_DOMAIN.length + 1];
+        System.arraycopy(LETTER_OR_DIGIT_DOMAIN, 0, domain, 0, LETTER_OR_DIGIT_DOMAIN.length);
+        domain[domain.length - 1] = CHAR_UNDERSCORE;

Review Comment:
   nit: it would be good to save this outside this method as this is unchanged and doesn't require reallocating for each call



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);

Review Comment:
   should we also check `isFor`?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;
+        for (String part : parts)
+            size += part.length();

Review Comment:
   should include `String` object overhead right?



##########
src/java/org/apache/cassandra/service/accord/AccordService.java:
##########
@@ -99,6 +110,39 @@ public static long nowInMicros()
         return TimeUnit.MILLISECONDS.toMicros(Clock.Global.currentTimeMillis());
     }
 
+    public TxnData coordinate(Txn txn)
+    {
+        try
+        {
+            Future<Result> future = node.coordinate(txn);
+            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.SECONDS), TimeUnit.SECONDS);

Review Comment:
   nit: if someone does `500ms` this will be incorrect, we should do `TimeUnit.NANOSECONDS` to avoid this issue



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }

Review Comment:
   wondering if it would be best to switch from `String[]` to `ByteBuffer[]` as this adds extra memory cost... this is a hidden internal detail so we can change w/e we want; just something to think about.



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;
+        for (String part : parts)
+            size += part.length();
+        return size;
+    }
+
+    @Override
+    public int compareTo(TxnDataName o)
+    {
+        int rc = kind.compareTo(o.kind);
+        if (rc != 0)
+            return rc;
+        // same kind has same length
+        int size = parts.length;
+        assert o.parts.length == size : String.format("Expected other.parts.length == %d but was %d", size, o.parts.length);
+        for (int i = 0; i < size; i++)
+        {
+            rc = parts[i].compareTo(o.parts[i]);
+            if (rc != 0)
+                return rc;
+        }
+        return 0;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnDataName that = (TxnDataName) o;
+        return kind == that.kind && Arrays.equals(parts, that.parts);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = Objects.hash(kind);
+        result = 31 * result + Arrays.hashCode(parts);
+        return result;
+    }
+
+    public String name()
+    {
+        return String.join(":", parts);
+    }
+
+    @Override
+    public String toString()
+    {
+        return kind.name() + ":" + name();
+    }
+
+    public static final IVersionedSerializer<TxnDataName> serializer = new IVersionedSerializer<TxnDataName>()
+    {
+        @Override
+        public void serialize(TxnDataName t, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeByte(t.kind.value);
+            out.writeInt(t.parts.length);

Review Comment:
   you have ArraySerializer now, should we delegate to that?  that also does `vint` rather than `int` so may also be smaller?



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)

Review Comment:
   TODO: honestly I don't remember why `int index`...



##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;

Review Comment:
   we should include `EMPTY` that we learn from JAMM...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039878746


##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   yeah, this is an issue; the following test fails
   
   ```
   @Test
       public void test() throws Exception
       {
           test(cluster -> {
              String cql = "BEGIN TRANSACTION\n" +
                           "LET row1 = (SELECT * FROM "+currentTable+" WHERE k=0 LIMIT ?);\n" +
                           "SELECT row1.v;\n" +
                           "COMMIT TRANSACTION";
   // I put rand value for return; I just wanted to hit this statement
               assertRowEqualsWithPreemptedRetry(cluster, new Object[] { 1 }, cql, 1);
           });
       }
   ```
   
   ```
   java.lang.IndexOutOfBoundsException: Index: 0
   
   	at java.util.Collections$EmptyList.get(Collections.java:4456)
   	at org.apache.cassandra.cql3.Constants$Marker.bindAndGet(Constants.java:425)
   	at org.apache.cassandra.cql3.statements.SelectStatement.getLimit(SelectStatement.java:840)
   	at org.apache.cassandra.cql3.statements.SelectStatement.getLimit(SelectStatement.java:819)
   	at org.apache.cassandra.cql3.statements.TransactionStatement$Parsed.checkAtMostOneRowSpecified(TransactionStatement.java:434)
   	at org.apache.cassandra.cql3.statements.TransactionStatement$Parsed.prepare(TransactionStatement.java:380)
   	at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:861)
   ```



##########
src/java/org/apache/cassandra/cql3/transactions/ReferenceOperation.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Operation;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.accord.txn.TxnReferenceOperation;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.schema.TableMetadata.UNDEFINED_COLUMN_NAME_MESSAGE;
+
+public class ReferenceOperation
+{
+    private final ColumnMetadata receiver;
+    private final TxnReferenceOperation.Kind kind;
+    private final FieldIdentifier field;
+    private final Term key;
+    private final ReferenceValue value;
+
+    public ReferenceOperation(ColumnMetadata receiver, TxnReferenceOperation.Kind kind, Term key, FieldIdentifier field, ReferenceValue value)
+    {
+        this.receiver = receiver;
+        this.kind = kind;
+        this.key = key;
+        this.field = field;
+        this.value = value;
+    }
+
+    /**
+     * Creates a {@link ReferenceOperation} from the given {@link  Operation} for the purpose of defering execution
+     * within a transaction. When the language sees an Operation using a reference one is created already, but for cases
+     * that needs to defer execution (such as when {@link Operation#requiresRead()} is true), this method can be used.
+     */
+    public static ReferenceOperation create(Operation operation)
+    {
+        TxnReferenceOperation.Kind kind = TxnReferenceOperation.Kind.from(operation);
+        ColumnMetadata receiver = operation.column;
+        ReferenceValue value = new ReferenceValue.Constant(operation.term());
+
+        Term key = extractKeyOrIndex(operation);
+        FieldIdentifier field = extractField(operation);
+        return new ReferenceOperation(receiver, kind, key, field, value);
+    }
+
+    public boolean requiresRead()
+    {
+        // TODO: Find a better way than delegating to the operation?
+        return kind.toOperation(receiver, null, null, null).requiresRead();
+    }
+
+    public TxnReferenceOperation bindAndGet(QueryOptions options)
+    {
+        return new TxnReferenceOperation(kind,
+                                         receiver,
+                                         key != null ? key.bindAndGet(options) : null,
+                                         field != null ? field.bytes : null,
+                                         value.bindAndGet(options));
+    }
+
+    public static class Raw
+    {
+        private final Operation.RawUpdate rawUpdate;
+        public final ColumnIdentifier column;
+        private final ReferenceValue.Raw value;
+
+        public Raw(Operation.RawUpdate rawUpdate, ColumnIdentifier column, ReferenceValue.Raw value)
+        {
+            this.rawUpdate = rawUpdate;
+            this.column = column;
+            this.value = value;
+        }
+
+        public ReferenceOperation prepare(TableMetadata metadata, VariableSpecifications bindVariables)
+        {
+            ColumnMetadata receiver = metadata.getColumn(column);
+            checkTrue(receiver != null, UNDEFINED_COLUMN_NAME_MESSAGE, column.toCQLString(), metadata);
+            AbstractType<?> type = receiver.type;
+
+            Operation operation = rawUpdate.prepare(metadata, receiver, true);
+            TxnReferenceOperation.Kind kind = TxnReferenceOperation.Kind.from(operation);
+
+            Term key = extractKeyOrIndex(operation);
+
+            if (type.isCollection())
+            {
+                CollectionType<?> collectionType = (CollectionType<?>) type;
+
+                // The value for a map subtraction is actually a set (see Operation.Substraction)
+                if (kind == TxnReferenceOperation.Kind.SetDiscarder)
+                    if (collectionType.kind == MAP)
+                        receiver = receiver.withNewType(SetType.getInstance(((MapType<?, ?>) type).getKeysType(), true));
+
+                if (kind == TxnReferenceOperation.Kind.SetterByIndex || kind == TxnReferenceOperation.Kind.SetterByKey)
+                    receiver = receiver.withNewType(collectionType.valueComparator());
+            }
+
+            FieldIdentifier field = extractField(operation);
+
+            if (type.isUDT())
+            {
+                if (kind == TxnReferenceOperation.Kind.SetterByField)
+                {
+                    @SuppressWarnings("ConstantConditions") UserType userType = (UserType) type;
+                    CellPath fieldPath = userType.cellPathForField(field);
+                    int i = ByteBufferUtil.getUnsignedShort(fieldPath.get(0), 0);
+                    receiver = receiver.withNewType(userType.fieldType(i));
+                }
+            }
+
+            return new ReferenceOperation(receiver, kind, key, field, value.prepare(receiver, bindVariables));
+        }
+    }

Review Comment:
   this is similar but different from `create` and wondering why... we prepare to get the Operation, then extract what we need from the Operation... why are these 2 methods so different?



##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   since this doesn't include the real QueryOptions doesn't this mean `LIMIT ?` will fail?



##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   `org.apache.cassandra.cql3.statements.TransactionStatement#execute` and `org.apache.cassandra.cql3.statements.TransactionStatement#executeLocally` both have access; we just can't check this in `prepare` =(



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042510530


##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            int n = readCollectionSize(collection, ByteBufferAccessor.instance, ProtocolVersion.V3);

Review Comment:
   I just went w/ precedent here and copied the other `getSerializedValue()` implementations. Also went with the suggestion of `CollectionSerializer#deserialize()`, although why that's "the right thing to do" isn't 100% clear to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041439096


##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   let me add this test to my feedback branch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040182098


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)

Review Comment:
   Check out `AccordCQLTest#testListSetByIndexMultiRow`.
   
   The index concept here allows us to distinguish between prefetched rows in the same partition. (see usages of `TxnDataName#atIndex()`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1043719432


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));
+
+            return new TransactionStatement(preparedAssignments, returningSelect, returningReferences, preparedUpdates, preparedConditions, bindVariables);
+        }
+
+        private void checkAtMostOneRowSpecified(SelectStatement prepared, String failureMessage, Object messageArg)
+        {
+            int limit = prepared.getLimit(QueryOptions.DEFAULT);
+
+            if (limit == DataLimits.NO_LIMIT)
+                checkTrue(prepared.getRestrictions().hasAllPKColumnsRestrictedByEqualities(), failureMessage, messageArg);
+            else
+                checkTrue(limit == 1, failureMessage, messageArg);

Review Comment:
   going to resolve this; fixed in feedback branch and there is another comment that this is an issue (so don't need 2 comments)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040125450


##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -60,6 +60,23 @@
     private static final Constraint DNS_DOMAIN_PART_CONSTRAINT = Constraint.between(0, DNS_DOMAIN_PART_DOMAIN.length - 1).withNoShrinkPoint();
 
     public static final Gen<String> IDENTIFIER_GEN = Generators.regexWord(SourceDSL.integers().between(1, 50));
+    public static final Gen<String> SYMBOL_GEN = filter(symbolGen(SourceDSL.integers().between(1, 48)), Generators::thisBugIsBroughtToYouByTheLetterP);
+    private static boolean thisBugIsBroughtToYouByTheLetterP(String value)
+    {
+        // In Lexer.g DURATION is before IDENT and Duration allows the following to parsse: P, and PT
+        // This causes an issue for cases that use IDENT as P and PT will not match as they matched DURATION already
+        // to avoid these cases, this function will be used to filter them out so only "valid" symbols are returned
+        // see CASSANDRA-17919
+        return !("P".equals(value) || "PT".equals(value));
+    }
+    private static final char CHAR_UNDERSCORE = 95;
+    public static Gen<String> symbolGen(Gen<Integer> size)
+    {
+        char[] domain = new char[LETTER_OR_DIGIT_DOMAIN.length + 1];
+        System.arraycopy(LETTER_OR_DIGIT_DOMAIN, 0, domain, 0, LETTER_OR_DIGIT_DOMAIN.length);
+        domain[domain.length - 1] = CHAR_UNDERSCORE;

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041555157


##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);

Review Comment:
   Looks like we're just going to nix `TxnBuilder`. It's outlived its usefulness.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#issuecomment-1347340207

   Committed in https://github.com/apache/cassandra/commit/33f670bab67643b4ac0220e4be99c23b3104080e


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041271049


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }

Review Comment:
   The serializer would be a bit more complex for sure, but not terribly so. I'll take a swing at it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039938474


##########
src/antlr/Cql.g:
##########
@@ -43,6 +43,7 @@ import Parser,Lexer;
     import org.apache.cassandra.cql3.selection.*;
     import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.cql3.statements.schema.*;
+    import org.apache.cassandra.cql3.transactions.*;

Review Comment:
   `Parser.g` doesn't have its own imports. They're defined here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038630486


##########
src/java/org/apache/cassandra/cql3/Operations.java:
##########
@@ -47,11 +49,34 @@ public final class Operations implements Iterable<Operation>
      */
     private final List<Operation> staticOperations = new ArrayList<>();
 
+    private final List<ReferenceOperation> regularSubstitutions = new ArrayList<>();
+    private final List<ReferenceOperation> staticSubstitutions = new ArrayList<>();

Review Comment:
   We need to know the ref columns in order to get the required columns read
   
   ~We should document why these are not included in `requiresRead`, the reason is `org.apache.cassandra.cql3.statements.ModificationStatement#readRequiredLists` would then perform a read which we need to avoid.~



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042670999


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()

Review Comment:
   when doing the refactor it was clear that the `isJson` stuff is a bit more annoying as where we use the `noop` wouldn't support json, so trying to pull that in didn't make much sense



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042829836


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));
+
+            return new TransactionStatement(preparedAssignments, returningSelect, returningReferences, preparedUpdates, preparedConditions, bindVariables);
+        }
+
+        private void checkAtMostOneRowSpecified(SelectStatement prepared, String failureMessage, Object messageArg)
+        {
+            int limit = prepared.getLimit(QueryOptions.DEFAULT);
+
+            if (limit == DataLimits.NO_LIMIT)
+                checkTrue(prepared.getRestrictions().hasAllPKColumnsRestrictedByEqualities(), failureMessage, messageArg);
+            else
+                checkTrue(limit == 1, failureMessage, messageArg);

Review Comment:
   pushed a fix for this into the feedback branch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042828133


##########
src/java/org/apache/cassandra/cql3/Operations.java:
##########
@@ -47,11 +49,34 @@ public final class Operations implements Iterable<Operation>
      */
     private final List<Operation> staticOperations = new ArrayList<>();
 
+    private final List<ReferenceOperation> regularSubstitutions = new ArrayList<>();
+    private final List<ReferenceOperation> staticSubstitutions = new ArrayList<>();

Review Comment:
   Yep we talked, we likely need this to know what columns to include in the read so that when we run we have the value rather than `null`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042523838


##########
src/java/org/apache/cassandra/service/accord/AccordSerializers.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.LIST;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.MAP;
+import static org.apache.cassandra.db.marshal.CollectionType.Kind.SET;
+
+public class AccordSerializers
+{
+    public static <T> ByteBuffer serialize(T item, IVersionedSerializer<T> serializer)
+    {
+        int version = MessagingService.current_version;
+        long size = serializer.serializedSize(item, version) + sizeofUnsignedVInt(version);
+        try (DataOutputBuffer out = new DataOutputBuffer((int) size))
+        {
+            out.writeUnsignedVInt(version);
+            serializer.serialize(item, out, version);
+            return out.buffer(false);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T deserialize(ByteBuffer bytes, IVersionedSerializer<T> serializer)
+    {
+        try (DataInputBuffer in = new DataInputBuffer(bytes, true))
+        {
+            int version = checkedCast(in.readUnsignedVInt());
+            return serializer.deserialize(in, version);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Term.Terminal deserializeCqlCollectionAsTerm(ByteBuffer buffer, AbstractType<?> type)
+    {
+        CollectionType<?> collectionType = (CollectionType<?>) type;
+
+        if (collectionType.kind == SET)
+            return Sets.Value.fromSerialized(buffer, (SetType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == LIST)
+            return Lists.Value.fromSerialized(buffer, (ListType<?>) type, ProtocolVersion.CURRENT);
+        else if (collectionType.kind == MAP)
+            return Maps.Value.fromSerialized(buffer, (MapType<?, ?>) type, ProtocolVersion.CURRENT);
+

Review Comment:
   Passing the protocol version through `TxnReferenceOperation#apply()` isn't too hard. A little more work for `TxnCondition#applies()`...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040194422


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;

Review Comment:
   Done (just measured `RETURNING`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040116591


##########
src/java/org/apache/cassandra/utils/CollectionSerializers.java:
##########
@@ -19,41 +19,43 @@
 package org.apache.cassandra.utils;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.RandomAccess;
 import java.util.Set;
 import java.util.function.IntFunction;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public class CollectionSerializer
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class CollectionSerializers
 {
 
-    public static <V> void serializeCollection(IVersionedSerializer<V> valueSerializer, Collection<V> values, DataOutputPlus out, int version) throws IOException
+    public static <V> void serializeCollection(Collection<V> values, DataOutputPlus out, int version, IVersionedSerializer<V> valueSerializer) throws IOException
     {
         out.writeUnsignedVInt(values.size());
         for (V value : values)
             valueSerializer.serialize(value, out, version);
     }
 
-    public static <V, L extends List<V> & RandomAccess> void serializeList(IVersionedSerializer<V> valueSerializer, L values, DataOutputPlus out, int version) throws IOException

Review Comment:
   I’d be fine removing RandomAccess and assume we’re passing in a safe list for random access. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040191091


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);

Review Comment:
   The only place we call `getDecoratedKey()` is in `AccordUpdateParameters#prefetchRow()`, but we call `isFor()` there already.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042671439


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()
+        {
+            List<ByteBuffer> current;
+            @Override public ColumnFilter getColumnFilter() { return ColumnFilter.NONE; }
+            @Override public boolean hasProcessing() { return false; }
+            @Override public boolean isAggregate() { return false; }
+            @Override public int numberOfFetchedColumns() { return 0; }

Review Comment:
   turns out this is dead code, so removed in feedback branch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042670016


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()

Review Comment:
   Figured it would be simplest to do in feedback branch; was asking to avoid copy/paste by refactoring, did that in these 2 case; see aace70018a08bb9a8a3724804279665efbf473da



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042806289


##########
src/java/org/apache/cassandra/service/accord/txn/TxnAppliedQuery.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+import accord.api.Data;
+import accord.api.Query;
+import accord.api.Read;
+import accord.api.Result;
+import accord.api.Update;
+import accord.primitives.TxnId;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static org.apache.cassandra.service.accord.AccordSerializers.deserialize;
+import static org.apache.cassandra.service.accord.AccordSerializers.serialize;
+
+// TODO: This is currently unused, but we might want to use it to support returning the condition result.
+public class TxnAppliedQuery implements Query

Review Comment:
   He does. We can always resurrect it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041794658


##########
src/java/org/apache/cassandra/cql3/Operations.java:
##########
@@ -47,11 +49,34 @@ public final class Operations implements Iterable<Operation>
      */
     private final List<Operation> staticOperations = new ArrayList<>();
 
+    private final List<ReferenceOperation> regularSubstitutions = new ArrayList<>();
+    private final List<ReferenceOperation> staticSubstitutions = new ArrayList<>();

Review Comment:
   `ModificationStatement#requiresRead()` calls `operations.requiresRead()` now, so actually I think it would be okay to add them...but I think we had a quick chat about this offline around one of the failing tests in the feedback branch, `testRefAutoRead()`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038624365


##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +517,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()
+        {
+            return !(column.type instanceof CounterColumnType);
+        }
+
         public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
-            ByteBuffer bytes = t.bindAndGet(params.options);
-            if (bytes == null)
-                throw new InvalidRequestException("Invalid null value for counter increment");
-            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                return;
+            if (column.type instanceof CounterColumnType)
+            {
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for counter increment");
+                if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    return;
 
-            long increment = ByteBufferUtil.toLong(bytes);
-            if (increment == Long.MIN_VALUE)
-                throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
+                long increment = ByteBufferUtil.toLong(bytes);
+                if (increment == Long.MIN_VALUE)
+                    throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
 
-            params.addCounter(column, -increment);
+                params.addCounter(column, -increment);
+            }
+            else if (column.type instanceof NumberType)
+            {
+                NumberType<?> type = (NumberType<?>) column.type;
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for number increment");

Review Comment:
   interesting that Subtracter does this but not Adder...



##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -365,17 +389,23 @@ public Substraction(Term.Raw value)
         }
 
         public Operation prepare(TableMetadata metadata, ColumnMetadata receiver, boolean canReadExistingState) throws InvalidRequestException
-        {
+        {   
             if (!(receiver.type instanceof CollectionType))
             {
-                if (!(receiver.type instanceof CounterColumnType))
+                if (canReadExistingState)

Review Comment:
   NOTE: `ParsedInsertJson` is missing checking for txn... need to make sure its part of the TODO



##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -386,7 +416,7 @@ else if (!(receiver.type.isMultiCell()))
                     ColumnSpecification vr = new ColumnSpecification(receiver.ksName,
                                                                      receiver.cfName,
                                                                      receiver.name,
-                                                                     SetType.getInstance(((MapType)receiver.type).getKeysType(), false));
+                                                                     SetType.getInstance(((MapType<?, ?>) receiver.type).getKeysType(), true));

Review Comment:
   why change from `false` to `true`?  it doesn't look to matter in this case?



##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()

Review Comment:
   im wondering if we can merge this with `org.apache.cassandra.cql3.selection.Selection.SimpleSelection#newSelectors` as its mostly overlaps... this would also make sure `getOutputRow` handles JSON where as this doesn't



##########
src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java:
##########
@@ -1044,4 +1121,23 @@ public List<Pair<ColumnIdentifier, ColumnCondition.Raw>> getConditions()
             return conditions;
         }
     }
+
+    private static final Constants.Value ONE = new Constants.Value(ByteBufferUtil.bytes(1));
+
+    public SelectStatement createSelectForTxn()
+    {
+        // TODO: get working with static-only updates that don't specify any/all primary key columns
+        Preconditions.checkState(getRestrictions().hasAllPKColumnsRestrictedByEqualities());
+        Selection selection = Selection.forColumns(metadata, Lists.newArrayList(requiresRead), false);

Review Comment:
   `requiresRead` is computed in the constructor, which is before migration... but if a Substitutions already exists, then won't that column be missing? `org.apache.cassandra.cql3.statements.UpdateStatement.ParsedInsert#prepareInternal` moves them there first. 



##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -365,17 +389,23 @@ public Substraction(Term.Raw value)
         }
 
         public Operation prepare(TableMetadata metadata, ColumnMetadata receiver, boolean canReadExistingState) throws InvalidRequestException
-        {
+        {   
             if (!(receiver.type instanceof CollectionType))
             {
-                if (!(receiver.type instanceof CounterColumnType))
+                if (canReadExistingState)
+                {
+                    if (!(receiver.type instanceof NumberType))

Review Comment:
   should merge the 2 if statements for now, can split if we ever add a new type



##########
src/java/org/apache/cassandra/cql3/Operations.java:
##########
@@ -47,11 +49,34 @@ public final class Operations implements Iterable<Operation>
      */
     private final List<Operation> staticOperations = new ArrayList<>();
 
+    private final List<ReferenceOperation> regularSubstitutions = new ArrayList<>();
+    private final List<ReferenceOperation> staticSubstitutions = new ArrayList<>();

Review Comment:
   We should document why these are not included in `requiresRead`, the reason is `org.apache.cassandra.cql3.statements.ModificationStatement#readRequiredLists` would then perform a read which we need to avoid.



##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -126,6 +126,23 @@ public ResultSet.ResultMetadata getResultMetadata()
         return resultMetadata;
     }
 
+    public static Selection.Selectors noopSelector()
+    {
+        return new Selectors()
+        {
+            List<ByteBuffer> current;
+            @Override public ColumnFilter getColumnFilter() { return ColumnFilter.NONE; }
+            @Override public boolean hasProcessing() { return false; }
+            @Override public boolean isAggregate() { return false; }
+            @Override public int numberOfFetchedColumns() { return 0; }

Review Comment:
   TODO confirm `0`0 is correct...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1044921374


##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +519,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()
+        {
+            return !(column.type instanceof CounterColumnType);
+        }
+
         public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
-            ByteBuffer bytes = t.bindAndGet(params.options);
-            if (bytes == null)
-                throw new InvalidRequestException("Invalid null value for counter increment");
-            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                return;
+            if (column.type instanceof CounterColumnType)
+            {
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for counter increment");

Review Comment:
   Yeah. The value is literally positive, but the errors should still indicate "decrement"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] aweisberg commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
aweisberg commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1017071865


##########
src/java/org/apache/cassandra/service/accord/txn/TxnUpdate.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import accord.api.Data;
+import accord.api.Update;
+import accord.api.Write;
+import accord.primitives.Keys;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.service.accord.SerializationUtils;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static org.apache.cassandra.service.accord.SerializationUtils.deserialize;
+import static org.apache.cassandra.service.accord.SerializationUtils.serialize;
+import static org.apache.cassandra.service.accord.SerializationUtils.serializeArray;
+import static org.apache.cassandra.utils.ByteBufferUtil.readWithVIntLength;
+import static org.apache.cassandra.utils.ByteBufferUtil.serializedSizeWithVIntLength;
+import static org.apache.cassandra.utils.ByteBufferUtil.writeWithVIntLength;
+
+public class TxnUpdate implements Update
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new TxnUpdate(new ByteBuffer[0], null));
+
+    private final ByteBuffer[] serializedUpdates;
+    private final ByteBuffer serializedCondition;
+
+    public TxnUpdate(List<TxnWrite.Fragment> updates, TxnCondition condition)
+    {
+        this.serializedUpdates = serialize(updates, TxnWrite.Fragment.serializer);
+        this.serializedCondition = serialize(condition, TxnCondition.serializer);
+    }
+
+    public TxnUpdate(ByteBuffer[] serializedUpdates, ByteBuffer serializedCondition)
+    {
+        this.serializedUpdates = serializedUpdates;
+        this.serializedCondition = serializedCondition;
+    }
+
+    public ByteBuffer serializedCondition()
+    {
+        return serializedCondition.duplicate();
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = EMPTY_SIZE + ByteBufferUtil.estimatedSizeOnHeap(serializedCondition);
+        for (ByteBuffer update : serializedUpdates)
+            size += ByteBufferUtil.estimatedSizeOnHeap(update);
+        return size;
+    }
+
+    @Override
+    public String toString()
+    {
+        return "TxnUpdate{" +
+               "updates=" + deserialize(serializedUpdates, TxnWrite.Fragment.serializer) +
+               ", condition=" + deserialize(serializedCondition, TxnCondition.serializer) +
+               '}';
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnUpdate txnUpdate = (TxnUpdate) o;
+        return Arrays.equals(serializedUpdates, txnUpdate.serializedUpdates) && Objects.equals(serializedCondition, txnUpdate.serializedCondition);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = Objects.hash(serializedCondition);
+        result = 31 * result + Arrays.hashCode(serializedUpdates);
+        return result;
+    }
+
+    @Override
+    public Keys keys()
+    {
+        // TODO: Does this actually work?

Review Comment:
   It doesn't I think. I would expect no writes to be applied. This eventually filters to [here](https://github.com/apache/cassandra-accord/blob/197e4c740e4e755fe477ecc0b6f2284df253d37a/accord-core/src/main/java/accord/primitives/Writes.java#L79) and left folds over no keys not applying the writes. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
maedhroz commented on PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#issuecomment-1334529329

   [CircleCI ru](https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17719-v2)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040118569


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;

Review Comment:
   That would only be if `Cluster` was `final`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040114254


##########
src/java/org/apache/cassandra/utils/CollectionSerializers.java:
##########
@@ -19,41 +19,43 @@
 package org.apache.cassandra.utils;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.RandomAccess;
 import java.util.Set;
 import java.util.function.IntFunction;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public class CollectionSerializer
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class CollectionSerializers
 {
 
-    public static <V> void serializeCollection(IVersionedSerializer<V> valueSerializer, Collection<V> values, DataOutputPlus out, int version) throws IOException
+    public static <V> void serializeCollection(Collection<V> values, DataOutputPlus out, int version, IVersionedSerializer<V> valueSerializer) throws IOException
     {
         out.writeUnsignedVInt(values.size());
         for (V value : values)
             valueSerializer.serialize(value, out, version);
     }
 
-    public static <V, L extends List<V> & RandomAccess> void serializeList(IVersionedSerializer<V> valueSerializer, L values, DataOutputPlus out, int version) throws IOException

Review Comment:
   In practice, it looks like this is always going to be an `ArrayList` anyway, but @belliottsmith was being cautious. I still don't think I'd delegate to `serializeCollection()`, given the clear intent to avoid creating iterators?
   
   @aweisberg Any opinion on this? I think this was part of your collection serializers patch I pulled in to make rebasing easier for you...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040154310


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -533,6 +551,35 @@ else if (obj instanceof byte[])
             return ByteBuffer.wrap((byte[]) obj);
         else if (obj instanceof ByteBuffer)
             return (ByteBuffer) obj;
+        else if (obj instanceof List)
+        {
+            List<?> list = (List<?>) obj;
+            // convert subtypes to BB
+            List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList());
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return ListType.getInstance(BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Map)
+        {
+            Map<?, ?> map = (Map<?, ?>) obj;
+            // convert subtypes to BB
+            Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
+            for (Map.Entry<?, ?> e : map.entrySet())
+                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));

Review Comment:
   The idea being that, while the existing map is obviously valid, two of its keys could map to the same BB?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040196719


##########
src/java/org/apache/cassandra/service/accord/AccordService.java:
##########
@@ -99,6 +110,39 @@ public static long nowInMicros()
         return TimeUnit.MILLISECONDS.toMicros(Clock.Global.currentTimeMillis());
     }
 
+    public TxnData coordinate(Txn txn)
+    {
+        try
+        {
+            Future<Result> future = node.coordinate(txn);
+            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.SECONDS), TimeUnit.SECONDS);

Review Comment:
   `ms` is the minimum unit, so I think `MILLISECONDS` should work



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040205625


##########
src/java/org/apache/cassandra/cql3/Operation.java:
##########
@@ -365,17 +389,23 @@ public Substraction(Term.Raw value)
         }
 
         public Operation prepare(TableMetadata metadata, ColumnMetadata receiver, boolean canReadExistingState) throws InvalidRequestException
-        {
+        {   
             if (!(receiver.type instanceof CollectionType))
             {
-                if (!(receiver.type instanceof CounterColumnType))
+                if (canReadExistingState)

Review Comment:
   Yup. It is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042664130


##########
src/java/org/apache/cassandra/service/accord/txn/AccordUpdateParameters.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Collections;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+
+public class AccordUpdateParameters
+{
+    private final TxnData data;
+    private final QueryOptions options;
+
+    public AccordUpdateParameters(TxnData data, QueryOptions options)
+    {
+        this.data = data;
+        this.options = options;
+    }
+
+    public TxnData getData()
+    {
+        return data;
+    }
+
+    public UpdateParameters updateParameters(TableMetadata metadata, int rowIndex)
+    {
+        // This is currently only used by Guardrails, but this logically have issues with Accord as drifts in config
+        // values could cause unexpected issues in Accord. (ex. some nodes reject writes while others accept)
+        // For the time being, guardrails are disabled for Accord queries.
+        ClientState disabledGuardrails = null;
+
+        // What we use here doesn't matter as they get replaced before actually performing the write.
+        // see org.apache.cassandra.service.accord.txn.TxnWrite.Update.write
+        int nowInSeconds = 42;
+        long timestamp = nowInSeconds;
+
+        // TODO: How should Accord work with TTL?
+        int ttl = metadata.params.defaultTimeToLive;
+        return new UpdateParameters(metadata,
+                                    disabledGuardrails,
+                                    options,
+                                    timestamp,
+                                    nowInSeconds,
+                                    ttl,
+                                    prefetchRow(metadata, rowIndex));
+    }
+
+    private Map<DecoratedKey, Partition> prefetchRow(TableMetadata metadata, int index)

Review Comment:
   I think this is covered in the feedback branch, but will come back to check on this at the end.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038642518


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1473,18 +1484,21 @@ public String toString()
         public final boolean isDistinct;
         public final boolean allowFiltering;
         public final boolean isJson;
+        public final String refName;
 
         public Parameters(Map<ColumnIdentifier, Boolean> orderings,
                           List<Selectable.Raw> groups,
                           boolean isDistinct,
                           boolean allowFiltering,
-                          boolean isJson)
+                          boolean isJson,
+                          String refName)

Review Comment:
   this param is only used in Cql_Parser, might be good to lower diff by adding the old constructor back that just sets this to null?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));

Review Comment:
   since this is first we can get into an issue where 2 updates on the same partition read different columns, leading to nulls
   
   ```
   UPDATE table SET a += 1 WHERE pk=0;
   UPDATE table SET b += 1 WHERE pk=0; // b is null as we try to filter columns
   ```
   
   Auto Read will not include `b` right?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)

Review Comment:
   why not validate selects?
   
   `org.apache.cassandra.cql3.statements.SelectStatement#validate` now does work
   
   ```
   public void validate(ClientState state) throws InvalidRequestException
       {
           if (parameters.allowFiltering && !SchemaConstants.isSystemKeyspace(table.keyspace))
               Guardrails.allowFilteringEnabled.ensureEnabled(state);
       }
   ```



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);

Review Comment:
   I need to look into it but not sure why this is an array and not a single value...
   
   ```
   a AND b AND c
   ```
   
   is clear and should be a single condition... so how do we create multiple in the array?



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -160,18 +169,30 @@ protected ModificationStatement prepareInternal(TableMetadata metadata,
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
 
-                Term.Raw value = columnValues.get(i);

Review Comment:
   can switch back to `Term.Raw`, `ReferenceValue.Raw` extends `Term.Raw`



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));

Review Comment:
   don't we need `select.setKeyspace(state);` as well?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);

Review Comment:
   Looking at parser it looks like we create the array in `org.apache.cassandra.cql3.Cql_Parser#txnConditions`. ill need to look closer but still feels like we should be aware of AND/OR



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();

Review Comment:
   this should move up top right?



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));

Review Comment:
   nit: `Iterables.getOnlyElement` works but could have a better error for users



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));
+
+            return new TransactionStatement(preparedAssignments, returningSelect, returningReferences, preparedUpdates, preparedConditions, bindVariables);
+        }
+
+        private void checkAtMostOneRowSpecified(SelectStatement prepared, String failureMessage, Object messageArg)
+        {
+            int limit = prepared.getLimit(QueryOptions.DEFAULT);
+
+            if (limit == DataLimits.NO_LIMIT)
+                checkTrue(prepared.getRestrictions().hasAllPKColumnsRestrictedByEqualities(), failureMessage, messageArg);
+            else
+                checkTrue(limit == 1, failureMessage, messageArg);

Review Comment:
   if I do `SELECT * FROM table where pk=? LIMIT 2` will this fail?  Looks like it will



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");

Review Comment:
   name can't be null, we control that... you can check for `select.parameters.refName`



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -160,18 +169,30 @@ protected ModificationStatement prepareInternal(TableMetadata metadata,
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
 
-                Term.Raw value = columnValues.get(i);
+                Object value = columnValues.get(i);
 
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, value));
+                    checkTrue(value instanceof Term.Raw, "value references can't be used with primary key columns");

Review Comment:
   this check would allow `pk = a.b` as ref are `Term.Raw`



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -160,18 +169,30 @@ protected ModificationStatement prepareInternal(TableMetadata metadata,
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
 
-                Term.Raw value = columnValues.get(i);
+                Object value = columnValues.get(i);
 
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, value));
+                    checkTrue(value instanceof Term.Raw, "value references can't be used with primary key columns");
+                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, (Term.Raw) value));
                 }
-                else
+                else if (value instanceof ReferenceValue.Raw)
+                {
+                    ReferenceValue.Raw raw = (ReferenceValue.Raw) value;
+                    ReferenceValue referenceValue = raw.prepare(def, bindVariables);
+                    ReferenceOperation operation = new ReferenceOperation(def, TxnReferenceOperation.Kind.setterFor(def), null, null, referenceValue);
+                    operations.add(def, operation);
+                }
+                else if (value instanceof Term.Raw)
                 {
-                    Operation operation = new Operation.SetValue(value).prepare(metadata, def, !conditions.isEmpty());
+                    Operation operation = new Operation.SetValue((Term.Raw) value).prepare(metadata, def, !conditions.isEmpty());
                     operation.collectMarkerSpecification(bindVariables);
                     operations.add(operation);
                 }
+                else
+                {
+                    throw new IllegalStateException();

Review Comment:
   detailed msg plz!



##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<RowDataReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    private List<TxnNamedRead> createNamedReads(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+
+        return reads;
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                // Reads are not merged by partition here due to potentially differing columns retrieved, etc.
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey(), idx);
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, modification.createSelectForTxn()));
+            }
+
+            idx++;
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+
+        if (updates.isEmpty())
+        {
+            // TODO: Test case around this...
+            Preconditions.checkState(conditions.isEmpty(), "No condition should exist without updates present");
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            List<TxnNamedRead> reads = createNamedReads(options, keySet::add);
+            Keys txnKeys = toKeys(keySet);
+            TxnRead read = new TxnRead(reads, txnKeys);
+            return new Txn.InMemory(txnKeys, read, TxnQuery.ALL, update);
+        }
+    }
+
+    @Override
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        TxnData data = AccordService.instance().coordinate(createTxn(state.getClientState(), options));
+        
+        if (returningSelect != null)
+        {
+            FilteredPartition partition = data.get(TxnDataName.returning());
+            Selection.Selectors selectors = returningSelect.select.getSelection().newSelectors(options);
+            ResultSetBuilder result = new ResultSetBuilder(returningSelect.select.getResultMetadata(), selectors, null);
+            returningSelect.select.processPartition(partition.rowIterator(), options, result, FBUtilities.nowInSeconds());
+            return new ResultMessage.Rows(result.build());
+        }
+        
+        if (returningReferences != null)
+        {
+            List<ColumnSpecification> names = new ArrayList<>(returningReferences.size());
+            List<ColumnMetadata> columns = new ArrayList<>(returningReferences.size());
+
+            for (RowDataReference reference : returningReferences)
+            {
+                ColumnMetadata forMetadata = reference.toResultMetadata();
+                names.add(forMetadata);
+                columns.add(reference.column());
+            }
+
+            ResultSetBuilder result = new ResultSetBuilder(new ResultSet.ResultMetadata(names), Selection.noopSelector(), null);
+            result.newRow(options.getProtocolVersion(), null, null, columns);
+
+            for (int i = 0; i < returningReferences.size(); i++)
+            {
+                RowDataReference reference = returningReferences.get(i);
+                TxnReference txnReference = reference.toTxnReference(options);
+                ByteBuffer buffer = txnReference.toByteBuffer(data, names.get(i).type);
+                result.add(buffer);
+            }
+
+            return new ResultMessage.Rows(result.build());
+        }
+
+        // In the case of a write-only transaction, just return and empty result.
+        // TODO: This could be modified to return an indication of whether a condition (if present) succeeds.
+        return new ResultMessage.Void();
+    }
+
+    @Override
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, options, nanoTime());
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.TRANSACTION);
+    }
+
+    public static class Parsed extends QualifiedStatement
+    {
+        private final List<SelectStatement.RawStatement> assignments;
+        private final SelectStatement.RawStatement select;
+        private final List<RowDataReference.Raw> returning;
+        private final List<ModificationStatement.Parsed> updates;
+        private final List<ConditionStatement.Raw> conditions;
+        private final List<RowDataReference.Raw> dataReferences;
+
+        public Parsed(List<SelectStatement.RawStatement> assignments,
+                      SelectStatement.RawStatement select,
+                      List<RowDataReference.Raw> returning,
+                      List<ModificationStatement.Parsed> updates,
+                      List<ConditionStatement.Raw> conditions,
+                      List<RowDataReference.Raw> dataReferences)
+        {
+            super(null);
+            this.assignments = assignments;
+            this.select = select;
+            this.returning = returning;
+            this.updates = updates;
+            this.conditions = conditions != null ? conditions : Collections.emptyList();
+            this.dataReferences = dataReferences;
+        }
+
+        @Override
+        public void setKeyspace(ClientState state)
+        {
+            assignments.forEach(select -> select.setKeyspace(state));
+            updates.forEach(update -> update.setKeyspace(state));
+        }
+
+        @Override
+        public CQLStatement prepare(ClientState state)
+        {
+            checkFalse(updates.isEmpty() && returning == null && select == null, EMPTY_TRANSACTION_MESSAGE);
+
+            if (select != null || returning != null)
+                checkTrue(select != null ^ returning != null, "Cannot specify both a full SELECT and a SELECT w/ LET references.");
+
+            List<NamedSelect> preparedAssignments = new ArrayList<>(assignments.size());
+            Map<TxnDataName, RowDataReference.ReferenceSource> refSources = new HashMap<>();
+            Set<TxnDataName> selectNames = new HashSet<>();
+
+            for (SelectStatement.RawStatement select : assignments)
+            {
+                TxnDataName name = TxnDataName.user(select.parameters.refName);
+                checkNotNull(name, "Assignments must be named");
+                checkTrue(selectNames.add(name), DUPLICATE_TUPLE_NAME_MESSAGE, name.name());
+
+                SelectStatement prepared = select.prepare(bindVariables);
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_LET_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+
+                NamedSelect namedSelect = new NamedSelect(name, prepared);
+                preparedAssignments.add(namedSelect);
+                refSources.put(name, new SelectReferenceSource(prepared));
+            }
+
+            if (dataReferences != null)
+                for (RowDataReference.Raw reference : dataReferences)
+                    reference.resolveReference(refSources);
+
+            NamedSelect returningSelect = null;
+            if (select != null)
+            {
+                SelectStatement prepared = select.prepare(bindVariables);
+                // TODO: Accord saves the result of this read, so limit to a single row until that is no longer true.
+                checkAtMostOneRowSpecified(prepared, INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE, LazyToString.lazy(() -> prepared.asCQL(QueryOptions.DEFAULT, state)));
+                returningSelect = new NamedSelect(TxnDataName.returning(), prepared);
+            }
+
+            List<RowDataReference> returningReferences = null;
+
+            if (returning != null)
+            {
+                // TODO: Eliminate/modify this check if we allow full tuple selections.
+                returningReferences = returning.stream().peek(raw -> checkTrue(raw.column() != null, SELECT_REFS_NEED_COLUMN_MESSAGE))
+                                                        .map(RowDataReference.Raw::prepareAsReceiver)
+                                                        .collect(Collectors.toList());
+            }
+
+            List<ModificationStatement> preparedUpdates = new ArrayList<>(updates.size());
+            
+            // check for any read-before-write updates
+            for (int i = 0; i < updates.size(); i++)
+            {
+                ModificationStatement.Parsed parsed = updates.get(i);
+
+                ModificationStatement prepared = parsed.prepare(bindVariables);
+                checkFalse(prepared.hasConditions(), NO_CONDITIONS_IN_UPDATES_MESSAGE);
+                checkFalse(prepared.isTimestampSet(), NO_TIMESTAMPS_IN_UPDATES_MESSAGE);
+
+                preparedUpdates.add(prepared);
+            }
+
+            List<ConditionStatement> preparedConditions = new ArrayList<>(conditions.size());
+            for (ConditionStatement.Raw condition : conditions)
+                // TODO: Is this synthetic ks name dangerous?
+                preparedConditions.add(condition.prepare("[txn]", bindVariables));

Review Comment:
   you need to use the keyspace for the data...  lets say you do the following
   
   ```
   LET a = (SELECT * FROM ks.table LIMIT 1);
   IF ks.fancy(a) THEN
     ...
   END
   ```
   
   since `fancy` belongs to `ks` you need that.  We fixed this bug in 1 other place as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040127624


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+                return executeWithRetry(cluster, check, boundValues);
+
+            throw t;
+        }
+    }
+
+    public static class ByteBuddyHelper

Review Comment:
   Yeah, that makes senes in the model where we install one rule per BB helper class. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040194173


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;
+        for (String part : parts)
+            size += part.length();

Review Comment:
   Probably, although let's wait until we have a resolution for https://github.com/apache/cassandra/pull/1962#discussion_r1038512017 to finalize.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040156650


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -533,6 +551,35 @@ else if (obj instanceof byte[])
             return ByteBuffer.wrap((byte[]) obj);
         else if (obj instanceof ByteBuffer)
             return (ByteBuffer) obj;
+        else if (obj instanceof List)
+        {
+            List<?> list = (List<?>) obj;
+            // convert subtypes to BB
+            List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList());
+            // decompose/serializer doesn't use the isMultiCell, so safe to do this
+            return ListType.getInstance(BytesType.instance, false).decompose(bbs);
+        }
+        else if (obj instanceof Map)
+        {
+            Map<?, ?> map = (Map<?, ?>) obj;
+            // convert subtypes to BB
+            Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
+            for (Map.Entry<?, ?> e : map.entrySet())
+                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));

Review Comment:
   In any case, something like this is what you have in mind?
   
   ```
   for (Map.Entry<?, ?> e : map.entrySet())
   {
       Object key = e.getKey();
       ByteBuffer previousValue = bbs.put(objectToBytes(key), objectToBytes(e.getValue()));
       if (previousValue != null)
           throw new IllegalStateException("Key " + key + " already maps to value " + previousValue);
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038557424


##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)

Review Comment:
   would be good to make `private`; can increase the visibility later, but nice to hide when we don't use externally



##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);
+        // the parser will only let us define a ref name if we're parsing a transaction, which we're not
+        // so we need to manually add it in the call, and confirm nothing got parsed
+        Preconditions.checkState(parsed.parameters.refName == null);
+
+        SelectStatement statement = parsed.prepare(bindVariables);
+        QueryOptions queryOptions = QueryProcessor.makeInternalOptions(statement, values);
+        ReadQuery readQuery = statement.getQuery(queryOptions, 0);
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+        reads.add(new TxnNamedRead(name, Iterables.getOnlyElement(selectQuery.queries)));
+        return this;
+    }
+
+    public TxnBuilder withWrite(PartitionUpdate update, TxnReferenceOperations referenceOps)
+    {
+        int index = writes.size();
+        writes.add(new TxnWrite.Fragment(AccordKey.of(update), index, update, referenceOps));
+        return this;
+    }
+
+    public TxnBuilder withWrite(String query, TxnReferenceOperations referenceOps, VariableSpecifications variables, Object... values)
+    {
+        ModificationStatement.Parsed parsed = (ModificationStatement.Parsed) QueryProcessor.parseStatement(query);
+        ModificationStatement prepared = parsed.prepare(variables);
+        QueryOptions options = QueryProcessor.makeInternalOptions(prepared, values);
+        return withWrite(prepared.getTxnUpdate(ClientState.forInternalCalls(), options), referenceOps);
+    }
+
+    public TxnBuilder withWrite(String query)
+    {
+        return withWrite(query, TxnReferenceOperations.empty(), VariableSpecifications.empty());
+    }
+
+    static TxnReference reference(TxnDataName name, String column)
+    {
+        ColumnMetadata metadata = null;
+        if (column != null)
+        {
+            String[] parts = column.split("\\.");
+            Preconditions.checkArgument(parts.length == 3);

Review Comment:
   so `keyspace.table.name`?  what happens if you do `a.b`?  that would be size 4.
   
   Also, can we get a useful error msg?



##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);

Review Comment:
   I do wonder if its best to change this class though to create a CQL and parse that... we have logic in `TransactionStatement` to better understand the semantics, but this doesn't do that and is very raw... 



##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);

Review Comment:
   Just tested this... you can tell parser we are doing a txn!
   
   ```
   public static CQLStatement.Raw parseStatement(String queryStr) throws SyntaxException
       {
           try
           {
               return CQLFragmentParser.parseAnyUnhandled(parser -> {
                   parser.gParser.isParsingTxn = true;
                   try
                   {
                       return parser.query();
                   }
                   finally
                   {
                       parser.gParser.isParsingTxn = false;
                   }
               }, queryStr);
           }
   ```



##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);
+        // the parser will only let us define a ref name if we're parsing a transaction, which we're not
+        // so we need to manually add it in the call, and confirm nothing got parsed
+        Preconditions.checkState(parsed.parameters.refName == null);
+
+        SelectStatement statement = parsed.prepare(bindVariables);
+        QueryOptions queryOptions = QueryProcessor.makeInternalOptions(statement, values);
+        ReadQuery readQuery = statement.getQuery(queryOptions, 0);
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+        reads.add(new TxnNamedRead(name, Iterables.getOnlyElement(selectQuery.queries)));
+        return this;
+    }
+
+    public TxnBuilder withWrite(PartitionUpdate update, TxnReferenceOperations referenceOps)
+    {
+        int index = writes.size();
+        writes.add(new TxnWrite.Fragment(AccordKey.of(update), index, update, referenceOps));
+        return this;
+    }
+
+    public TxnBuilder withWrite(String query, TxnReferenceOperations referenceOps, VariableSpecifications variables, Object... values)
+    {
+        ModificationStatement.Parsed parsed = (ModificationStatement.Parsed) QueryProcessor.parseStatement(query);
+        ModificationStatement prepared = parsed.prepare(variables);
+        QueryOptions options = QueryProcessor.makeInternalOptions(prepared, values);
+        return withWrite(prepared.getTxnUpdate(ClientState.forInternalCalls(), options), referenceOps);
+    }
+
+    public TxnBuilder withWrite(String query)
+    {
+        return withWrite(query, TxnReferenceOperations.empty(), VariableSpecifications.empty());
+    }
+
+    static TxnReference reference(TxnDataName name, String column)

Review Comment:
   this function feels incorrect... `name` *has to be a USER name*, and rather than looking up what table that references, we rely on the `column` param to also repeat this information...  so if I have the following
   
   ```
   LET a = (SELECT * FROM ks.tbl1 LIMIT 1);
   LET b = (SELECT * FROM ks.tbl2 LIMIT 1);
   ```
   
   and
   
   ```
   builder.withExistenceCondition(TxnDataName.user("b"), "ks.tbl1.testing", IS_NOT_NULL);
   ```
   
   working on a patch to change this...



##########
test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import accord.primitives.Keys;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Txn;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.accord.api.AccordKey;
+
+public class TxnBuilder
+{
+    private final List<TxnNamedRead> reads = new ArrayList<>();
+    private final List<TxnWrite.Fragment> writes = new ArrayList<>();
+    private final List<TxnCondition> conditions = new ArrayList<>();
+
+    public static TxnBuilder builder()
+    {
+        return new TxnBuilder();
+    }
+
+    public TxnBuilder withRead(String name, String query)
+    {
+        return withRead(TxnDataName.user(name), query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query)
+    {
+        return withRead(name, query, VariableSpecifications.empty());
+    }
+
+    public TxnBuilder withRead(TxnDataName name, String query, VariableSpecifications bindVariables, Object... values)
+    {
+        SelectStatement.RawStatement parsed = (SelectStatement.RawStatement) QueryProcessor.parseStatement(query);
+        // the parser will only let us define a ref name if we're parsing a transaction, which we're not
+        // so we need to manually add it in the call, and confirm nothing got parsed
+        Preconditions.checkState(parsed.parameters.refName == null);
+
+        SelectStatement statement = parsed.prepare(bindVariables);
+        QueryOptions queryOptions = QueryProcessor.makeInternalOptions(statement, values);
+        ReadQuery readQuery = statement.getQuery(queryOptions, 0);
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+        reads.add(new TxnNamedRead(name, Iterables.getOnlyElement(selectQuery.queries)));
+        return this;
+    }
+
+    public TxnBuilder withWrite(PartitionUpdate update, TxnReferenceOperations referenceOps)
+    {
+        int index = writes.size();
+        writes.add(new TxnWrite.Fragment(AccordKey.of(update), index, update, referenceOps));
+        return this;
+    }
+
+    public TxnBuilder withWrite(String query, TxnReferenceOperations referenceOps, VariableSpecifications variables, Object... values)
+    {
+        ModificationStatement.Parsed parsed = (ModificationStatement.Parsed) QueryProcessor.parseStatement(query);
+        ModificationStatement prepared = parsed.prepare(variables);
+        QueryOptions options = QueryProcessor.makeInternalOptions(prepared, values);
+        return withWrite(prepared.getTxnUpdate(ClientState.forInternalCalls(), options), referenceOps);
+    }
+
+    public TxnBuilder withWrite(String query)
+    {
+        return withWrite(query, TxnReferenceOperations.empty(), VariableSpecifications.empty());
+    }
+
+    static TxnReference reference(TxnDataName name, String column)

Review Comment:
   ```
   diff --git test/unit/org/apache/cassandra/cql3/statements/TransactionStatementTest.java test/unit/org/apache/cassandra/cql3/statements/TransactionStatementTest.java
   index 45e11a8fcf..492e3e67a2 100644
   --- test/unit/org/apache/cassandra/cql3/statements/TransactionStatementTest.java
   +++ test/unit/org/apache/cassandra/cql3/statements/TransactionStatementTest.java
   @@ -378,8 +378,8 @@ public class TransactionStatementTest
                                     .withRead(TxnDataName.returning(), "SELECT v FROM ks.tbl1 WHERE k=2 AND c=2")
                                     .withWrite("UPDATE ks.tbl1 SET v=1 WHERE k=1 AND c=2")
                                     .withIsNotNullCondition(user("row1"), null)
   -                                 .withEqualsCondition("row1", "ks.tbl1.v", bytes(3))
   -                                 .withEqualsCondition("row2", "ks.tbl2.v", bytes(4))
   +                                 .withEqualsCondition("row1", "v", bytes(3))
   +                                 .withEqualsCondition("row2", "v", bytes(4))
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(query);
   @@ -407,8 +407,8 @@ public class TransactionStatementTest
                                     .withRead("row2", "SELECT * FROM ks.tbl2 WHERE k=2 AND c=2")
                                     .withWrite("UPDATE ks.tbl1 SET v=1 WHERE k=1 AND c=2")
                                     .withIsNotNullCondition(user("row1"), null)
   -                                 .withEqualsCondition("row1", "ks.tbl1.v", bytes(3))
   -                                 .withEqualsCondition("row2", "ks.tbl2.v", bytes(4))
   +                                 .withEqualsCondition("row1", "v", bytes(3))
   +                                 .withEqualsCondition("row2", "v", bytes(4))
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(query);
   @@ -433,7 +433,7 @@ public class TransactionStatementTest
            Txn expected = TxnBuilder.builder()
                                     .withRead("row1", "SELECT * FROM ks.tbl3 WHERE k=1")
                                     .withWrite("INSERT INTO ks.tbl3 (k, \"with spaces\") VALUES (1, 2)")
   -                                 .withIsNullCondition(user("row1"), "ks.tbl3.with spaces")
   +                                 .withIsNullCondition(user("row1"), "with spaces")
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(query);
   @@ -467,8 +467,8 @@ public class TransactionStatementTest
                                     .withRead("row2", "SELECT * FROM ks.tbl2 WHERE k=2 AND c=2")
                                     .withRead(TxnDataName.returning(), "SELECT v FROM ks.tbl1 WHERE k=1 AND c=2")
                                     .withWrite(emptyUpdate(TABLE1, 1, 2, false), referenceOps)
   -                                 .withEqualsCondition("row1", "ks.tbl1.v", bytes(3))
   -                                 .withEqualsCondition("row2", "ks.tbl2.v", bytes(4))
   +                                 .withEqualsCondition("row1", "v", bytes(3))
   +                                 .withEqualsCondition("row2", "v", bytes(4))
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(query);
   @@ -497,8 +497,8 @@ public class TransactionStatementTest
                                     .withRead("row2", "SELECT * FROM ks.tbl2 WHERE k=2 AND c=2")
                                     .withRead(TxnDataName.returning(), "SELECT v FROM ks.tbl1 WHERE k=1 AND c=2")
                                     .withWrite(emptyUpdate(TABLE1, 1, 2, true), referenceOps)
   -                                 .withEqualsCondition("row1", "ks.tbl1.v", bytes(3))
   -                                 .withEqualsCondition("row2", "ks.tbl2.v", bytes(4))
   +                                 .withEqualsCondition("row1", "v", bytes(3))
   +                                 .withEqualsCondition("row2", "v", bytes(4))
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(query);
   @@ -531,7 +531,7 @@ public class TransactionStatementTest
                                                TxnReferenceOperations.empty(),
                                                new VariableSpecifications(Collections.singletonList(null)),
                                                updatedListBytes)
   -                                 .withEqualsCondition("row1", "ks.tbl4.int_list", initialListBytes)
   +                                 .withEqualsCondition("row1", "int_list", initialListBytes)
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(update);
   @@ -569,7 +569,7 @@ public class TransactionStatementTest
            Txn expected = TxnBuilder.builder()
                                     .withRead("row1", "SELECT * FROM ks.tbl4 WHERE k = 0")
                                     .withWrite(emptyUpdate(TABLE4, 1, Clustering.EMPTY, false), referenceOps)
   -                                 .withEqualsCondition("row1", "ks.tbl4.int_list", initialListBytes)
   +                                 .withEqualsCondition("row1", "int_list", initialListBytes)
                                     .build();
    
            TransactionStatement.Parsed parsed = (TransactionStatement.Parsed) QueryProcessor.parseStatement(update);
   diff --git test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java
   index 86a42042f8..5762958eb7 100644
   --- test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java
   +++ test/unit/org/apache/cassandra/service/accord/txn/TxnBuilder.java
   @@ -21,6 +21,7 @@ package org.apache.cassandra.service.accord.txn;
    import java.nio.ByteBuffer;
    import java.util.ArrayList;
    import java.util.List;
   +import java.util.Optional;
    import java.util.SortedSet;
    import java.util.TreeSet;
    
   @@ -102,18 +103,17 @@ public class TxnBuilder
            return withWrite(query, TxnReferenceOperations.empty(), VariableSpecifications.empty());
        }
    
   -    static TxnReference reference(TxnDataName name, String column)
   +    private TxnReference reference(TxnDataName name, String column)
        {
   -        ColumnMetadata metadata = null;
   -        if (column != null)
   -        {
   -            String[] parts = column.split("\\.");
   -            Preconditions.checkArgument(parts.length == 3);
   -            TableMetadata table = Schema.instance.getTableMetadata(parts[0], parts[1]);
   -            Preconditions.checkArgument(table != null);
   -            metadata = table.getColumn(new ColumnIdentifier(parts[2], true));
   -            Preconditions.checkArgument(metadata != null);
   -        }
   +        // do any reads match the name?
   +        Optional<TxnNamedRead> match = reads.stream().filter(n -> n.name().equals(name)).findFirst();
   +        if (!match.isPresent())
   +            throw new IllegalArgumentException("Attempted to create a reference for " + name + " but no read exists with that name");
   +        TxnNamedRead read = match.get();
   +        TableMetadata table = Schema.instance.getTableMetadata(read.key().tableId());
   +        ColumnMetadata metadata = column == null ?
   +                                  null :
   +                                  table.getColumn(new ColumnIdentifier(column, true)); //TODO support a.b.c for UDT
            return new TxnReference(name, metadata);
        }
    
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] aweisberg commented on a diff in pull request #1962: CQL support for CEP-15 multi-partition transactions

Posted by GitBox <gi...@apache.org>.
aweisberg commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1017061274


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.txn.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnReference;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<ColumnReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<ColumnReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)
+            statement.validate(state);
+    }
+
+    @VisibleForTesting
+    public List<ColumnReference> getReturningReferences()
+    {
+        return returningReferences;
+    }
+
+    TxnNamedRead createNamedRead(NamedSelect namedSelect, QueryOptions options)
+    {
+        SelectStatement select = namedSelect.select;
+        ReadQuery readQuery = select.getQuery(options, 0);
+
+        // We reject reads from both LET and SELECT that do not specify a single row.
+        @SuppressWarnings("unchecked") 
+        SinglePartitionReadQuery.Group<SinglePartitionReadCommand> selectQuery = (SinglePartitionReadQuery.Group<SinglePartitionReadCommand>) readQuery;
+
+        return new TxnNamedRead(namedSelect.name, Iterables.getOnlyElement(selectQuery.queries));
+    }
+
+    TxnRead createRead(QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnNamedRead> reads = new ArrayList<>(assignments.size() + 1);
+
+        for (NamedSelect select : assignments)
+        {
+            TxnNamedRead read = createNamedRead(select, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        if (returningSelect != null)
+        {
+            TxnNamedRead read = createNamedRead(returningSelect, options);
+            keyConsumer.accept(read.key());
+            reads.add(read);
+        }
+
+        for (NamedSelect select : autoReads.values())
+            // don't need keyConsumer as the keys are known to exist due to Modification
+            reads.add(createNamedRead(select, options));
+        
+        return new TxnRead(reads);
+    }
+
+    TxnCondition createCondition(QueryOptions options)
+    {
+        if (conditions.isEmpty())
+            return TxnCondition.none();
+        if (conditions.size() == 1)
+            return conditions.get(0).createCondition(options);
+
+        List<TxnCondition> result = new ArrayList<>(conditions.size());
+        for (ConditionStatement condition : conditions)
+            result.add(condition.createCondition(options));
+
+        // TODO: OR support
+        return new TxnCondition.BooleanGroup(TxnCondition.Kind.AND, result);
+    }
+
+    private final Map<TxnDataName, NamedSelect> autoReads = new HashMap<>();
+
+    List<TxnWrite.Fragment> createWriteFragments(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        List<TxnWrite.Fragment> fragments = new ArrayList<>(updates.size());
+        int idx = 0;
+        for (ModificationStatement modification : updates)
+        {
+            TxnWrite.Fragment fragment = modification.getTxnWriteFragment(idx++, state, options);
+            keyConsumer.accept(fragment.key);
+            fragments.add(fragment);
+
+            if (modification.allReferenceOperations().stream().anyMatch(ReferenceOperation::requiresRead))
+            {
+                TxnDataName partitionName = TxnDataName.partitionRead(modification.metadata(), fragment.key.partitionKey());
+                if (!autoReads.containsKey(partitionName))
+                    autoReads.put(partitionName, new NamedSelect(partitionName, ((UpdateStatement) modification).createSelectForTxn()));
+            }
+        }
+        return fragments;
+    }
+
+    TxnUpdate createUpdate(ClientState state, QueryOptions options, Consumer<Key> keyConsumer)
+    {
+        return new TxnUpdate(createWriteFragments(state, options, keyConsumer), createCondition(options));
+    }
+
+    Keys toKeys(SortedSet<Key> keySet)
+    {
+        return new Keys(keySet);
+    }
+
+    @VisibleForTesting
+    public Txn createTxn(ClientState state, QueryOptions options)
+    {
+        SortedSet<Key> keySet = new TreeSet<>();
+        if (updates.isEmpty())
+        {
+            Preconditions.checkState(conditions.isEmpty());
+            TxnRead read = createRead(options, keySet::add);
+            return new Txn.InMemory(toKeys(keySet), read, TxnQuery.ALL);
+        }
+        else
+        {
+            TxnUpdate update = createUpdate(state, options, keySet::add);
+            TxnRead read = createRead(options, keySet::add);

Review Comment:
   Could `createRead` be pulled out of the two branches since it is done the same in both?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042725731


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)

Review Comment:
   added to my feedback branch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042826551


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;
+        for (String part : parts)
+            size += part.length();

Review Comment:
   pushed in feedback branch: used `size += ObjectSizes.sizeOf(part);`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#issuecomment-1341840513

   so close.  I am mostly +1 but there are still a few bugs remaining; once resolved I am +1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1043790169


##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            ProtocolVersion version = ProtocolVersion.V3;

Review Comment:
   `org.apache.cassandra.serializers.SetSerializer#getSerializedValue` and `org.apache.cassandra.serializers.MapSerializer#getSerializedValue` both do this so its consistent at least...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041452559


##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   pushed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040075041


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -1171,8 +1173,9 @@ public static void log(Config config)
             String value;
             try
             {
-                // Field.get() can throw NPE if the value of the field is null
-                value = field.get(config).toString();
+                // don't use exceptions for normal control flow!
+                Object obj = field.get(config);
+                value = obj != null ? obj.toString() : "null";

Review Comment:
   I also don't see any reason to retain the new comment, since it's a reaction to a previous decision that has been wiped out anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040175405


##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   There's definitely a lot of `checkedCast()` and unchecked `(int)`. Do we want to audit/change all of those in this PR, or leave it for another issue. (I'd be fine w/ something like `read4bUnsignedVInt()` if we want to add to `DataInputPlus`.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041553899


##########
src/java/org/apache/cassandra/serializers/ListSerializer.java:
##########
@@ -224,10 +226,29 @@ public Class<List<T>> getType()
     }
 
     @Override
-    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer key, AbstractType<?> comparator)
+    public ByteBuffer getSerializedValue(ByteBuffer collection, ByteBuffer index, AbstractType<?> comparator)
     {
-        // We don't allow selecting an element of a list, so we don't need this.
-        throw new UnsupportedOperationException();
+        try
+        {
+            ProtocolVersion version = ProtocolVersion.V3;

Review Comment:
   hard coding this could come to bite us later on... not sure how we would solve this without changing the method signature...



##########
src/antlr/Parser.g:
##########
@@ -651,6 +717,102 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     | d=deleteStatement  { $statement = d; }
     ;
 
+/**
+ * ex. conditional update returning pre-update values
+ *
+ * BEGIN TRANSACTION
+ *   LET row1 = (SELECT * FROM <table> WHERE k=1 AND c=2);
+ *   LET row2 = (SELECT * FROM <table> WHERE k=2 AND c=2);
+ *   SELECT row1.v, row2.v;
+ *   IF row1.v = 3 AND row2.v = 4 THEN
+ *     UPDATE <table> SET v = row1.v + 1 WHERE k = 1 AND c = 2;
+ *   END IF
+ * COMMIT TRANSACTION
+ *
+ * ex. read-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   SELECT * FROM <table> WHERE k=1 AND c=2;
+ * COMMIT TRANSACTION
+ *
+ * ex. write-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   INSERT INTO <table> (k, c, v) VALUES (0, 0, 1);
+ * COMMIT TRANSACTION
+ */
+batchTxnStatement returns [TransactionStatement.Parsed expr]
+    @init {
+        isParsingTxn = true;
+        List<SelectStatement.RawStatement> assignments = new ArrayList<>();
+        SelectStatement.RawStatement select = null;
+        List<RowDataReference.Raw> returning = null;
+        List<ModificationStatement.Parsed> updates = new ArrayList<>();
+    }
+    : K_BEGIN K_TRANSACTION
+      ( let=letStatement ';' { assignments.add(let); })*
+      ( ( (selectStatement) => s=selectStatement ';' { select = s; }) | ( K_SELECT drs=rowDataReferences ';' { returning = drs; }) )?
+      ( K_IF conditions=txnConditions K_THEN { isTxnConditional = true; } )?
+      ( upd=batchStatementObjective ';' { updates.add(upd); } )*
+      ( {!isTxnConditional}? (K_COMMIT K_TRANSACTION) | {isTxnConditional}? (K_END K_IF K_COMMIT K_TRANSACTION))
+    {
+        $expr = new TransactionStatement.Parsed(assignments, select, returning, updates, conditions, references);
+    }
+    ;
+    finally { isParsingTxn = false; }
+
+rowDataReferences returns [List<RowDataReference.Raw> refs]
+    : r1=rowDataReference { refs = new ArrayList<RowDataReference.Raw>(); refs.add(r1); } (',' rN=rowDataReference { refs.add(rN); })*
+    ;
+
+rowDataReference returns [RowDataReference.Raw rawRef]
+    @init { Selectable.RawIdentifier tuple = null; Selectable.Raw selectable = null; }
+    @after { $rawRef = newRowDataReference(tuple, selectable); }
+    : t=sident ('.' s=referenceSelection)? { tuple = t; selectable = s; }
+    ;
+
+referenceSelection returns [Selectable.Raw s]
+    : g=referenceSelectionWithoutField m=selectorModifier[g] {$s = m;}
+    ;
+
+referenceSelectionWithoutField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : sn=sident  { tmp=sn; }
+    | (selectionTypeHint)=> h=selectionTypeHint { tmp=h; }
+    | t=selectionTupleOrNestedSelector { tmp=t; }
+    | l=selectionList { tmp=l; }
+    | m=selectionMapOrSet { tmp=m; }
+    // UDTs are equivalent to maps from the syntax point of view, so the final decision will be done in Selectable.WithMapOrUdt
+    ;
+
+txnConditions returns [List<ConditionStatement.Raw> conditions]
+    @init { conditions = new ArrayList<ConditionStatement.Raw>(); }
+    : txnColumnCondition[conditions] ( K_AND txnColumnCondition[conditions] )*

Review Comment:
   no `OR` support?



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -1522,6 +1587,17 @@ public long getTruncateRpcTimeout()
         return DatabaseDescriptor.getTruncateRpcTimeout(MILLISECONDS);
     }
 
+    public void setTransactionTimeout(long value)
+    {
+        DatabaseDescriptor.setTransactionTimeout(value);
+        logger.info("set transaction timeout to {} ms", value);
+    }
+
+    public long getTransactionTimeout()
+    {
+        return DatabaseDescriptor.getTransactionTimeout(MILLISECONDS);
+    }

Review Comment:
   heh... checked trunk... no timeout uses `String`...  and most don't give a unit!  =/



##########
src/antlr/Parser.g:
##########
@@ -651,6 +717,102 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     | d=deleteStatement  { $statement = d; }
     ;
 
+/**
+ * ex. conditional update returning pre-update values
+ *
+ * BEGIN TRANSACTION
+ *   LET row1 = (SELECT * FROM <table> WHERE k=1 AND c=2);
+ *   LET row2 = (SELECT * FROM <table> WHERE k=2 AND c=2);
+ *   SELECT row1.v, row2.v;
+ *   IF row1.v = 3 AND row2.v = 4 THEN
+ *     UPDATE <table> SET v = row1.v + 1 WHERE k = 1 AND c = 2;
+ *   END IF
+ * COMMIT TRANSACTION
+ *
+ * ex. read-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   SELECT * FROM <table> WHERE k=1 AND c=2;
+ * COMMIT TRANSACTION
+ *
+ * ex. write-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   INSERT INTO <table> (k, c, v) VALUES (0, 0, 1);
+ * COMMIT TRANSACTION
+ */
+batchTxnStatement returns [TransactionStatement.Parsed expr]
+    @init {
+        isParsingTxn = true;
+        List<SelectStatement.RawStatement> assignments = new ArrayList<>();
+        SelectStatement.RawStatement select = null;
+        List<RowDataReference.Raw> returning = null;
+        List<ModificationStatement.Parsed> updates = new ArrayList<>();
+    }
+    : K_BEGIN K_TRANSACTION
+      ( let=letStatement ';' { assignments.add(let); })*
+      ( ( (selectStatement) => s=selectStatement ';' { select = s; }) | ( K_SELECT drs=rowDataReferences ';' { returning = drs; }) )?
+      ( K_IF conditions=txnConditions K_THEN { isTxnConditional = true; } )?
+      ( upd=batchStatementObjective ';' { updates.add(upd); } )*
+      ( {!isTxnConditional}? (K_COMMIT K_TRANSACTION) | {isTxnConditional}? (K_END K_IF K_COMMIT K_TRANSACTION))
+    {
+        $expr = new TransactionStatement.Parsed(assignments, select, returning, updates, conditions, references);
+    }
+    ;
+    finally { isParsingTxn = false; }
+
+rowDataReferences returns [List<RowDataReference.Raw> refs]
+    : r1=rowDataReference { refs = new ArrayList<RowDataReference.Raw>(); refs.add(r1); } (',' rN=rowDataReference { refs.add(rN); })*
+    ;
+
+rowDataReference returns [RowDataReference.Raw rawRef]
+    @init { Selectable.RawIdentifier tuple = null; Selectable.Raw selectable = null; }
+    @after { $rawRef = newRowDataReference(tuple, selectable); }
+    : t=sident ('.' s=referenceSelection)? { tuple = t; selectable = s; }
+    ;
+
+referenceSelection returns [Selectable.Raw s]
+    : g=referenceSelectionWithoutField m=selectorModifier[g] {$s = m;}
+    ;
+
+referenceSelectionWithoutField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : sn=sident  { tmp=sn; }
+    | (selectionTypeHint)=> h=selectionTypeHint { tmp=h; }
+    | t=selectionTupleOrNestedSelector { tmp=t; }
+    | l=selectionList { tmp=l; }
+    | m=selectionMapOrSet { tmp=m; }
+    // UDTs are equivalent to maps from the syntax point of view, so the final decision will be done in Selectable.WithMapOrUdt
+    ;
+
+txnConditions returns [List<ConditionStatement.Raw> conditions]
+    @init { conditions = new ArrayList<ConditionStatement.Raw>(); }
+    : txnColumnCondition[conditions] ( K_AND txnColumnCondition[conditions] )*
+    ;
+
+txnConditionKind returns [ConditionStatement.Kind op]
+    : '='  { $op = ConditionStatement.Kind.EQ; }
+    | '<'  { $op = ConditionStatement.Kind.LT; }
+    | '<=' { $op = ConditionStatement.Kind.LTE; }
+    | '>'  { $op = ConditionStatement.Kind.GT; }
+    | '>=' { $op = ConditionStatement.Kind.GTE; }
+    | '!=' { $op = ConditionStatement.Kind.NEQ; }
+    ;
+
+txnColumnCondition[List<ConditionStatement.Raw> conditions]
+    : lhs=rowDataReference

Review Comment:
   pushed test to show this issue org.apache.cassandra.distributed.test.accord.AccordCQLTest#testConditionRefSideHandling



##########
src/antlr/Parser.g:
##########
@@ -651,6 +717,102 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     | d=deleteStatement  { $statement = d; }
     ;
 
+/**
+ * ex. conditional update returning pre-update values
+ *
+ * BEGIN TRANSACTION
+ *   LET row1 = (SELECT * FROM <table> WHERE k=1 AND c=2);
+ *   LET row2 = (SELECT * FROM <table> WHERE k=2 AND c=2);
+ *   SELECT row1.v, row2.v;
+ *   IF row1.v = 3 AND row2.v = 4 THEN
+ *     UPDATE <table> SET v = row1.v + 1 WHERE k = 1 AND c = 2;
+ *   END IF
+ * COMMIT TRANSACTION
+ *
+ * ex. read-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   SELECT * FROM <table> WHERE k=1 AND c=2;
+ * COMMIT TRANSACTION
+ *
+ * ex. write-only transaction
+ * 
+ * BEGIN TRANSACTION
+ *   INSERT INTO <table> (k, c, v) VALUES (0, 0, 1);
+ * COMMIT TRANSACTION
+ */
+batchTxnStatement returns [TransactionStatement.Parsed expr]
+    @init {
+        isParsingTxn = true;
+        List<SelectStatement.RawStatement> assignments = new ArrayList<>();
+        SelectStatement.RawStatement select = null;
+        List<RowDataReference.Raw> returning = null;
+        List<ModificationStatement.Parsed> updates = new ArrayList<>();
+    }
+    : K_BEGIN K_TRANSACTION
+      ( let=letStatement ';' { assignments.add(let); })*
+      ( ( (selectStatement) => s=selectStatement ';' { select = s; }) | ( K_SELECT drs=rowDataReferences ';' { returning = drs; }) )?
+      ( K_IF conditions=txnConditions K_THEN { isTxnConditional = true; } )?
+      ( upd=batchStatementObjective ';' { updates.add(upd); } )*
+      ( {!isTxnConditional}? (K_COMMIT K_TRANSACTION) | {isTxnConditional}? (K_END K_IF K_COMMIT K_TRANSACTION))
+    {
+        $expr = new TransactionStatement.Parsed(assignments, select, returning, updates, conditions, references);
+    }
+    ;
+    finally { isParsingTxn = false; }
+
+rowDataReferences returns [List<RowDataReference.Raw> refs]
+    : r1=rowDataReference { refs = new ArrayList<RowDataReference.Raw>(); refs.add(r1); } (',' rN=rowDataReference { refs.add(rN); })*
+    ;
+
+rowDataReference returns [RowDataReference.Raw rawRef]
+    @init { Selectable.RawIdentifier tuple = null; Selectable.Raw selectable = null; }
+    @after { $rawRef = newRowDataReference(tuple, selectable); }
+    : t=sident ('.' s=referenceSelection)? { tuple = t; selectable = s; }
+    ;
+
+referenceSelection returns [Selectable.Raw s]
+    : g=referenceSelectionWithoutField m=selectorModifier[g] {$s = m;}
+    ;
+
+referenceSelectionWithoutField returns [Selectable.Raw s]
+    @init { Selectable.Raw tmp = null; }
+    @after { $s = tmp; }
+    : sn=sident  { tmp=sn; }
+    | (selectionTypeHint)=> h=selectionTypeHint { tmp=h; }
+    | t=selectionTupleOrNestedSelector { tmp=t; }
+    | l=selectionList { tmp=l; }
+    | m=selectionMapOrSet { tmp=m; }
+    // UDTs are equivalent to maps from the syntax point of view, so the final decision will be done in Selectable.WithMapOrUdt
+    ;
+
+txnConditions returns [List<ConditionStatement.Raw> conditions]
+    @init { conditions = new ArrayList<ConditionStatement.Raw>(); }
+    : txnColumnCondition[conditions] ( K_AND txnColumnCondition[conditions] )*
+    ;
+
+txnConditionKind returns [ConditionStatement.Kind op]
+    : '='  { $op = ConditionStatement.Kind.EQ; }
+    | '<'  { $op = ConditionStatement.Kind.LT; }
+    | '<=' { $op = ConditionStatement.Kind.LTE; }
+    | '>'  { $op = ConditionStatement.Kind.GT; }
+    | '>=' { $op = ConditionStatement.Kind.GTE; }
+    | '!=' { $op = ConditionStatement.Kind.NEQ; }
+    ;
+
+txnColumnCondition[List<ConditionStatement.Raw> conditions]
+    : lhs=rowDataReference

Review Comment:
   so has to be LHS?
   
   ```
   IF 42 <= a.counter -- rejected?
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041437730


##########
src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java:
##########
@@ -1044,4 +1121,23 @@ public List<Pair<ColumnIdentifier, ColumnCondition.Raw>> getConditions()
             return conditions;
         }
     }
+
+    private static final Constants.Value ONE = new Constants.Value(ByteBufferUtil.bytes(1));
+
+    public SelectStatement createSelectForTxn()
+    {
+        // TODO: get working with static-only updates that don't specify any/all primary key columns
+        Preconditions.checkState(getRestrictions().hasAllPKColumnsRestrictedByEqualities());
+        Selection selection = Selection.forColumns(metadata, Lists.newArrayList(requiresRead), false);

Review Comment:
   feedback branch shows this is an issue



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040186443


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }

Review Comment:
   What if we just use `Object[]` internally and hide all the casting?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1041513115


##########
src/java/org/apache/cassandra/cql3/Constants.java:
##########
@@ -518,19 +517,39 @@ public Substracter(ColumnMetadata column, Term t)
             super(column, t);
         }
 
+        public boolean requiresRead()
+        {
+            return !(column.type instanceof CounterColumnType);
+        }
+
         public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
-            ByteBuffer bytes = t.bindAndGet(params.options);
-            if (bytes == null)
-                throw new InvalidRequestException("Invalid null value for counter increment");
-            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                return;
+            if (column.type instanceof CounterColumnType)
+            {
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for counter increment");
+                if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    return;
 
-            long increment = ByteBufferUtil.toLong(bytes);
-            if (increment == Long.MIN_VALUE)
-                throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
+                long increment = ByteBufferUtil.toLong(bytes);
+                if (increment == Long.MIN_VALUE)
+                    throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
 
-            params.addCounter(column, -increment);
+                params.addCounter(column, -increment);
+            }
+            else if (column.type instanceof NumberType)
+            {
+                NumberType<?> type = (NumberType<?>) column.type;
+                ByteBuffer bytes = t.bindAndGet(params.options);
+                if (bytes == null)
+                    throw new InvalidRequestException("Invalid null value for number increment");

Review Comment:
   This really feels like an oversight...going to just add it to... `Adder`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039960847


##########
src/java/org/apache/cassandra/audit/AuditLogEntryType.java:
##########
@@ -60,6 +60,9 @@
     CREATE_ROLE(AuditLogEntryCategory.DCL),
     USE_KEYSPACE(AuditLogEntryCategory.OTHER),
     DESCRIBE(AuditLogEntryCategory.OTHER),
+    
+    // TODO: Is DML the most appropriate classification, given a transaction can read, write, or both?
+    TRANSACTION(AuditLogEntryCategory.DML),

Review Comment:
   Conceptually, `TRANSACTION = QUERY + DML`...that works.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040141000


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.accord;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Throwables;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.implementation.bind.annotation.This;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+
+import accord.coordinate.Preempted;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.TransactionStatement;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.FailingConsumer;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class AccordTestBase extends TestBaseImpl
+{
+    protected static final AtomicInteger COUNTER = new AtomicInteger(0);
+
+    protected static Cluster sharedCluster;
+    
+    protected String currentTable;
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        sharedCluster = createCluster();
+    }
+
+    @AfterClass
+    public static void teardown()
+    {
+        if (sharedCluster != null)
+            sharedCluster.close();
+    }
+    
+    @Before
+    public void setup()
+    {
+        currentTable = KEYSPACE + ".tbl" + COUNTER.getAndIncrement();
+    }
+
+    protected static void assertRow(Cluster cluster, String query, int k, int c, int v)
+    {
+        Object[][] result = cluster.coordinator(1).execute(query, ConsistencyLevel.QUORUM);
+        assertArrayEquals(new Object[]{new Object[] {k, c, v}}, result);
+    }
+
+    protected void test(String tableDDL, FailingConsumer<Cluster> fn) throws Exception
+    {
+        sharedCluster.schemaChange(tableDDL);
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().createEpochFromConfigUnsafe()));
+
+        // Evict commands from the cache immediately to expose problems loading from disk.
+        sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
+
+        fn.accept(sharedCluster);
+
+        // Reset any messaging filters.
+        sharedCluster.filters().reset();
+    }
+
+    protected void test(FailingConsumer<Cluster> fn) throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, v int, primary key (k, c))", fn);
+    }
+
+    private static Cluster createCluster() throws IOException
+    {
+        // need to up the timeout else tests get flaky
+        // disable vnode for now, but should enable before trunk
+        return init(Cluster.build(2)
+                           .withoutVNodes()
+                           .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
+                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .start());
+    }
+
+    // TODO: Retry on preemption may become unnecessary after the Unified Log is integrated.
+    protected static SimpleQueryResult assertRowEqualsWithPreemptedRetry(Cluster cluster, Object[] row, String check, Object... boundValues)
+    {
+        SimpleQueryResult result = executeWithRetry(cluster, check, boundValues);
+        Assertions.assertThat(result.toObjectArrays()).isEqualTo(row == null ? new Object[0] : new Object[] { row });
+        return result;
+    }
+
+    protected static SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
+    {
+        try
+        {
+            return cluster.coordinator(1).executeWithResult(check, ConsistencyLevel.ANY, boundValues);
+        }
+        catch (Throwable t)
+        {
+            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+                return executeWithRetry(cluster, check, boundValues);

Review Comment:
   Preemption seems like a pretty rare case in my runs, and even if it does happen, I kind of like the fact that we're not waiting (and thereby going to easy on the protocol, hiding failures by allowing an excessive amount of time to pass).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] belliottsmith commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
belliottsmith commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040120268


##########
src/java/org/apache/cassandra/utils/ArraySerializers.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+import java.util.function.IntFunction;
+
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
+
+public class ArraySerializers
+{
+    public static <T> void serializeArray(T[] items, DataOutputPlus out, int version, IVersionedSerializer<T> serializer) throws IOException
+    {
+        out.writeUnsignedVInt(items.length);
+        for (T item : items)
+            serializer.serialize(item, out, version);
+    }
+
+    public static <T> T[] deserializeArray(DataInputPlus in, int version, IVersionedSerializer<T> serializer, IntFunction<T[]> arrayFactory) throws IOException
+    {
+        int size = checkedCast(in.readUnsignedVInt());

Review Comment:
   Probably need to think about naming as vint doesn’t mean int, even if it usually is. Perhaps readUnsignedVInt4b? We do do this a lot, so would be nice to figure out a way to express this, and to do so symmetrically on the write side too



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1040200550


##########
src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TxnDataName implements Comparable<TxnDataName>
+{
+    private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+
+    public enum Kind
+    {
+        USER((byte) 1),
+        RETURNING((byte) 2),
+        AUTO_READ((byte) 3);
+
+        private final byte value;
+
+        Kind(byte value)
+        {
+            this.value = value;
+        }
+
+        public static Kind from(byte b)
+        {
+            switch (b)
+            {
+                case 1:
+                    return USER;
+                case 2:
+                    return RETURNING;
+                case 3:
+                    return AUTO_READ;
+                default:
+                    throw new IllegalArgumentException("Unknown kind: " + b);
+            }
+        }
+    }
+
+    private final Kind kind;
+    private final String[] parts;
+
+    public TxnDataName(Kind kind, String... parts)
+    {
+        this.kind = kind;
+        this.parts = parts;
+    }
+
+    public static TxnDataName user(String name)
+    {
+        return new TxnDataName(Kind.USER, name);
+    }
+
+    public static TxnDataName returning()
+    {
+        return RETURNING;
+    }
+
+    public static TxnDataName partitionRead(TableMetadata metadata, DecoratedKey key, int index)
+    {
+        return new TxnDataName(Kind.AUTO_READ, metadata.keyspace, metadata.name, bytesToString(key.getKey()), String.valueOf(index));
+    }
+
+    private static String bytesToString(ByteBuffer bytes)
+    {
+        return ByteBufferUtil.bytesToHex(bytes);
+    }
+
+    private static ByteBuffer stringToBytes(String string)
+    {
+        return ByteBufferUtil.hexToBytes(string);
+    }
+
+    public Kind getKind()
+    {
+        return kind;
+    }
+
+    public List<String> getParts()
+    {
+        return Collections.unmodifiableList(Arrays.asList(parts));
+    }
+
+    public boolean isFor(TableMetadata metadata)
+    {
+        if (kind != Kind.AUTO_READ)
+            return false;
+        return metadata.keyspace.equals(parts[0])
+               && metadata.name.equals(parts[1]);
+    }
+
+    public DecoratedKey getDecoratedKey(TableMetadata metadata)
+    {
+        checkKind(Kind.AUTO_READ);
+        ByteBuffer data = stringToBytes(parts[2]);
+        return metadata.partitioner.decorateKey(data);
+    }
+
+    public boolean atIndex(int index)
+    {
+        checkKind(Kind.AUTO_READ);
+        return Integer.parseInt(parts[3]) == index;
+    }
+
+    private void checkKind(Kind expected)
+    {
+        if (kind != expected)
+            throw new IllegalStateException("Expected kind " + expected + " but is " + kind);
+    }
+
+    public long estimatedSizeOnHeap()
+    {
+        long size = 0;
+        for (String part : parts)
+            size += part.length();
+        return size;
+    }
+
+    @Override
+    public int compareTo(TxnDataName o)
+    {
+        int rc = kind.compareTo(o.kind);
+        if (rc != 0)
+            return rc;
+        // same kind has same length
+        int size = parts.length;
+        assert o.parts.length == size : String.format("Expected other.parts.length == %d but was %d", size, o.parts.length);
+        for (int i = 0; i < size; i++)
+        {
+            rc = parts[i].compareTo(o.parts[i]);
+            if (rc != 0)
+                return rc;
+        }
+        return 0;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TxnDataName that = (TxnDataName) o;
+        return kind == that.kind && Arrays.equals(parts, that.parts);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = Objects.hash(kind);
+        result = 31 * result + Arrays.hashCode(parts);
+        return result;
+    }
+
+    public String name()
+    {
+        return String.join(":", parts);
+    }
+
+    @Override
+    public String toString()
+    {
+        return kind.name() + ":" + name();
+    }
+
+    public static final IVersionedSerializer<TxnDataName> serializer = new IVersionedSerializer<TxnDataName>()
+    {
+        @Override
+        public void serialize(TxnDataName t, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeByte(t.kind.value);
+            out.writeInt(t.parts.length);

Review Comment:
   Changed this to use vints. Let's wait to see where https://github.com/apache/cassandra/pull/1962#discussion_r1038512017 goes first before delegating to `ArraySerializer`...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1039964434


##########
src/java/org/apache/cassandra/service/accord/txn/AbstractKeySorted.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.service.accord.txn;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+
+import accord.primitives.Keys;
+import org.apache.cassandra.service.accord.api.AccordKey.PartitionKey;
+import org.apache.cassandra.service.accord.api.AccordRoutingKey;
+
+/**
+ * Immutable collection of items, sorted first by their partition key
+ */
+public abstract class AbstractKeySorted<T> implements Iterable<T>
+{
+    public static final String ITEMS_OUT_OF_ORDER_MESSAGE = "Items are out of order ([%s] %s >= [%s] %s)";
+
+    protected final Keys itemKeys;
+    protected final T[] items;
+
+    // items are expected to be sorted
+    public AbstractKeySorted(T[] items)

Review Comment:
   unsafe, we don't know this is sorted; can call `validateOrder` or just sort like we do for `List`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
maedhroz commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042495852


##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -160,18 +169,30 @@ protected ModificationStatement prepareInternal(TableMetadata metadata,
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
 
-                Term.Raw value = columnValues.get(i);
+                Object value = columnValues.get(i);
 
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(columnNames.get(i), Operator.EQ, value));
+                    checkTrue(value instanceof Term.Raw, "value references can't be used with primary key columns");

Review Comment:
   Fixed and added a new test in `TransactionStatementTest`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1042724816


##########
src/java/org/apache/cassandra/cql3/statements/TransactionStatement.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import accord.api.Key;
+import accord.primitives.Keys;
+import accord.primitives.Txn;
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.ResultSetBuilder;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.transactions.RowDataReference;
+import org.apache.cassandra.cql3.transactions.ConditionStatement;
+import org.apache.cassandra.cql3.transactions.ReferenceOperation;
+import org.apache.cassandra.cql3.transactions.SelectReferenceSource;
+import org.apache.cassandra.db.ReadQuery;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SinglePartitionReadQuery;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.service.accord.txn.TxnCondition;
+import org.apache.cassandra.service.accord.txn.TxnData;
+import org.apache.cassandra.service.accord.txn.TxnDataName;
+import org.apache.cassandra.service.accord.txn.TxnNamedRead;
+import org.apache.cassandra.service.accord.txn.TxnQuery;
+import org.apache.cassandra.service.accord.txn.TxnRead;
+import org.apache.cassandra.service.accord.txn.TxnReference;
+import org.apache.cassandra.service.accord.txn.TxnUpdate;
+import org.apache.cassandra.service.accord.txn.TxnWrite;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.LazyToString;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
+public class TransactionStatement implements CQLStatement
+{
+    public static final String DUPLICATE_TUPLE_NAME_MESSAGE = "The name '%s' has already been used by a LET assignment.";
+    public static final String INCOMPLETE_PRIMARY_KEY_LET_MESSAGE = "SELECT in LET assignment without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String INCOMPLETE_PRIMARY_KEY_SELECT_MESSAGE = "Normal SELECT without LIMIT 1 must specify all primary key elements; CQL %s";
+    public static final String NO_CONDITIONS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify their own conditions.";
+    public static final String NO_TIMESTAMPS_IN_UPDATES_MESSAGE = "Updates within transactions may not specify custom timestamps.";
+    public static final String EMPTY_TRANSACTION_MESSAGE = "Transaction contains no reads or writes";
+    public static final String SELECT_REFS_NEED_COLUMN_MESSAGE = "SELECT references must specify a column.";
+
+    static class NamedSelect
+    {
+        final TxnDataName name;
+        final SelectStatement select;
+
+        public NamedSelect(TxnDataName name, SelectStatement select)
+        {
+            this.name = name;
+            this.select = select;
+        }
+    }
+
+    private final List<NamedSelect> assignments;
+    private final NamedSelect returningSelect;
+    private final List<RowDataReference> returningReferences;
+    private final List<ModificationStatement> updates;
+    private final List<ConditionStatement> conditions;
+
+    private final VariableSpecifications bindVariables;
+
+    public TransactionStatement(List<NamedSelect> assignments,
+                                NamedSelect returningSelect,
+                                List<RowDataReference> returningReferences,
+                                List<ModificationStatement> updates,
+                                List<ConditionStatement> conditions,
+                                VariableSpecifications bindVariables)
+    {
+        this.assignments = assignments;
+        this.returningSelect = returningSelect;
+        this.returningReferences = returningReferences;
+        this.updates = updates;
+        this.conditions = conditions;
+        this.bindVariables = bindVariables;
+    }
+
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public void authorize(ClientState state)
+    {
+        // Assess read permissions for all data from both explicit LET statements and generated reads.
+        for (NamedSelect let : assignments)
+            let.select.authorize(state);
+
+        if (returningSelect != null)
+            returningSelect.select.authorize(state);
+
+        for (ModificationStatement update : updates)
+            update.authorize(state);
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        for (ModificationStatement statement : updates)

Review Comment:
   why would we want to avoid if we wouldn't normally?
   
   ```
   SELECT ... ALLOW FILTERING; -- rejected by guardrails
   ```
   
   ```
   BEGIN
     SELECT ... ALLOW FILTERING; -- allowed because we don't validate.
   COMMIT
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #1962: CASSANDRA-17719 CEP-15: Multi-Partition Transaction CQL Support (Alpha)

Posted by GitBox <gi...@apache.org>.
dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1043716558


##########
src/java/org/apache/cassandra/cql3/transactions/SelectReferenceSource.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.transactions;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+public class SelectReferenceSource implements RowDataReference.ReferenceSource
+{
+    public static final String COLUMN_NOT_IN_SELECT_MESSAGE = "%s refererences a column not included in the select";
+    private final SelectStatement statement;
+    private final Set<ColumnMetadata> selectedColumns;
+    private final TableMetadata metadata;
+
+    public SelectReferenceSource(SelectStatement statement)
+    {
+        this.statement = statement;
+        this.metadata = statement.table;
+        Selection selection = statement.getSelection();
+        selectedColumns = new HashSet<>(selection.getColumns());
+    }
+
+    @Override
+    public boolean isPointSelect()
+    {
+        return statement.getRestrictions().hasAllPKColumnsRestrictedByEqualities()
+               || statement.getLimit(QueryOptions.DEFAULT) == 1;

Review Comment:
   pushed fix to feedback branch: e903eb9e14404f39111d905615f3ce75606945a7



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org