yifan-c commented on code in PR #3562:
URL: https://github.com/apache/cassandra/pull/3562#discussion_r1811067018


##########
NEWS.txt:
##########
@@ -76,6 +76,9 @@ New features
     metadata. In the first instance, this encompasses cluster membership, 
token ownership and schema metadata. See
     
https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-21%3A+Transactional+Cluster+Metadata
 for more detail on
     the motivation and design, and see "Upgrading" below for specific 
instructions on migrating clusters to this system.
+    - CEP-42 Constraints Framework provides more flexibility to Cassandra 
users and operators by providing a set of

Review Comment:
   "provides more flexibility" -> "provides flexibility" since it is not really 
comparing with another thing. 



##########
src/java/org/apache/cassandra/cql3/ConstraintFunction.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedAsymmetricSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class ConstraintFunction

Review Comment:
   The naming of the constraint classes has been bugging me. 
   
   I think `ConstraintFunction` is really `ConstraintFunctionExpression`
   And `CqlConstraintFunctionExecutor` is really `ConstraintFunction`.
   
   Basically, it is `ConstraintFunctionExpression(ConstraintFunction, 
arguments)`. In cql, `LENGTH(i) < 10` is the expression with function, and 
`LENGTH()` is the function.



##########
pylib/cqlshlib/cqlhandling.py:
##########
@@ -29,7 +29,7 @@
                          'full', 'grant', 'if', 'in', 'index', 'infinity', 
'insert', 'into', 'is', 'keyspace', 'limit',
                          'materialized', 'modify', 'nan', 'norecursive', 
'not', 'null', 'of', 'on', 'or', 'order',
                          'primary', 'rename', 'revoke', 'schema', 'select', 
'set', 'table', 'to', 'token', 'truncate',
-                         'unlogged', 'update', 'use', 'using', 'view', 
'where', 'with'}
+                         'unlogged', 'update', 'use', 'using', 'view', 
'where', 'with', 'check', 'length', 'constraint'}

Review Comment:
   Need to update the reversed keywords list.



