dcapwell commented on code in PR #1962:
URL: https://github.com/apache/cassandra/pull/1962#discussion_r1038557424


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

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



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

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



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

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



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

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



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

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



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

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



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

To unsubscribe, e-mail: [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