|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.cassandra.cql3.constraints; |
| 20 | + |
| 21 | +import java.io.IOException; |
| 22 | +import java.nio.ByteBuffer; |
| 23 | +import java.util.function.Function; |
| 24 | + |
| 25 | +import org.apache.cassandra.cql3.ColumnIdentifier; |
| 26 | +import org.apache.cassandra.cql3.CqlBuilder; |
| 27 | +import org.apache.cassandra.db.TypeSizes; |
| 28 | +import org.apache.cassandra.db.marshal.AbstractType; |
| 29 | +import org.apache.cassandra.io.util.DataInputPlus; |
| 30 | +import org.apache.cassandra.io.util.DataOutputPlus; |
| 31 | +import org.apache.cassandra.schema.ColumnMetadata; |
| 32 | +import org.apache.cassandra.tcm.serialization.MetadataSerializer; |
| 33 | +import org.apache.cassandra.tcm.serialization.Version; |
| 34 | +import org.apache.cassandra.utils.LocalizeString; |
| 35 | + |
| 36 | +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.UNARY_FUNCTION; |
| 37 | + |
| 38 | +public class UnaryFunctionColumnConstraint implements ColumnConstraint<UnaryFunctionColumnConstraint> |
| 39 | +{ |
| 40 | + public static final Serializer serializer = new Serializer(); |
| 41 | + |
| 42 | + public final ConstraintFunction function; |
| 43 | + public final ColumnIdentifier columnName; |
| 44 | + |
| 45 | + public final static class Raw |
| 46 | + { |
| 47 | + public final ConstraintFunction function; |
| 48 | + public final ColumnIdentifier columnName; |
| 49 | + |
| 50 | + public Raw(ColumnIdentifier functionName, ColumnIdentifier columnName) |
| 51 | + { |
| 52 | + this.columnName = columnName; |
| 53 | + function = createConstraintFunction(functionName.toCQLString(), columnName); |
| 54 | + } |
| 55 | + |
| 56 | + public UnaryFunctionColumnConstraint prepare() |
| 57 | + { |
| 58 | + return new UnaryFunctionColumnConstraint(function, columnName); |
| 59 | + } |
| 60 | + } |
| 61 | + |
| 62 | + private enum Functions |
| 63 | + { |
| 64 | + NOT_NULL(NotNullConstraint::new); |
| 65 | + |
| 66 | + private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; |
| 67 | + |
| 68 | + Functions(Function<ColumnIdentifier, ConstraintFunction> functionCreator) |
| 69 | + { |
| 70 | + this.functionCreator = functionCreator; |
| 71 | + } |
| 72 | + } |
| 73 | + |
| 74 | + private static ConstraintFunction createConstraintFunction(String functionName, ColumnIdentifier columnName) |
| 75 | + { |
| 76 | + try |
| 77 | + { |
| 78 | + return Functions.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName); |
| 79 | + } |
| 80 | + catch (IllegalArgumentException ex) |
| 81 | + { |
| 82 | + throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); |
| 83 | + } |
| 84 | + } |
| 85 | + |
| 86 | + private UnaryFunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName) |
| 87 | + { |
| 88 | + this.function = function; |
| 89 | + this.columnName = columnName; |
| 90 | + } |
| 91 | + |
| 92 | + @Override |
| 93 | + public MetadataSerializer<UnaryFunctionColumnConstraint> serializer() |
| 94 | + { |
| 95 | + return serializer; |
| 96 | + } |
| 97 | + |
| 98 | + @Override |
| 99 | + public void appendCqlTo(CqlBuilder builder) |
| 100 | + { |
| 101 | + builder.append(toString()); |
| 102 | + } |
| 103 | + |
| 104 | + @Override |
| 105 | + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException |
| 106 | + { |
| 107 | + function.evaluate(valueType, columnValue); |
| 108 | + } |
| 109 | + |
| 110 | + @Override |
| 111 | + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException |
| 112 | + { |
| 113 | + function.validate(columnMetadata); |
| 114 | + } |
| 115 | + |
| 116 | + @Override |
| 117 | + public ConstraintType getConstraintType() |
| 118 | + { |
| 119 | + return UNARY_FUNCTION; |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + public String toString() |
| 124 | + { |
| 125 | + return function.getName() + '(' + columnName + ") "; |
| 126 | + } |
| 127 | + |
| 128 | + public static class Serializer implements MetadataSerializer<UnaryFunctionColumnConstraint> |
| 129 | + { |
| 130 | + @Override |
| 131 | + public void serialize(UnaryFunctionColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException |
| 132 | + { |
| 133 | + out.writeUTF(columnConstraint.function.getName()); |
| 134 | + out.writeUTF(columnConstraint.columnName.toCQLString()); |
| 135 | + } |
| 136 | + |
| 137 | + @Override |
| 138 | + public UnaryFunctionColumnConstraint deserialize(DataInputPlus in, Version version) throws IOException |
| 139 | + { |
| 140 | + String functionName = in.readUTF(); |
| 141 | + ConstraintFunction function; |
| 142 | + String columnNameString = in.readUTF(); |
| 143 | + ColumnIdentifier columnName = new ColumnIdentifier(columnNameString, true); |
| 144 | + try |
| 145 | + { |
| 146 | + function = createConstraintFunction(functionName, columnName); |
| 147 | + } |
| 148 | + catch (Exception e) |
| 149 | + { |
| 150 | + throw new IOException(e); |
| 151 | + } |
| 152 | + |
| 153 | + return new UnaryFunctionColumnConstraint(function, columnName); |
| 154 | + } |
| 155 | + |
| 156 | + @Override |
| 157 | + public long serializedSize(UnaryFunctionColumnConstraint columnConstraint, Version version) |
| 158 | + { |
| 159 | + return TypeSizes.sizeof(columnConstraint.function.getClass().getName()) |
| 160 | + + TypeSizes.sizeof(columnConstraint.columnName.toCQLString()); |
| 161 | + } |
| 162 | + } |
| 163 | +} |
0 commit comments