##########
src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java:
##########
@@ -153,6 +172,10 @@ public ModificationStatement(StatementType type,
             modifiedColumns = metadata.regularAndStaticColumns();
 
         this.updatedColumns = modifiedColumns;
+
+        this.pkRestrictedColumns = 
Columns.from(restrictions.getRestrictions(ColumnMetadata.Kind.PARTITION_KEY).columns());
+        this.clusteringRestrictedColumns = 
Columns.from(restrictions.getRestrictions(ColumnMetadata.Kind.CLUSTERING).columns());

Review Comment:
   Why getting those columns this way? I think you can lookup the column 
metadata with the column names from `columnValues` map. 
   
   Loop through `metadata.columns()` and progress when the column name matches



##########
src/java/org/apache/cassandra/cql3/ConstraintFunction.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.tcm.serialization.Version;
+
+public class ConstraintFunction
+{
+    public final CqlConstraintFunctionExecutor executor;
+    public final List<ColumnIdentifier> arg;
+
+    public static Serializer serializer = new Serializer();
+
+    public ConstraintFunction(CqlConstraintFunctionExecutor executor, 
List<ColumnIdentifier> arg)
+    {
+        this.executor = executor;
+        this.arg = arg;
+    }
+
+
+    public void checkConstraint(Operator relationType, String term, 
TableMetadata tableMetadata, Map<String, String> columnValues)
+    {
+        executor.checkConstraint(this.arg, relationType, term, tableMetadata, 
columnValues);
+    }
+
+    public void validateConstraint(Operator relationType, String term, 
TableMetadata tableMetadata)
+    {
+        executor.validate(this.arg, relationType, term, tableMetadata);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<String> argsString = arg.stream().map(a -> 
a.toCQLString()).collect(Collectors.toList());
+        String args = String.join(", ", argsString);
+        return String.format("%s(%s)", executor.getName(), args);
+    }
+
+    public static class Serializer
+    {
+        public void serialize(ConstraintFunction constraintFunction, 
DataOutputPlus out, Version version) throws IOException
+        {
+            out.writeUTF(constraintFunction.executor.getClass().getName());
+            out.writeUnsignedVInt32(constraintFunction.arg.size());
+            for (ColumnIdentifier arg : constraintFunction.arg)
+            {
+                out.writeUTF(arg.toString());
+            }
+        }
+
+        public ConstraintFunction deserialize(DataInputPlus in) throws 
IOException
+        {
+            String executorClass = in.readUTF();
+            CqlConstraintFunctionExecutor executor;
+            try
+            {
+                executor = (CqlConstraintFunctionExecutor) 
Class.forName(executorClass).getConstructor().newInstance();

Review Comment:
   Can you lookup the instance by name, instead of creating new instances?



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -333,7 +345,8 @@ protected ModificationStatement prepareInternal(ClientState 
state,
                                        operations,
                                        restrictions,
                                        conditions,
-                                       attrs);
+                                       attrs,
+                                       java.util.Map.of());

Review Comment:
   No constraints applied if updating. 



##########
src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java:
##########
@@ -316,11 +337,45 @@ public void validateTimestamp(QueryState queryState, 
QueryOptions options)
         Guardrails.minimumAllowableTimestamp.guard(ts, table(), false, 
queryState.getClientState());
     }
 
+    public void validateConstraints()

Review Comment:
   Constraints are only applicable for inserts/updates. `ModificationStatement` 
applies to deletion too.
   
   Asking 1. Can you relocate the validation to `UpdateStatement.java` file?
   
   Asking 2. Constraints validation can be performed easier before `execute`, 
during `prepare`, where it has all the bound values and corresponding column 
metadata. The constraints should be validated before an `UpdateStatement` can 
be created. 



##########
NEWS.txt:
##########
@@ -76,6 +76,9 @@ New features
     metadata. In the first instance, this encompasses cluster membership, 
token ownership and schema metadata. See
     
https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-21%3A+Transactional+Cluster+Metadata
 for more detail on
     the motivation and design, and see "Upgrading" below for specific 
instructions on migrating clusters to this system.
+    - CEP-42 Constraints Framework provides more flexibility to Cassandra 
users and operators by providing a set of
+    different usable constraints at a more granular level, that will ease 
validations at application level and protect

Review Comment:
   - "different usable constraints" -> "usable constraints". different is 
redundant in the context.
   - "at a more granular level" -> "at table level" to be specific. 



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -190,13 +194,20 @@ protected ModificationStatement 
prepareInternal(ClientState state,
                                                                            
false,
                                                                            
false);
 
+            Map<String, String> columnMap = new HashMap<>();
+            for (int i = 0; i < columnNames.size(); i++)
+            {
+                columnMap.put(columnNames.get(i).toString(), 
columnValues.get(i).toString());
+            }

Review Comment:
   hmmm... casting value to string and back is really inefficient. 



##########
src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java:
##########
@@ -267,7 +278,8 @@ protected ModificationStatement prepareInternal(ClientState 
state,
                                        operations,
                                        restrictions,
                                        conditions,
-                                       attrs);
+                                       attrs,
+                                       java.util.Map.of());

Review Comment:
   No constraints applied if inserting json.



##########
src/java/org/apache/cassandra/cql3/ConstraintCondition.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.io.IVersionedAsymmetricSerializer;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+/**
+ * Common class for the conditions that a CQL Constraint needs to implement to 
be integrated in the
+ * CQL Constraints framework.
+ */
+public interface ConstraintCondition
+{
+    IVersionedAsymmetricSerializer<ConstraintCondition, ConstraintCondition> 
getSerializer();
+
+    /**
+     * Method that evaluates the condition. It can either succeed or throw a 
{@link ConstraintViolationException}.
+     *
+     * @param columnValues Column values to be evaluated at write time.
+     * @param columnMetadata Metadata of the column in which the constraint is 
defined.
+     * @param tableMetadata Metadata of the table in which the constraint is 
defined.
+     */
+    void evaluate(Map<String, String> columnValues, ColumnMetadata 
columnMetadata, TableMetadata tableMetadata) throws 
ConstraintViolationException;

Review Comment:
   Why the value of `columnValues` map is `String`?



##########
src/antlr/Parser.g:
##########
@@ -780,9 +780,32 @@ tableDefinition[CreateTableStatement.Raw stmt]
 
 tableColumns[CreateTableStatement.Raw stmt]
     @init { boolean isStatic = false; }
-    : k=ident v=comparatorType (K_STATIC { isStatic = true; })? 
(mask=columnMask)? { $stmt.addColumn(k, v, isStatic, mask); }
+    : k=ident v=comparatorType (K_STATIC { isStatic = true; })? 
(mask=columnMask)? (K_CHECK kconst=cqlConstraintExp[stmt])? { 
$stmt.addColumn(k, v, isStatic, mask, kconst == null ? null : 
kconst.prepare(k)); }
         (K_PRIMARY K_KEY { $stmt.setPartitionKeyColumn(k); })?
     | K_PRIMARY K_KEY '(' tablePartitionKey[stmt] (',' c=ident { 
$stmt.markClusteringColumn(c); } )* ')'
+    | K_CONSTRAINT (cn=ident)? K_CHECK expr=cqlConstraintExp[stmt] { 
$stmt.addTableConstraint(expr.prepareWithName(cn)); }
+    ;
+
+cqlConstraintExp[CreateTableStatement.Raw stmt] returns [CqlConstraint.Raw 
cqlConstraint]
+    : cond=cqlConstraintFunctionCondition[stmt] { cqlConstraint = new 
CqlConstraint.Raw(cond); }
+    ;
+
+cqlConstraintFunctionCondition[CreateTableStatement.Raw stmt] returns 
[ConstraintCondition cond]

Review Comment:
   This indirection (cqlConstraintFunctionCondition) can be eliminated. Can 
expression contain anything else rather than `cqlConstraintFunctionCondition`?



##########
test/distributed/org/apache/cassandra/distributed/test/CqlConstraintsTest.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.ConstraintInvalidException;
+import org.apache.cassandra.cql3.ConstraintViolationException;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.assertj.core.api.Assertions;
+import org.assertj.core.api.Condition;
+
+public class CqlConstraintsTest extends TestBaseImpl

Review Comment:
   1. Add test cases for update. 
   2. Add test cases for insert json. 
   3. Add qt test cases that go over all applicable data types, i.e. numeric, 
blob, text, varchar, ascii, collections (which length should be applicable to). 



-- 
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