beobal commented on code in PR #3562:
URL: https://github.com/apache/cassandra/pull/3562#discussion_r1930988583


##########
src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.constraints;
+
+import java.io.IOException;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.utils.LocalizeString;
+
+public class FunctionColumnConstraint implements 
ColumnConstraint<FunctionColumnConstraint>
+{
+    public static final Serializer serializer = new Serializer();
+
+    public final ConstraintFunction function;
+    public final ColumnIdentifier columnName;
+    public final Operator relationType;
+    public final String term;
+
+    public final static class Raw
+    {
+        public final ConstraintFunction function;
+        public final ColumnIdentifier columnName;
+        public final Operator relationType;
+        public final String term;
+
+        public Raw(ColumnIdentifier functionName, ColumnIdentifier columnName, 
Operator relationType, String term)
+        {
+            this.relationType = relationType;
+            this.columnName = columnName;
+            this.term = term;
+            function = createConstraintFunction(functionName.toCQLString(), 
columnName);
+        }
+
+        public FunctionColumnConstraint prepare()
+        {
+            return new FunctionColumnConstraint(function, columnName, 
relationType, term);
+        }
+    }
+
+    private enum Function
+    {
+        LENGTH(LengthConstraint::new);
+
+        private final java.util.function.Function<ColumnIdentifier, 
ConstraintFunction> functionCreator;
+
+        Function(java.util.function.Function<ColumnIdentifier, 
ConstraintFunction> functionCreator)
+        {
+            this.functionCreator = functionCreator;
+        }
+    }
+
+    private static ConstraintFunction createConstraintFunction(String 
functionName, ColumnIdentifier columnName)
+    {
+        try
+        {
+            return 
Function.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName);
+        }
+        catch (IllegalArgumentException ex)
+        {
+            throw new InvalidConstraintDefinitionException("Unrecognized 
constraint function: " + functionName);
+        }
+    }
+
+    private FunctionColumnConstraint(ConstraintFunction function, 
ColumnIdentifier columnName, Operator relationType, String term)
+    {
+        this.function = function;
+        this.columnName = columnName;
+        this.relationType = relationType;
+        this.term = term;
+    }
+
+    @Override
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(toString());
+    }
+
+    @Override
+    public IVersionedSerializer<FunctionColumnConstraint> serializer()
+    {
+        return serializer;
+    }
+
+    @Override
+    public void evaluate(Class<? extends AbstractType> valueType, Object 
columnValue)
+    {
+        function.evaluate(valueType, relationType, term, columnValue);
+    }
+
+    @Override
+    public void validate(ColumnMetadata columnMetadata)
+    {
+        validateArgs(columnMetadata);
+        function.validate(columnMetadata);
+    }
+
+    @Override
+    public ConstraintType getConstraintType()
+    {
+        return ConstraintType.FUNCTION;
+    }
+
+    void validateArgs(ColumnMetadata columnMetadata)
+    {
+        if (!columnMetadata.name.equals(columnName))
+            throw new InvalidConstraintDefinitionException("Function parameter 
should be the column name");
+    }
+
+    @Override
+    public String toString()
+    {
+        return function.getName() + "(" + columnName + ") " + relationType + " 
" + term;
+    }
+
+    public static class Serializer implements 
IVersionedSerializer<FunctionColumnConstraint>
+    {
+        @Override
+        public void serialize(FunctionColumnConstraint columnConstraint, 
DataOutputPlus out, int version) throws IOException
+        {
+            out.writeUTF(columnConstraint.function.getName());
+            out.writeUTF(columnConstraint.columnName.toCQLString());
+            out.writeUTF(columnConstraint.relationType.toString());
+            out.writeUTF(columnConstraint.term);
+        }
+
+        @Override
+        public FunctionColumnConstraint deserialize(DataInputPlus in, int 
version) throws IOException
+        {
+            String functionName = in.readUTF();
+            ConstraintFunction function;
+            String columnNameString = in.readUTF();
+            ColumnIdentifier columnName = new 
ColumnIdentifier(columnNameString, true);
+            try
+            {
+                function = createConstraintFunction(functionName, columnName);
+            }
+            catch (Exception e)
+            {
+                throw new IOException(e);
+            }
+            String relationTypeString = in.readUTF();
+            Operator relationType = Operator.valueOf(relationTypeString);

Review Comment:
   These serializations  are not round trippable, see the comment on 
`ScalarCommentConstraint`.



##########
src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.constraints;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ScalarColumnConstraint implements 
ColumnConstraint<ScalarColumnConstraint>
+{
+    public final ColumnIdentifier param;
+    public final Operator relationType;
+    public final String term;
+
+    public final static Serializer serializer = new Serializer();
+
+    public final static class Raw
+    {
+        public final ColumnIdentifier param;
+        public final Operator relationType;
+        public final String term;
+
+        public Raw(ColumnIdentifier param, Operator relationType, String term)
+        {
+            this.param = param;
+            this.relationType = relationType;
+            this.term = term;
+        }
+
+        public ScalarColumnConstraint prepare()
+        {
+            return new ScalarColumnConstraint(param, relationType, term);
+        }
+    }
+
+    private ScalarColumnConstraint(ColumnIdentifier param, Operator 
relationType, String term)
+    {
+        this.param = param;
+        this.relationType = relationType;
+        this.term = term;
+    }
+
+    @Override
+    public void evaluate(Class<? extends AbstractType> valueType, Object 
columnValue)
+    {
+        Number columnValueNumber;
+        float sizeConstraint;
+
+        try
+        {
+            columnValueNumber = (Number) columnValue;
+            sizeConstraint = Float.parseFloat(term);
+        }
+        catch (NumberFormatException exception)
+        {
+            throw new ConstraintViolationException(param + " and " + term + " 
need to be numbers.");
+        }
+
+        ByteBuffer buffera = 
ByteBufferUtil.bytes(columnValueNumber.floatValue());
+        ByteBuffer bufferb = ByteBufferUtil.bytes(sizeConstraint);
+
+        if (!relationType.isSatisfiedBy(FloatType.instance, buffera, bufferb))
+            throw new ConstraintViolationException(columnValueNumber + " does 
not satisfy length constraint. "
+                                                   + sizeConstraint + " should 
be " + relationType + ' ' + term);
+    }
+
+    @Override
+    public void validate(ColumnMetadata columnMetadata) throws 
InvalidConstraintDefinitionException
+    {
+        if (!columnMetadata.type.isNumber())
+            throw new InvalidConstraintDefinitionException(param + " is not a 
number");
+    }
+
+    @Override
+    public ConstraintType getConstraintType()
+    {
+        return ConstraintType.SCALAR;
+    }
+
+    @Override
+    public String toString()
+    {
+        return param + " " + relationType + " " + term;
+    }
+
+    @Override
+    public IVersionedSerializer<ScalarColumnConstraint> serializer()
+    {
+        return serializer;
+    }
+
+    @Override
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(toString());
+    }
+
+    private static class Serializer implements 
IVersionedSerializer<ScalarColumnConstraint>
+    {
+        @Override
+        public void serialize(ScalarColumnConstraint columnConstraint, 
DataOutputPlus out, int version) throws IOException
+        {
+            out.writeUTF(columnConstraint.param.toString());
+            out.writeUTF(columnConstraint.relationType.toString());
+            out.writeUTF(columnConstraint.term);
+        }
+
+        @Override
+        public ScalarColumnConstraint deserialize(DataInputPlus in, int 
version) throws IOException
+        {
+            ColumnIdentifier param = new ColumnIdentifier(in.readUTF(), true);
+            Operator relationType = Operator.valueOf(in.readUTF());

Review Comment:
   `serialize` writes the `toString` which usually does not return the same 
string as `name()`, making these serializations non-roundtrippable. This should 
probably use the `b` int field on `Operator` like `RowFilter.Serializer` does. 
Or even better, something similar but which would allow us to write a VInt 
rather than an int.



##########
src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.constraints;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+
+
+// group of constraints for the column
+public class ColumnConstraints implements ColumnConstraint<ColumnConstraints>
+{
+    public static final Serializer serializer = new Serializer();
+    public static final ColumnConstraints NO_OP = new Noop();
+
+    private final List<ColumnConstraint<?>> constraints;
+
+    public ColumnConstraints(List<ColumnConstraint<?>> constraints)
+    {
+        this.constraints = constraints;
+    }
+
+    @Override
+    public IVersionedSerializer<ColumnConstraints> serializer()
+    {
+        return serializer;
+    }
+
+    @Override
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        for (ColumnConstraint<?> constraint : constraints)
+            constraint.appendCqlTo(builder);
+    }
+
+    @Override
+    public void evaluate(Class<? extends AbstractType> valueType, Object 
columnValue) throws ConstraintViolationException
+    {
+        for (ColumnConstraint<?> constraint : constraints)
+            constraint.evaluate(valueType, columnValue);
+    }
+
+    public List<ColumnConstraint<?>> getConstraints()
+    {
+        return constraints;
+    }
+
+    public boolean isEmpty()
+    {
+        return constraints.isEmpty();
+    }
+
+    public int getSize()
+    {
+        return constraints.size();
+    }
+
+    // Checks if there is at least one constraint that will perform checks
+    public boolean hasRelevantConstraints()
+    {
+        for (ColumnConstraint c : constraints)
+        {
+            if (c != ColumnConstraints.NO_OP)
+                return true;
+        }
+        return false;
+    }
+
+    @Override
+    public void validate(ColumnMetadata columnMetadata) throws 
InvalidConstraintDefinitionException
+    {
+        if (!columnMetadata.type.isConstrainable())
+            throw new InvalidConstraintDefinitionException("Constraint cannot 
be defined on the column "
+                                                           + 
columnMetadata.name + " of type " + columnMetadata.type.asCQL3Type()
+                                                           + " for the table " 
+ columnMetadata.ksName + "." + columnMetadata.cfName);
+
+        for (ColumnConstraint<?> constraint : constraints)
+            constraint.validate(columnMetadata);
+    }
+
+    @Override
+    public ConstraintType getConstraintType()
+    {
+        return ConstraintType.COMPOSED;
+    }
+
+    private static class Noop extends ColumnConstraints
+    {
+        private Noop()
+        {
+            super(Collections.emptyList());
+        }
+
+        @Override
+        public void validate(ColumnMetadata columnMetadata)
+        {
+            // Do nothing. It is always valid
+        }
+    }
+
+    public final static class Raw
+    {
+        private final List<ColumnConstraint<?>> constraints;
+
+        public Raw(List<ColumnConstraint<?>> constraints)
+        {
+            this.constraints = constraints;
+        }
+
+        public Raw()
+        {
+            this.constraints = Collections.emptyList();
+        }
+
+        public ColumnConstraints prepare()
+        {
+            if (constraints.isEmpty())
+                return NO_OP;
+            return new ColumnConstraints(constraints);
+        }
+    }
+
+    public static class Serializer implements 
IVersionedSerializer<ColumnConstraints>

Review Comment:
   I think this (and the serializers on `ColumnConstraint` & its impls) should 
implement `MetadataSerializer` rather than `IVersionedSerializer`. The latter 
is primarily used for internode messaging, whereas the former is for writing 
entries in the cluster metadata log and for serializing `ClusterMetadata` 
snapshots (which are both persisted locally and sent over the wire). 
Constraints only seem to require serialization as components of schema in a 
`ClusterMetadata`.



##########
src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.constraints;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ScalarColumnConstraint implements 
ColumnConstraint<ScalarColumnConstraint>
+{
+    public final ColumnIdentifier param;
+    public final Operator relationType;
+    public final String term;
+
+    public final static Serializer serializer = new Serializer();
+
+    public final static class Raw
+    {
+        public final ColumnIdentifier param;
+        public final Operator relationType;
+        public final String term;
+
+        public Raw(ColumnIdentifier param, Operator relationType, String term)
+        {
+            this.param = param;
+            this.relationType = relationType;
+            this.term = term;
+        }
+
+        public ScalarColumnConstraint prepare()
+        {
+            return new ScalarColumnConstraint(param, relationType, term);
+        }
+    }
+
+    private ScalarColumnConstraint(ColumnIdentifier param, Operator 
relationType, String term)
+    {
+        this.param = param;
+        this.relationType = relationType;
+        this.term = term;
+    }
+
+    @Override
+    public void evaluate(Class<? extends AbstractType> valueType, Object 
columnValue)
+    {
+        Number columnValueNumber;
+        float sizeConstraint;
+
+        try
+        {
+            columnValueNumber = (Number) columnValue;
+            sizeConstraint = Float.parseFloat(term);
+        }
+        catch (NumberFormatException exception)
+        {
+            throw new ConstraintViolationException(param + " and " + term + " 
need to be numbers.");
+        }
+
+        ByteBuffer buffera = 
ByteBufferUtil.bytes(columnValueNumber.floatValue());
+        ByteBuffer bufferb = ByteBufferUtil.bytes(sizeConstraint);
+
+        if (!relationType.isSatisfiedBy(FloatType.instance, buffera, bufferb))
+            throw new ConstraintViolationException(columnValueNumber + " does 
not satisfy length constraint. "
+                                                   + sizeConstraint + " should 
be " + relationType + ' ' + term);
+    }
+
+    @Override
+    public void validate(ColumnMetadata columnMetadata) throws 
InvalidConstraintDefinitionException
+    {
+        if (!columnMetadata.type.isNumber())
+            throw new InvalidConstraintDefinitionException(param + " is not a 
number");
+    }
+
+    @Override
+    public ConstraintType getConstraintType()
+    {
+        return ConstraintType.SCALAR;
+    }
+
+    @Override
+    public String toString()
+    {
+        return param + " " + relationType + " " + term;
+    }
+
+    @Override
+    public IVersionedSerializer<ScalarColumnConstraint> serializer()
+    {
+        return serializer;
+    }
+
+    @Override
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(toString());
+    }
+
+    private static class Serializer implements 
IVersionedSerializer<ScalarColumnConstraint>
+    {
+        @Override
+        public void serialize(ScalarColumnConstraint columnConstraint, 
DataOutputPlus out, int version) throws IOException
+        {
+            out.writeUTF(columnConstraint.param.toString());
+            out.writeUTF(columnConstraint.relationType.toString());
+            out.writeUTF(columnConstraint.term);
+        }
+
+        @Override
+        public ScalarColumnConstraint deserialize(DataInputPlus in, int 
version) throws IOException
+        {
+            ColumnIdentifier param = new ColumnIdentifier(in.readUTF(), true);
+            Operator relationType = Operator.valueOf(in.readUTF());

Review Comment:
   I found this issue by triggering a metadata snapshot using `nodetool cms 
snapshot` which serializes the current ClusterMetadata, including schema. A 
subsequent restart then reads the most recent snapshot when replaying the 
metadata log during startup, which fails currently. Another way to do this 
would be too add some constraints to the schema in 
`org.apache.cassandra.distributed.test.log.SnapshotTest`.



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