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

