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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to