From 792f606a645e172c9be908807bb99fdb48aceac6 Mon Sep 17 00:00:00 2001 From: Avi Kivity Date: Wed, 24 Dec 2014 10:04:55 +0200 Subject: [PATCH] Import cql3 package and sub-packages Contains the cql3 grammer and supporting classes, which will be converted one by one. From commit bf599fb5b062cbcc652da78b7d699e7a01b949ad. --- cql3/AbstractMarker.java | 111 ++ cql3/AssignmentTestable.java | 72 + cql3/Attributes.java | 143 ++ cql3/BatchQueryOptions.java | 111 ++ cql3/CFName.java | 57 + cql3/CQL3Row.java | 41 + cql3/CQL3Type.java | 594 +++++++ cql3/CQLStatement.java | 63 + cql3/ColumnCondition.java | 820 +++++++++ cql3/ColumnIdentifier.java | 182 ++ cql3/ColumnSpecification.java | 53 + cql3/Constants.java | 378 ++++ cql3/Cql.g | 1581 +++++++++++++++++ cql3/ErrorCollector.java | 290 +++ cql3/ErrorListener.java | 44 + cql3/IndexName.java | 65 + cql3/Lists.java | 508 ++++++ cql3/Maps.java | 399 +++++ cql3/MultiColumnRelation.java | 216 +++ cql3/Operation.java | 429 +++++ cql3/Operator.java | 170 ++ cql3/QueryHandler.java | 36 + cql3/QueryOptions.java | 410 +++++ cql3/QueryProcessor.java | 662 +++++++ cql3/Relation.java | 249 +++ cql3/ResultSet.java | 462 +++++ cql3/Sets.java | 334 ++++ cql3/SingleColumnRelation.java | 297 ++++ cql3/Term.java | 177 ++ cql3/TokenRelation.java | 164 ++ cql3/Tuples.java | 413 +++++ cql3/TypeCast.java | 73 + cql3/UTName.java | 63 + cql3/UntypedResultSet.java | 301 ++++ cql3/UpdateParameters.java | 102 ++ cql3/UserOptions.java | 62 + cql3/UserTypes.java | 201 +++ cql3/VariableSpecifications.java | 68 + cql3/functions/AbstractFunction.java | 99 ++ cql3/functions/AggregateFcts.java | 661 +++++++ cql3/functions/AggregateFunction.java | 63 + cql3/functions/BytesConversionFcts.java | 84 + cql3/functions/Function.java | 58 + cql3/functions/FunctionCall.java | 215 +++ cql3/functions/FunctionName.java | 73 + cql3/functions/Functions.java | 329 ++++ cql3/functions/JavaSourceUDFFactory.java | 258 +++ cql3/functions/NativeAggregateFunction.java | 36 + cql3/functions/NativeFunction.java | 44 + cql3/functions/NativeScalarFunction.java | 36 + cql3/functions/ScalarFunction.java | 39 + cql3/functions/ScriptBasedUDF.java | 145 ++ cql3/functions/TimeuuidFcts.java | 94 + cql3/functions/TokenFct.java | 65 + cql3/functions/UDAggregate.java | 206 +++ cql3/functions/UDFunction.java | 244 +++ cql3/functions/UDHelper.java | 125 ++ cql3/functions/UuidFcts.java | 42 + .../AbstractPrimaryKeyRestrictions.java | 36 + cql3/restrictions/AbstractRestriction.java | 129 ++ .../ForwardingPrimaryKeyRestrictions.java | 159 ++ cql3/restrictions/MultiColumnRestriction.java | 520 ++++++ cql3/restrictions/PrimaryKeyRestrictions.java | 40 + cql3/restrictions/Restriction.java | 97 + cql3/restrictions/Restrictions.java | 82 + .../ReversedPrimaryKeyRestrictions.java | 77 + .../SingleColumnPrimaryKeyRestrictions.java | 312 ++++ .../restrictions/SingleColumnRestriction.java | 518 ++++++ .../SingleColumnRestrictions.java | 209 +++ cql3/restrictions/StatementRestrictions.java | 600 +++++++ cql3/restrictions/TermSlice.java | 167 ++ cql3/restrictions/TokenRestriction.java | 224 +++ cql3/selection/AbstractFunctionSelector.java | 119 ++ cql3/selection/AggregateFunctionSelector.java | 66 + cql3/selection/FieldSelector.java | 103 ++ cql3/selection/RawSelector.java | 61 + cql3/selection/ScalarFunctionSelector.java | 67 + cql3/selection/Selectable.java | 246 +++ cql3/selection/Selection.java | 512 ++++++ cql3/selection/Selector.java | 170 ++ cql3/selection/SelectorFactories.java | 189 ++ cql3/selection/SimpleSelector.java | 93 + cql3/selection/WritetimeOrTTLSelector.java | 108 ++ cql3/statements/AlterKeyspaceStatement.java | 97 + cql3/statements/AlterTableStatement.java | 292 +++ cql3/statements/AlterTypeStatement.java | 348 ++++ cql3/statements/AlterUserStatement.java | 92 + cql3/statements/AuthenticationStatement.java | 54 + cql3/statements/AuthorizationStatement.java | 62 + cql3/statements/BatchStatement.java | 419 +++++ cql3/statements/Bound.java | 45 + cql3/statements/CFPropDefs.java | 218 +++ cql3/statements/CFStatement.java | 64 + cql3/statements/CQL3CasRequest.java | 265 +++ cql3/statements/CreateAggregateStatement.java | 194 ++ cql3/statements/CreateFunctionStatement.java | 145 ++ cql3/statements/CreateIndexStatement.java | 201 +++ cql3/statements/CreateKeyspaceStatement.java | 119 ++ cql3/statements/CreateTableStatement.java | 442 +++++ cql3/statements/CreateTriggerStatement.java | 91 + cql3/statements/CreateTypeStatement.java | 133 ++ cql3/statements/CreateUserStatement.java | 75 + cql3/statements/DeleteStatement.java | 149 ++ cql3/statements/DropAggregateStatement.java | 136 ++ cql3/statements/DropFunctionStatement.java | 149 ++ cql3/statements/DropIndexStatement.java | 133 ++ cql3/statements/DropKeyspaceStatement.java | 77 + cql3/statements/DropTableStatement.java | 76 + cql3/statements/DropTriggerStatement.java | 78 + cql3/statements/DropTypeStatement.java | 164 ++ cql3/statements/DropUserStatement.java | 72 + cql3/statements/GrantStatement.java | 43 + cql3/statements/IndexPropDefs.java | 70 + cql3/statements/IndexTarget.java | 114 ++ cql3/statements/KSPropDefs.java | 89 + cql3/statements/ListPermissionsStatement.java | 123 ++ cql3/statements/ListUsersStatement.java | 47 + cql3/statements/ModificationStatement.java | 770 ++++++++ cql3/statements/ParsedStatement.java | 69 + .../PermissionAlteringStatement.java | 66 + cql3/statements/PropertyDefinitions.java | 143 ++ cql3/statements/RequestValidations.java | 194 ++ cql3/statements/RevokeStatement.java | 43 + cql3/statements/SchemaAlteringStatement.java | 104 ++ cql3/statements/SelectStatement.java | 1074 +++++++++++ cql3/statements/TruncateStatement.java | 84 + cql3/statements/UpdateStatement.java | 241 +++ cql3/statements/UseStatement.java | 67 + 128 files changed, 25952 insertions(+) create mode 100644 cql3/AbstractMarker.java create mode 100644 cql3/AssignmentTestable.java create mode 100644 cql3/Attributes.java create mode 100644 cql3/BatchQueryOptions.java create mode 100644 cql3/CFName.java create mode 100644 cql3/CQL3Row.java create mode 100644 cql3/CQL3Type.java create mode 100644 cql3/CQLStatement.java create mode 100644 cql3/ColumnCondition.java create mode 100644 cql3/ColumnIdentifier.java create mode 100644 cql3/ColumnSpecification.java create mode 100644 cql3/Constants.java create mode 100644 cql3/Cql.g create mode 100644 cql3/ErrorCollector.java create mode 100644 cql3/ErrorListener.java create mode 100644 cql3/IndexName.java create mode 100644 cql3/Lists.java create mode 100644 cql3/Maps.java create mode 100644 cql3/MultiColumnRelation.java create mode 100644 cql3/Operation.java create mode 100644 cql3/Operator.java create mode 100644 cql3/QueryHandler.java create mode 100644 cql3/QueryOptions.java create mode 100644 cql3/QueryProcessor.java create mode 100644 cql3/Relation.java create mode 100644 cql3/ResultSet.java create mode 100644 cql3/Sets.java create mode 100644 cql3/SingleColumnRelation.java create mode 100644 cql3/Term.java create mode 100644 cql3/TokenRelation.java create mode 100644 cql3/Tuples.java create mode 100644 cql3/TypeCast.java create mode 100644 cql3/UTName.java create mode 100644 cql3/UntypedResultSet.java create mode 100644 cql3/UpdateParameters.java create mode 100644 cql3/UserOptions.java create mode 100644 cql3/UserTypes.java create mode 100644 cql3/VariableSpecifications.java create mode 100644 cql3/functions/AbstractFunction.java create mode 100644 cql3/functions/AggregateFcts.java create mode 100644 cql3/functions/AggregateFunction.java create mode 100644 cql3/functions/BytesConversionFcts.java create mode 100644 cql3/functions/Function.java create mode 100644 cql3/functions/FunctionCall.java create mode 100644 cql3/functions/FunctionName.java create mode 100644 cql3/functions/Functions.java create mode 100644 cql3/functions/JavaSourceUDFFactory.java create mode 100644 cql3/functions/NativeAggregateFunction.java create mode 100644 cql3/functions/NativeFunction.java create mode 100644 cql3/functions/NativeScalarFunction.java create mode 100644 cql3/functions/ScalarFunction.java create mode 100644 cql3/functions/ScriptBasedUDF.java create mode 100644 cql3/functions/TimeuuidFcts.java create mode 100644 cql3/functions/TokenFct.java create mode 100644 cql3/functions/UDAggregate.java create mode 100644 cql3/functions/UDFunction.java create mode 100644 cql3/functions/UDHelper.java create mode 100644 cql3/functions/UuidFcts.java create mode 100644 cql3/restrictions/AbstractPrimaryKeyRestrictions.java create mode 100644 cql3/restrictions/AbstractRestriction.java create mode 100644 cql3/restrictions/ForwardingPrimaryKeyRestrictions.java create mode 100644 cql3/restrictions/MultiColumnRestriction.java create mode 100644 cql3/restrictions/PrimaryKeyRestrictions.java create mode 100644 cql3/restrictions/Restriction.java create mode 100644 cql3/restrictions/Restrictions.java create mode 100644 cql3/restrictions/ReversedPrimaryKeyRestrictions.java create mode 100644 cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java create mode 100644 cql3/restrictions/SingleColumnRestriction.java create mode 100644 cql3/restrictions/SingleColumnRestrictions.java create mode 100644 cql3/restrictions/StatementRestrictions.java create mode 100644 cql3/restrictions/TermSlice.java create mode 100644 cql3/restrictions/TokenRestriction.java create mode 100644 cql3/selection/AbstractFunctionSelector.java create mode 100644 cql3/selection/AggregateFunctionSelector.java create mode 100644 cql3/selection/FieldSelector.java create mode 100644 cql3/selection/RawSelector.java create mode 100644 cql3/selection/ScalarFunctionSelector.java create mode 100644 cql3/selection/Selectable.java create mode 100644 cql3/selection/Selection.java create mode 100644 cql3/selection/Selector.java create mode 100644 cql3/selection/SelectorFactories.java create mode 100644 cql3/selection/SimpleSelector.java create mode 100644 cql3/selection/WritetimeOrTTLSelector.java create mode 100644 cql3/statements/AlterKeyspaceStatement.java create mode 100644 cql3/statements/AlterTableStatement.java create mode 100644 cql3/statements/AlterTypeStatement.java create mode 100644 cql3/statements/AlterUserStatement.java create mode 100644 cql3/statements/AuthenticationStatement.java create mode 100644 cql3/statements/AuthorizationStatement.java create mode 100644 cql3/statements/BatchStatement.java create mode 100644 cql3/statements/Bound.java create mode 100644 cql3/statements/CFPropDefs.java create mode 100644 cql3/statements/CFStatement.java create mode 100644 cql3/statements/CQL3CasRequest.java create mode 100644 cql3/statements/CreateAggregateStatement.java create mode 100644 cql3/statements/CreateFunctionStatement.java create mode 100644 cql3/statements/CreateIndexStatement.java create mode 100644 cql3/statements/CreateKeyspaceStatement.java create mode 100644 cql3/statements/CreateTableStatement.java create mode 100644 cql3/statements/CreateTriggerStatement.java create mode 100644 cql3/statements/CreateTypeStatement.java create mode 100644 cql3/statements/CreateUserStatement.java create mode 100644 cql3/statements/DeleteStatement.java create mode 100644 cql3/statements/DropAggregateStatement.java create mode 100644 cql3/statements/DropFunctionStatement.java create mode 100644 cql3/statements/DropIndexStatement.java create mode 100644 cql3/statements/DropKeyspaceStatement.java create mode 100644 cql3/statements/DropTableStatement.java create mode 100644 cql3/statements/DropTriggerStatement.java create mode 100644 cql3/statements/DropTypeStatement.java create mode 100644 cql3/statements/DropUserStatement.java create mode 100644 cql3/statements/GrantStatement.java create mode 100644 cql3/statements/IndexPropDefs.java create mode 100644 cql3/statements/IndexTarget.java create mode 100644 cql3/statements/KSPropDefs.java create mode 100644 cql3/statements/ListPermissionsStatement.java create mode 100644 cql3/statements/ListUsersStatement.java create mode 100644 cql3/statements/ModificationStatement.java create mode 100644 cql3/statements/ParsedStatement.java create mode 100644 cql3/statements/PermissionAlteringStatement.java create mode 100644 cql3/statements/PropertyDefinitions.java create mode 100644 cql3/statements/RequestValidations.java create mode 100644 cql3/statements/RevokeStatement.java create mode 100644 cql3/statements/SchemaAlteringStatement.java create mode 100644 cql3/statements/SelectStatement.java create mode 100644 cql3/statements/TruncateStatement.java create mode 100644 cql3/statements/UpdateStatement.java create mode 100644 cql3/statements/UseStatement.java diff --git a/cql3/AbstractMarker.java b/cql3/AbstractMarker.java new file mode 100644 index 0000000000..990bf6816b --- /dev/null +++ b/cql3/AbstractMarker.java @@ -0,0 +1,111 @@ +/* + * 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 org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A single bind marker. + */ +public abstract class AbstractMarker extends Term.NonTerminal +{ + protected final int bindIndex; + protected final ColumnSpecification receiver; + + protected AbstractMarker(int bindIndex, ColumnSpecification receiver) + { + this.bindIndex = bindIndex; + this.receiver = receiver; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + boundNames.add(bindIndex, receiver); + } + + public boolean containsBindMarker() + { + return true; + } + + /** + * A parsed, but non prepared, bind marker. + */ + public static class Raw implements Term.Raw + { + protected final int bindIndex; + + public Raw(int bindIndex) + { + this.bindIndex = bindIndex; + } + + public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof CollectionType)) + return new Constants.Marker(bindIndex, receiver); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: return new Lists.Marker(bindIndex, receiver); + case SET: return new Sets.Marker(bindIndex, receiver); + case MAP: return new Maps.Marker(bindIndex, receiver); + } + throw new AssertionError(); + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + + @Override + public String toString() + { + return "?"; + } + } + + /** + * A raw placeholder for multiple values of the same type for a single column. + * For example, "SELECT ... WHERE user_id IN ?'. + * + * Because a single type is used, a List is used to represent the values. + */ + public static class INRaw extends Raw + { + public INRaw(int bindIndex) + { + super(bindIndex); + } + + private static ColumnSpecification makeInReceiver(ColumnSpecification receiver) + { + ColumnIdentifier inName = new ColumnIdentifier("in(" + receiver.name + ")", true); + return new ColumnSpecification(receiver.ksName, receiver.cfName, inName, ListType.getInstance(receiver.type, false)); + } + + @Override + public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + return new Lists.Marker(bindIndex, makeInReceiver(receiver)); + } + } +} diff --git a/cql3/AssignmentTestable.java b/cql3/AssignmentTestable.java new file mode 100644 index 0000000000..41b80eb87b --- /dev/null +++ b/cql3/AssignmentTestable.java @@ -0,0 +1,72 @@ +/* + * 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.Collection; + +public interface AssignmentTestable +{ + /** + * @return whether this object can be assigned to the provided receiver. We distinguish + * between 3 values: + * - EXACT_MATCH if this object is exactly of the type expected by the receiver + * - WEAKLY_ASSIGNABLE if this object is not exactly the expected type but is assignable nonetheless + * - NOT_ASSIGNABLE if it's not assignable + * Most caller should just call the isAssignable() method on the result, though functions have a use for + * testing "strong" equality to decide the most precise overload to pick when multiple could match. + */ + public TestResult testAssignment(String keyspace, ColumnSpecification receiver); + + public enum TestResult + { + EXACT_MATCH, WEAKLY_ASSIGNABLE, NOT_ASSIGNABLE; + + public boolean isAssignable() + { + return this != NOT_ASSIGNABLE; + } + + public boolean isExactMatch() + { + return this == EXACT_MATCH; + } + + // Test all elements of toTest for assignment. If all are exact match, return exact match. If any is not assignable, + // return not assignable. Otherwise, return weakly assignable. + public static TestResult testAll(String keyspace, ColumnSpecification receiver, Collection toTest) + { + TestResult res = EXACT_MATCH; + for (AssignmentTestable rt : toTest) + { + if (rt == null) + { + res = WEAKLY_ASSIGNABLE; + continue; + } + + TestResult t = rt.testAssignment(keyspace, receiver); + if (t == NOT_ASSIGNABLE) + return NOT_ASSIGNABLE; + + if (t == WEAKLY_ASSIGNABLE) + res = WEAKLY_ASSIGNABLE; + } + return res; + } + } +} diff --git a/cql3/Attributes.java b/cql3/Attributes.java new file mode 100644 index 0000000000..851e1b48d9 --- /dev/null +++ b/cql3/Attributes.java @@ -0,0 +1,143 @@ +/* + * 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.nio.ByteBuffer; + +import org.apache.cassandra.db.ExpiringCell; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.MarshalException; + +/** + * Utility class for the Parser to gather attributes for modification + * statements. + */ +public class Attributes +{ + private final Term timestamp; + private final Term timeToLive; + + public static Attributes none() + { + return new Attributes(null, null); + } + + private Attributes(Term timestamp, Term timeToLive) + { + this.timestamp = timestamp; + this.timeToLive = timeToLive; + } + + public boolean usesFunction(String ksName, String functionName) + { + return (timestamp != null && timestamp.usesFunction(ksName, functionName)) + || (timeToLive != null && timeToLive.usesFunction(ksName, functionName)); + } + + public boolean isTimestampSet() + { + return timestamp != null; + } + + public boolean isTimeToLiveSet() + { + return timeToLive != null; + } + + public long getTimestamp(long now, QueryOptions options) throws InvalidRequestException + { + if (timestamp == null) + return now; + + ByteBuffer tval = timestamp.bindAndGet(options); + if (tval == null) + throw new InvalidRequestException("Invalid null value of timestamp"); + + try + { + LongType.instance.validate(tval); + } + catch (MarshalException e) + { + throw new InvalidRequestException("Invalid timestamp value"); + } + + return LongType.instance.compose(tval); + } + + public int getTimeToLive(QueryOptions options) throws InvalidRequestException + { + if (timeToLive == null) + return 0; + + ByteBuffer tval = timeToLive.bindAndGet(options); + if (tval == null) + throw new InvalidRequestException("Invalid null value of TTL"); + + try + { + Int32Type.instance.validate(tval); + } + catch (MarshalException e) + { + throw new InvalidRequestException("Invalid timestamp value"); + } + + int ttl = Int32Type.instance.compose(tval); + if (ttl < 0) + throw new InvalidRequestException("A TTL must be greater or equal to 0"); + + if (ttl > ExpiringCell.MAX_TTL) + throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL)); + + return ttl; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + if (timestamp != null) + timestamp.collectMarkerSpecification(boundNames); + if (timeToLive != null) + timeToLive.collectMarkerSpecification(boundNames); + } + + public static class Raw + { + public Term.Raw timestamp; + public Term.Raw timeToLive; + + public Attributes prepare(String ksName, String cfName) throws InvalidRequestException + { + Term ts = timestamp == null ? null : timestamp.prepare(ksName, timestampReceiver(ksName, cfName)); + Term ttl = timeToLive == null ? null : timeToLive.prepare(ksName, timeToLiveReceiver(ksName, cfName)); + return new Attributes(ts, ttl); + } + + private ColumnSpecification timestampReceiver(String ksName, String cfName) + { + return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[timestamp]", true), LongType.instance); + } + + private ColumnSpecification timeToLiveReceiver(String ksName, String cfName) + { + return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance); + } + } +} diff --git a/cql3/BatchQueryOptions.java b/cql3/BatchQueryOptions.java new file mode 100644 index 0000000000..2fcee5b2a0 --- /dev/null +++ b/cql3/BatchQueryOptions.java @@ -0,0 +1,111 @@ +/* + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.service.QueryState; + +public abstract class BatchQueryOptions +{ + public static BatchQueryOptions DEFAULT = withoutPerStatementVariables(QueryOptions.DEFAULT); + + protected final QueryOptions wrapped; + private final List queryOrIdList; + + protected BatchQueryOptions(QueryOptions wrapped, List queryOrIdList) + { + this.wrapped = wrapped; + this.queryOrIdList = queryOrIdList; + } + + public static BatchQueryOptions withoutPerStatementVariables(QueryOptions options) + { + return new WithoutPerStatementVariables(options, Collections.emptyList()); + } + + public static BatchQueryOptions withPerStatementVariables(QueryOptions options, List> variables, List queryOrIdList) + { + return new WithPerStatementVariables(options, variables, queryOrIdList); + } + + public abstract QueryOptions forStatement(int i); + + public ConsistencyLevel getConsistency() + { + return wrapped.getConsistency(); + } + + public ConsistencyLevel getSerialConsistency() + { + return wrapped.getSerialConsistency(); + } + + public List getQueryOrIdList() + { + return queryOrIdList; + } + + public long getTimestamp(QueryState state) + { + return wrapped.getTimestamp(state); + } + + private static class WithoutPerStatementVariables extends BatchQueryOptions + { + private WithoutPerStatementVariables(QueryOptions wrapped, List queryOrIdList) + { + super(wrapped, queryOrIdList); + } + + public QueryOptions forStatement(int i) + { + return wrapped; + } + } + + private static class WithPerStatementVariables extends BatchQueryOptions + { + private final List perStatementOptions; + + private WithPerStatementVariables(QueryOptions wrapped, List> variables, List queryOrIdList) + { + super(wrapped, queryOrIdList); + this.perStatementOptions = new ArrayList<>(variables.size()); + for (final List vars : variables) + { + perStatementOptions.add(new QueryOptions.QueryOptionsWrapper(wrapped) + { + public List getValues() + { + return vars; + } + }); + } + } + + public QueryOptions forStatement(int i) + { + return perStatementOptions.get(i); + } + } +} diff --git a/cql3/CFName.java b/cql3/CFName.java new file mode 100644 index 0000000000..8446a3535d --- /dev/null +++ b/cql3/CFName.java @@ -0,0 +1,57 @@ +/* + * 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.Locale; + +public class CFName +{ + private String ksName; + private String cfName; + + public void setKeyspace(String ks, boolean keepCase) + { + ksName = keepCase ? ks : ks.toLowerCase(Locale.US); + } + + public void setColumnFamily(String cf, boolean keepCase) + { + cfName = keepCase ? cf : cf.toLowerCase(Locale.US); + } + + public boolean hasKeyspace() + { + return ksName != null; + } + + public String getKeyspace() + { + return ksName; + } + + public String getColumnFamily() + { + return cfName; + } + + @Override + public String toString() + { + return (hasKeyspace() ? (ksName + ".") : "") + cfName; + } +} diff --git a/cql3/CQL3Row.java b/cql3/CQL3Row.java new file mode 100644 index 0000000000..e3e76d18e6 --- /dev/null +++ b/cql3/CQL3Row.java @@ -0,0 +1,41 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; + +import org.apache.cassandra.db.Cell; + +public interface CQL3Row +{ + public ByteBuffer getClusteringColumn(int i); + public Cell getColumn(ColumnIdentifier name); + public List getMultiCellColumn(ColumnIdentifier name); + + public interface Builder + { + public RowIterator group(Iterator cells); + } + + public interface RowIterator extends Iterator + { + public CQL3Row getStaticRow(); + } +} diff --git a/cql3/CQL3Type.java b/cql3/CQL3Type.java new file mode 100644 index 0000000000..98d1b15578 --- /dev/null +++ b/cql3/CQL3Type.java @@ -0,0 +1,594 @@ +/* + * 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.ArrayList; +import java.util.List; + +import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.SyntaxException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public interface CQL3Type +{ + static final Logger logger = LoggerFactory.getLogger(CQL3Type.class); + + public boolean isCollection(); + public AbstractType getType(); + + public enum Native implements CQL3Type + { + ASCII (AsciiType.instance), + BIGINT (LongType.instance), + BLOB (BytesType.instance), + BOOLEAN (BooleanType.instance), + COUNTER (CounterColumnType.instance), + DECIMAL (DecimalType.instance), + DOUBLE (DoubleType.instance), + FLOAT (FloatType.instance), + INET (InetAddressType.instance), + INT (Int32Type.instance), + TEXT (UTF8Type.instance), + TIMESTAMP(TimestampType.instance), + UUID (UUIDType.instance), + VARCHAR (UTF8Type.instance), + VARINT (IntegerType.instance), + TIMEUUID (TimeUUIDType.instance); + + private final AbstractType type; + + private Native(AbstractType type) + { + this.type = type; + } + + public boolean isCollection() + { + return false; + } + + public AbstractType getType() + { + return type; + } + + @Override + public String toString() + { + return super.toString().toLowerCase(); + } + } + + public static class Custom implements CQL3Type + { + private final AbstractType type; + + public Custom(AbstractType type) + { + this.type = type; + } + + public Custom(String className) throws SyntaxException, ConfigurationException + { + this(TypeParser.parse(className)); + } + + public boolean isCollection() + { + return false; + } + + public AbstractType getType() + { + return type; + } + + @Override + public final boolean equals(Object o) + { + if(!(o instanceof Custom)) + return false; + + Custom that = (Custom)o; + return type.equals(that.type); + } + + @Override + public final int hashCode() + { + return type.hashCode(); + } + + @Override + public String toString() + { + return "'" + type + "'"; + } + } + + public static class Collection implements CQL3Type + { + private final CollectionType type; + + public Collection(CollectionType type) + { + this.type = type; + } + + public AbstractType getType() + { + return type; + } + + public boolean isCollection() + { + return true; + } + + @Override + public final boolean equals(Object o) + { + if(!(o instanceof Collection)) + return false; + + Collection that = (Collection)o; + return type.equals(that.type); + } + + @Override + public final int hashCode() + { + return type.hashCode(); + } + + @Override + public String toString() + { + boolean isFrozen = !this.type.isMultiCell(); + StringBuilder sb = new StringBuilder(isFrozen ? "frozen<" : ""); + switch (type.kind) + { + case LIST: + AbstractType listType = ((ListType)type).getElementsType(); + sb.append("list<").append(listType.asCQL3Type()); + break; + case SET: + AbstractType setType = ((SetType)type).getElementsType(); + sb.append("set<").append(setType.asCQL3Type()); + break; + case MAP: + AbstractType keysType = ((MapType)type).getKeysType(); + AbstractType valuesType = ((MapType)type).getValuesType(); + sb.append("map<").append(keysType.asCQL3Type()).append(", ").append(valuesType.asCQL3Type()); + break; + default: + throw new AssertionError(); + } + sb.append(">"); + if (isFrozen) + sb.append(">"); + return sb.toString(); + } + } + + public static class UserDefined implements CQL3Type + { + // Keeping this separatly from type just to simplify toString() + private final String name; + private final UserType type; + + private UserDefined(String name, UserType type) + { + this.name = name; + this.type = type; + } + + public static UserDefined create(UserType type) + { + return new UserDefined(UTF8Type.instance.compose(type.name), type); + } + + public boolean isCollection() + { + return false; + } + + public AbstractType getType() + { + return type; + } + + @Override + public final boolean equals(Object o) + { + if(!(o instanceof UserDefined)) + return false; + + UserDefined that = (UserDefined)o; + return type.equals(that.type); + } + + @Override + public final int hashCode() + { + return type.hashCode(); + } + + @Override + public String toString() + { + return name; + } + } + + public static class Tuple implements CQL3Type + { + private final TupleType type; + + private Tuple(TupleType type) + { + this.type = type; + } + + public static Tuple create(TupleType type) + { + return new Tuple(type); + } + + public boolean isCollection() + { + return false; + } + + public AbstractType getType() + { + return type; + } + + @Override + public final boolean equals(Object o) + { + if(!(o instanceof Tuple)) + return false; + + Tuple that = (Tuple)o; + return type.equals(that.type); + } + + @Override + public final int hashCode() + { + return type.hashCode(); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append("tuple<"); + for (int i = 0; i < type.size(); i++) + { + if (i > 0) + sb.append(", "); + sb.append(type.type(i).asCQL3Type()); + } + sb.append(">"); + return sb.toString(); + } + } + + // For UserTypes, we need to know the current keyspace to resolve the + // actual type used, so Raw is a "not yet prepared" CQL3Type. + public abstract class Raw + { + protected boolean frozen = false; + + protected abstract boolean supportsFreezing(); + + public boolean isCollection() + { + return false; + } + + public boolean isCounter() + { + return false; + } + + public String keyspace() + { + return null; + } + + public void freeze() throws InvalidRequestException + { + String message = String.format("frozen<> is only allowed on collections, tuples, and user-defined types (got %s)", this); + throw new InvalidRequestException(message); + } + + public abstract CQL3Type prepare(String keyspace) throws InvalidRequestException; + + public static Raw from(CQL3Type type) + { + return new RawType(type); + } + + public static Raw userType(UTName name) + { + return new RawUT(name); + } + + public static Raw map(CQL3Type.Raw t1, CQL3Type.Raw t2) + { + return new RawCollection(CollectionType.Kind.MAP, t1, t2); + } + + public static Raw list(CQL3Type.Raw t) + { + return new RawCollection(CollectionType.Kind.LIST, null, t); + } + + public static Raw set(CQL3Type.Raw t) + { + return new RawCollection(CollectionType.Kind.SET, null, t); + } + + public static Raw tuple(List ts) + { + return new RawTuple(ts); + } + + public static Raw frozen(CQL3Type.Raw t) throws InvalidRequestException + { + t.freeze(); + return t; + } + + private static class RawType extends Raw + { + private CQL3Type type; + + private RawType(CQL3Type type) + { + this.type = type; + } + + public CQL3Type prepare(String keyspace) throws InvalidRequestException + { + return type; + } + + protected boolean supportsFreezing() + { + return false; + } + + public boolean isCounter() + { + return type == Native.COUNTER; + } + + @Override + public String toString() + { + return type.toString(); + } + } + + private static class RawCollection extends Raw + { + private final CollectionType.Kind kind; + private final CQL3Type.Raw keys; + private final CQL3Type.Raw values; + + private RawCollection(CollectionType.Kind kind, CQL3Type.Raw keys, CQL3Type.Raw values) + { + this.kind = kind; + this.keys = keys; + this.values = values; + } + + public void freeze() throws InvalidRequestException + { + if (keys != null && keys.supportsFreezing()) + keys.freeze(); + if (values != null && values.supportsFreezing()) + values.freeze(); + frozen = true; + } + + protected boolean supportsFreezing() + { + return true; + } + + public boolean isCollection() + { + return true; + } + + public CQL3Type prepare(String keyspace) throws InvalidRequestException + { + assert values != null : "Got null values type for a collection"; + + if (!frozen && values.supportsFreezing() && !values.frozen) + throw new InvalidRequestException("Non-frozen collections are not allowed inside collections: " + this); + if (values.isCounter()) + throw new InvalidRequestException("Counters are not allowed inside collections: " + this); + + if (keys != null) + { + if (!frozen && keys.supportsFreezing() && !keys.frozen) + throw new InvalidRequestException("Non-frozen collections are not allowed inside collections: " + this); + } + + switch (kind) + { + case LIST: + return new Collection(ListType.getInstance(values.prepare(keyspace).getType(), !frozen)); + case SET: + return new Collection(SetType.getInstance(values.prepare(keyspace).getType(), !frozen)); + case MAP: + assert keys != null : "Got null keys type for a collection"; + return new Collection(MapType.getInstance(keys.prepare(keyspace).getType(), values.prepare(keyspace).getType(), !frozen)); + } + throw new AssertionError(); + } + + @Override + public String toString() + { + String start = frozen? "frozen<" : ""; + String end = frozen ? ">" : ""; + switch (kind) + { + case LIST: return start + "list<" + values + ">" + end; + case SET: return start + "set<" + values + ">" + end; + case MAP: return start + "map<" + keys + ", " + values + ">" + end; + } + throw new AssertionError(); + } + } + + private static class RawUT extends Raw + { + private final UTName name; + + private RawUT(UTName name) + { + this.name = name; + } + + public String keyspace() + { + return name.getKeyspace(); + } + + public void freeze() + { + frozen = true; + } + + public CQL3Type prepare(String keyspace) throws InvalidRequestException + { + if (name.hasKeyspace()) + { + // The provided keyspace is the one of the current statement this is part of. If it's different from the keyspace of + // the UTName, we reject since we want to limit user types to their own keyspace (see #6643) + if (keyspace != null && !keyspace.equals(name.getKeyspace())) + throw new InvalidRequestException(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; " + + "user types can only be used in the keyspace they are defined in", + keyspace, name.getKeyspace())); + } + else + { + name.setKeyspace(keyspace); + } + + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + if (ksm == null) + throw new InvalidRequestException("Unknown keyspace " + name.getKeyspace()); + UserType type = ksm.userTypes.getType(name.getUserTypeName()); + if (type == null) + throw new InvalidRequestException("Unknown type " + name); + + if (!frozen) + throw new InvalidRequestException("Non-frozen User-Defined types are not supported, please use frozen<>"); + + return new UserDefined(name.toString(), type); + } + + protected boolean supportsFreezing() + { + return true; + } + + @Override + public String toString() + { + return name.toString(); + } + } + + private static class RawTuple extends Raw + { + private final List types; + + private RawTuple(List types) + { + this.types = types; + } + + protected boolean supportsFreezing() + { + return true; + } + + public boolean isCollection() + { + return false; + } + + public void freeze() throws InvalidRequestException + { + for (CQL3Type.Raw t : types) + { + if (t.supportsFreezing()) + t.freeze(); + } + frozen = true; + } + + public CQL3Type prepare(String keyspace) throws InvalidRequestException + { + if (!frozen) + freeze(); + + List> ts = new ArrayList<>(types.size()); + for (CQL3Type.Raw t : types) + { + if (t.isCounter()) + throw new InvalidRequestException("Counters are not allowed inside tuples"); + + ts.add(t.prepare(keyspace).getType()); + } + return new Tuple(new TupleType(ts)); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append("tuple<"); + for (int i = 0; i < types.size(); i++) + { + if (i > 0) + sb.append(", "); + sb.append(types.get(i)); + } + sb.append(">"); + return sb.toString(); + } + } + } +} diff --git a/cql3/CQLStatement.java b/cql3/CQLStatement.java new file mode 100644 index 0000000000..d555ec31b9 --- /dev/null +++ b/cql3/CQLStatement.java @@ -0,0 +1,63 @@ +/* + * 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 org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.exceptions.*; + +public interface CQLStatement +{ + /** + * Returns the number of bound terms in this statement. + */ + public int getBoundTerms(); + + /** + * Perform any access verification necessary for the statement. + * + * @param state the current client state + */ + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException; + + /** + * Perform additional validation required by the statment. + * To be overriden by subclasses if needed. + * + * @param state the current client state + */ + public void validate(ClientState state) throws RequestValidationException; + + /** + * Execute the statement and return the resulting result or null if there is no result. + * + * @param state the current query state + * @param options options for this query (consistency, variables, pageSize, ...) + */ + public ResultMessage execute(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException; + + /** + * Variante of execute used for internal query against the system tables, and thus only query the local node. + * + * @param state the current query state + */ + public ResultMessage executeInternal(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException; + + boolean usesFunction(String ksName, String functionName); +} diff --git a/cql3/ColumnCondition.java b/cql3/ColumnCondition.java new file mode 100644 index 0000000000..a8c8cc2ada --- /dev/null +++ b/cql3/ColumnCondition.java @@ -0,0 +1,820 @@ +/* + * 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.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterators; +import static com.google.common.collect.Lists.newArrayList; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.CellNameType; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * A CQL3 condition on the value of a column or collection element. For example, "UPDATE .. IF a = 0". + */ +public class ColumnCondition +{ + + public final ColumnDefinition column; + + // For collection, when testing the equality of a specific element, null otherwise. + private final Term collectionElement; + + private final Term value; // a single value or a marker for a list of IN values + private final List inValues; + + public final Operator operator; + + private ColumnCondition(ColumnDefinition column, Term collectionElement, Term value, List inValues, Operator op) + { + this.column = column; + this.collectionElement = collectionElement; + this.value = value; + this.inValues = inValues; + this.operator = op; + + if (operator != Operator.IN) + assert this.inValues == null; + } + + public static ColumnCondition condition(ColumnDefinition column, Term value, Operator op) + { + return new ColumnCondition(column, null, value, null, op); + } + + public static ColumnCondition condition(ColumnDefinition column, Term collectionElement, Term value, Operator op) + { + return new ColumnCondition(column, collectionElement, value, null, op); + } + + public static ColumnCondition inCondition(ColumnDefinition column, List inValues) + { + return new ColumnCondition(column, null, null, inValues, Operator.IN); + } + + public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, List inValues) + { + return new ColumnCondition(column, collectionElement, null, inValues, Operator.IN); + } + + public static ColumnCondition inCondition(ColumnDefinition column, Term inMarker) + { + return new ColumnCondition(column, null, inMarker, null, Operator.IN); + } + + public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, Term inMarker) + { + return new ColumnCondition(column, collectionElement, inMarker, null, Operator.IN); + } + + public boolean usesFunction(String ksName, String functionName) + { + if (collectionElement != null && collectionElement.usesFunction(ksName, functionName)) + return true; + if (value != null && value.usesFunction(ksName, functionName)) + return true; + if (inValues != null) + for (Term value : inValues) + if (value != null && value.usesFunction(ksName, functionName)) + return true; + return false; + } + + /** + * Collects the column specification for the bind variables of this operation. + * + * @param boundNames the list of column specification where to collect the + * bind variables of this term in. + */ + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + if (collectionElement != null) + collectionElement.collectMarkerSpecification(boundNames); + + if ((operator == Operator.IN) && inValues != null) + { + for (Term value : inValues) + value.collectMarkerSpecification(boundNames); + } + else + { + value.collectMarkerSpecification(boundNames); + } + } + + public ColumnCondition.Bound bind(QueryOptions options) throws InvalidRequestException + { + boolean isInCondition = operator == Operator.IN; + if (column.type instanceof CollectionType) + { + if (collectionElement == null) + return isInCondition ? new CollectionInBound(this, options) : new CollectionBound(this, options); + else + return isInCondition ? new ElementAccessInBound(this, options) : new ElementAccessBound(this, options); + } + return isInCondition ? new SimpleInBound(this, options) : new SimpleBound(this, options); + } + + public static abstract class Bound + { + public final ColumnDefinition column; + public final Operator operator; + + protected Bound(ColumnDefinition column, Operator operator) + { + this.column = column; + this.operator = operator; + } + + /** + * Validates whether this condition applies to {@code current}. + */ + public abstract boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException; + + public ByteBuffer getCollectionElementValue() + { + return null; + } + + protected boolean isSatisfiedByValue(ByteBuffer value, Cell c, AbstractType type, Operator operator, long now) throws InvalidRequestException + { + ByteBuffer columnValue = (c == null || !c.isLive(now)) ? null : c.value(); + return compareWithOperator(operator, type, value, columnValue); + } + + /** Returns true if the operator is satisfied (i.e. "value operator otherValue == true"), false otherwise. */ + protected boolean compareWithOperator(Operator operator, AbstractType type, ByteBuffer value, ByteBuffer otherValue) throws InvalidRequestException + { + if (value == null) + { + switch (operator) + { + case EQ: + return otherValue == null; + case NEQ: + return otherValue != null; + default: + throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator)); + } + } + else if (otherValue == null) + { + // the condition value is not null, so only NEQ can return true + return operator == Operator.NEQ; + } + int comparison = type.compare(otherValue, value); + switch (operator) + { + case EQ: + return comparison == 0; + case LT: + return comparison < 0; + case LTE: + return comparison <= 0; + case GT: + return comparison > 0; + case GTE: + return comparison >= 0; + case NEQ: + return comparison != 0; + default: + // we shouldn't get IN, CONTAINS, or CONTAINS KEY here + throw new AssertionError(); + } + } + + protected Iterator collectionColumns(CellName collection, ColumnFamily cf, final long now) + { + // We are testing for collection equality, so we need to have the expected values *and* only those. + ColumnSlice[] collectionSlice = new ColumnSlice[]{ collection.slice() }; + // Filter live columns, this makes things simpler afterwards + return Iterators.filter(cf.iterator(collectionSlice), new Predicate() + { + public boolean apply(Cell c) + { + // we only care about live columns + return c.isLive(now); + } + }); + } + } + + /** + * A condition on a single non-collection column. This does not support IN operators (see SimpleInBound). + */ + static class SimpleBound extends Bound + { + public final ByteBuffer value; + + private SimpleBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert !(column.type instanceof CollectionType) && condition.collectionElement == null; + assert condition.operator != Operator.IN; + this.value = condition.value.bindAndGet(options); + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException + { + CellName name = current.metadata().comparator.create(rowPrefix, column); + return isSatisfiedByValue(value, current.getColumn(name), column.type, operator, now); + } + } + + /** + * An IN condition on a single non-collection column. + */ + static class SimpleInBound extends Bound + { + public final List inValues; + + private SimpleInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert !(column.type instanceof CollectionType) && condition.collectionElement == null; + assert condition.operator == Operator.IN; + if (condition.inValues == null) + this.inValues = ((Lists.Marker) condition.value).bind(options).getElements(); + else + { + this.inValues = new ArrayList<>(condition.inValues.size()); + for (Term value : condition.inValues) + this.inValues.add(value.bindAndGet(options)); + } + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException + { + CellName name = current.metadata().comparator.create(rowPrefix, column); + for (ByteBuffer value : inValues) + { + if (isSatisfiedByValue(value, current.getColumn(name), column.type, Operator.EQ, now)) + return true; + } + return false; + } + } + + /** A condition on an element of a collection column. IN operators are not supported here, see ElementAccessInBound. */ + static class ElementAccessBound extends Bound + { + public final ByteBuffer collectionElement; + public final ByteBuffer value; + + private ElementAccessBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert column.type instanceof CollectionType && condition.collectionElement != null; + assert condition.operator != Operator.IN; + this.collectionElement = condition.collectionElement.bindAndGet(options); + this.value = condition.value.bindAndGet(options); + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException + { + if (collectionElement == null) + throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access"); + + if (column.type instanceof MapType) + { + MapType mapType = (MapType) column.type; + if (column.type.isMultiCell()) + { + Cell cell = current.getColumn(current.metadata().comparator.create(rowPrefix, column, collectionElement)); + return isSatisfiedByValue(value, cell, mapType.getValuesType(), operator, now); + } + else + { + Cell cell = current.getColumn(current.metadata().comparator.create(rowPrefix, column)); + ByteBuffer mapElementValue = cell.isLive(now) ? mapType.getSerializer().getSerializedValue(cell.value(), collectionElement, mapType.getKeysType()) + : null; + return compareWithOperator(operator, mapType.getValuesType(), value, mapElementValue); + } + } + + // sets don't have element access, so it's a list + ListType listType = (ListType) column.type; + if (column.type.isMultiCell()) + { + ByteBuffer columnValue = getListItem( + collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now), + getListIndex(collectionElement)); + return compareWithOperator(operator, listType.getElementsType(), value, columnValue); + } + else + { + Cell cell = current.getColumn(current.metadata().comparator.create(rowPrefix, column)); + ByteBuffer listElementValue = cell.isLive(now) ? listType.getSerializer().getElement(cell.value(), getListIndex(collectionElement)) + : null; + return compareWithOperator(operator, listType.getElementsType(), value, listElementValue); + } + } + + static int getListIndex(ByteBuffer collectionElement) throws InvalidRequestException + { + int idx = ByteBufferUtil.toInt(collectionElement); + if (idx < 0) + throw new InvalidRequestException(String.format("Invalid negative list index %d", idx)); + return idx; + } + + static ByteBuffer getListItem(Iterator iter, int index) + { + int adv = Iterators.advance(iter, index); + if (adv == index && iter.hasNext()) + return iter.next().value(); + else + return null; + } + + public ByteBuffer getCollectionElementValue() + { + return collectionElement; + } + } + + static class ElementAccessInBound extends Bound + { + public final ByteBuffer collectionElement; + public final List inValues; + + private ElementAccessInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert column.type instanceof CollectionType && condition.collectionElement != null; + this.collectionElement = condition.collectionElement.bindAndGet(options); + + if (condition.inValues == null) + this.inValues = ((Lists.Marker) condition.value).bind(options).getElements(); + else + { + this.inValues = new ArrayList<>(condition.inValues.size()); + for (Term value : condition.inValues) + this.inValues.add(value.bindAndGet(options)); + } + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException + { + if (collectionElement == null) + throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access"); + + CellNameType nameType = current.metadata().comparator; + if (column.type instanceof MapType) + { + MapType mapType = (MapType) column.type; + AbstractType valueType = mapType.getValuesType(); + if (column.type.isMultiCell()) + { + CellName name = nameType.create(rowPrefix, column, collectionElement); + Cell item = current.getColumn(name); + for (ByteBuffer value : inValues) + { + if (isSatisfiedByValue(value, item, valueType, Operator.EQ, now)) + return true; + } + return false; + } + else + { + Cell cell = current.getColumn(nameType.create(rowPrefix, column)); + ByteBuffer mapElementValue = null; + if (cell != null && cell.isLive(now)) + mapElementValue = mapType.getSerializer().getSerializedValue(cell.value(), collectionElement, mapType.getKeysType()); + for (ByteBuffer value : inValues) + { + if (value == null) + { + if (mapElementValue == null) + return true; + continue; + } + if (valueType.compare(value, mapElementValue) == 0) + return true; + } + return false; + } + } + + ListType listType = (ListType) column.type; + AbstractType elementsType = listType.getElementsType(); + if (column.type.isMultiCell()) + { + ByteBuffer columnValue = ElementAccessBound.getListItem( + collectionColumns(nameType.create(rowPrefix, column), current, now), + ElementAccessBound.getListIndex(collectionElement)); + + for (ByteBuffer value : inValues) + { + if (compareWithOperator(Operator.EQ, elementsType, value, columnValue)) + return true; + } + } + else + { + Cell cell = current.getColumn(nameType.create(rowPrefix, column)); + ByteBuffer listElementValue = null; + if (cell != null && cell.isLive(now)) + listElementValue = listType.getSerializer().getElement(cell.value(), ElementAccessBound.getListIndex(collectionElement)); + + for (ByteBuffer value : inValues) + { + if (value == null) + { + if (listElementValue == null) + return true; + continue; + } + if (elementsType.compare(value, listElementValue) == 0) + return true; + } + } + return false; + } + } + + /** A condition on an entire collection column. IN operators are not supported here, see CollectionInBound. */ + static class CollectionBound extends Bound + { + private final Term.Terminal value; + + private CollectionBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert column.type.isCollection() && condition.collectionElement == null; + assert condition.operator != Operator.IN; + this.value = condition.value.bind(options); + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException + { + CollectionType type = (CollectionType)column.type; + + if (type.isMultiCell()) + { + Iterator iter = collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now); + if (value == null) + { + if (operator == Operator.EQ) + return !iter.hasNext(); + else if (operator == Operator.NEQ) + return iter.hasNext(); + else + throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator)); + } + + return valueAppliesTo(type, iter, value, operator); + } + + // frozen collections + Cell cell = current.getColumn(current.metadata().comparator.create(rowPrefix, column)); + if (value == null) + { + if (operator == Operator.EQ) + return cell == null || !cell.isLive(now); + else if (operator == Operator.NEQ) + return cell != null && cell.isLive(now); + else + throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator)); + } + + // make sure we use v3 serialization format for comparison + ByteBuffer conditionValue; + if (type.kind == CollectionType.Kind.LIST) + conditionValue = ((Lists.Value) value).getWithProtocolVersion(Server.VERSION_3); + else if (type.kind == CollectionType.Kind.SET) + conditionValue = ((Sets.Value) value).getWithProtocolVersion(Server.VERSION_3); + else + conditionValue = ((Maps.Value) value).getWithProtocolVersion(Server.VERSION_3); + + return compareWithOperator(operator, type, conditionValue, cell.value()); + } + + static boolean valueAppliesTo(CollectionType type, Iterator iter, Term.Terminal value, Operator operator) + { + if (value == null) + return !iter.hasNext(); + + switch (type.kind) + { + case LIST: return listAppliesTo((ListType)type, iter, ((Lists.Value)value).elements, operator); + case SET: return setAppliesTo((SetType)type, iter, ((Sets.Value)value).elements, operator); + case MAP: return mapAppliesTo((MapType)type, iter, ((Maps.Value)value).map, operator); + } + throw new AssertionError(); + } + + private static boolean setOrListAppliesTo(AbstractType type, Iterator iter, Iterator conditionIter, Operator operator, boolean isSet) + { + while(iter.hasNext()) + { + if (!conditionIter.hasNext()) + return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ); + + // for lists we use the cell value; for sets we use the cell name + ByteBuffer cellValue = isSet? iter.next().name().collectionElement() : iter.next().value(); + int comparison = type.compare(cellValue, conditionIter.next()); + if (comparison != 0) + return evaluateComparisonWithOperator(comparison, operator); + } + + if (conditionIter.hasNext()) + return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ); + + // they're equal + return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE; + } + + private static boolean evaluateComparisonWithOperator(int comparison, Operator operator) + { + // called when comparison != 0 + switch (operator) + { + case EQ: + return false; + case LT: + case LTE: + return comparison < 0; + case GT: + case GTE: + return comparison > 0; + case NEQ: + return true; + default: + throw new AssertionError(); + } + } + + static boolean listAppliesTo(ListType type, Iterator iter, List elements, Operator operator) + { + return setOrListAppliesTo(type.getElementsType(), iter, elements.iterator(), operator, false); + } + + static boolean setAppliesTo(SetType type, Iterator iter, Set elements, Operator operator) + { + ArrayList sortedElements = new ArrayList<>(elements.size()); + sortedElements.addAll(elements); + Collections.sort(sortedElements, type.getElementsType()); + return setOrListAppliesTo(type.getElementsType(), iter, sortedElements.iterator(), operator, true); + } + + static boolean mapAppliesTo(MapType type, Iterator iter, Map elements, Operator operator) + { + Iterator> conditionIter = elements.entrySet().iterator(); + while(iter.hasNext()) + { + if (!conditionIter.hasNext()) + return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ); + + Map.Entry conditionEntry = conditionIter.next(); + Cell c = iter.next(); + + // compare the keys + int comparison = type.getKeysType().compare(c.name().collectionElement(), conditionEntry.getKey()); + if (comparison != 0) + return evaluateComparisonWithOperator(comparison, operator); + + // compare the values + comparison = type.getValuesType().compare(c.value(), conditionEntry.getValue()); + if (comparison != 0) + return evaluateComparisonWithOperator(comparison, operator); + } + + if (conditionIter.hasNext()) + return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ); + + // they're equal + return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE; + } + } + + public static class CollectionInBound extends Bound + { + private final List inValues; + + private CollectionInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException + { + super(condition.column, condition.operator); + assert column.type instanceof CollectionType && condition.collectionElement == null; + assert condition.operator == Operator.IN; + inValues = new ArrayList<>(); + if (condition.inValues == null) + { + // We have a list of serialized collections that need to be deserialized for later comparisons + CollectionType collectionType = (CollectionType) column.type; + Lists.Marker inValuesMarker = (Lists.Marker) condition.value; + if (column.type instanceof ListType) + { + ListType deserializer = ListType.getInstance(collectionType.valueComparator(), false); + for (ByteBuffer buffer : inValuesMarker.bind(options).elements) + { + if (buffer == null) + this.inValues.add(null); + else + this.inValues.add(Lists.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion())); + } + } + else if (column.type instanceof MapType) + { + MapType deserializer = MapType.getInstance(collectionType.nameComparator(), collectionType.valueComparator(), false); + for (ByteBuffer buffer : inValuesMarker.bind(options).elements) + { + if (buffer == null) + this.inValues.add(null); + else + this.inValues.add(Maps.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion())); + } + } + else if (column.type instanceof SetType) + { + SetType deserializer = SetType.getInstance(collectionType.valueComparator(), false); + for (ByteBuffer buffer : inValuesMarker.bind(options).elements) + { + if (buffer == null) + this.inValues.add(null); + else + this.inValues.add(Sets.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion())); + } + } + } + else + { + for (Term value : condition.inValues) + this.inValues.add(value.bind(options)); + } + } + + public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException + { + CollectionType type = (CollectionType)column.type; + CellName name = current.metadata().comparator.create(rowPrefix, column); + if (type.isMultiCell()) + { + // copy iterator contents so that we can properly reuse them for each comparison with an IN value + List cells = newArrayList(collectionColumns(name, current, now)); + for (Term.Terminal value : inValues) + { + if (CollectionBound.valueAppliesTo(type, cells.iterator(), value, Operator.EQ)) + return true; + } + return false; + } + else + { + Cell cell = current.getColumn(name); + for (Term.Terminal value : inValues) + { + if (value == null) + { + if (cell == null || !cell.isLive(now)) + return true; + } + else if (type.compare(((Term.CollectionTerminal)value).getWithProtocolVersion(Server.VERSION_3), cell.value()) == 0) + { + return true; + } + } + return false; + } + } + } + + public static class Raw + { + private final Term.Raw value; + private final List inValues; + private final AbstractMarker.INRaw inMarker; + + // Can be null, only used with the syntax "IF m[e] = ..." (in which case it's 'e') + private final Term.Raw collectionElement; + + private final Operator operator; + + private Raw(Term.Raw value, List inValues, AbstractMarker.INRaw inMarker, Term.Raw collectionElement, Operator op) + { + this.value = value; + this.inValues = inValues; + this.inMarker = inMarker; + this.collectionElement = collectionElement; + this.operator = op; + } + + /** A condition on a column. For example: "IF col = 'foo'" */ + public static Raw simpleCondition(Term.Raw value, Operator op) + { + return new Raw(value, null, null, null, op); + } + + /** An IN condition on a column. For example: "IF col IN ('foo', 'bar', ...)" */ + public static Raw simpleInCondition(List inValues) + { + return new Raw(null, inValues, null, null, Operator.IN); + } + + /** An IN condition on a column with a single marker. For example: "IF col IN ?" */ + public static Raw simpleInCondition(AbstractMarker.INRaw inMarker) + { + return new Raw(null, null, inMarker, null, Operator.IN); + } + + /** A condition on a collection element. For example: "IF col['key'] = 'foo'" */ + public static Raw collectionCondition(Term.Raw value, Term.Raw collectionElement, Operator op) + { + return new Raw(value, null, null, collectionElement, op); + } + + /** An IN condition on a collection element. For example: "IF col['key'] IN ('foo', 'bar', ...)" */ + public static Raw collectionInCondition(Term.Raw collectionElement, List inValues) + { + return new Raw(null, inValues, null, collectionElement, Operator.IN); + } + + /** An IN condition on a collection element with a single marker. For example: "IF col['key'] IN ?" */ + public static Raw collectionInCondition(Term.Raw collectionElement, AbstractMarker.INRaw inMarker) + { + return new Raw(null, null, inMarker, collectionElement, Operator.IN); + } + + public ColumnCondition prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + if (receiver.type instanceof CounterColumnType) + throw new InvalidRequestException("Conditions on counters are not supported"); + + if (collectionElement == null) + { + if (operator == Operator.IN) + { + if (inValues == null) + return ColumnCondition.inCondition(receiver, inMarker.prepare(keyspace, receiver)); + List terms = new ArrayList<>(inValues.size()); + for (Term.Raw value : inValues) + terms.add(value.prepare(keyspace, receiver)); + return ColumnCondition.inCondition(receiver, terms); + } + else + { + return ColumnCondition.condition(receiver, value.prepare(keyspace, receiver), operator); + } + } + + if (!(receiver.type.isCollection())) + throw new InvalidRequestException(String.format("Invalid element access syntax for non-collection column %s", receiver.name)); + + ColumnSpecification elementSpec, valueSpec; + switch ((((CollectionType)receiver.type).kind)) + { + case LIST: + elementSpec = Lists.indexSpecOf(receiver); + valueSpec = Lists.valueSpecOf(receiver); + break; + case MAP: + elementSpec = Maps.keySpecOf(receiver); + valueSpec = Maps.valueSpecOf(receiver); + break; + case SET: + throw new InvalidRequestException(String.format("Invalid element access syntax for set column %s", receiver.name)); + default: + throw new AssertionError(); + } + if (operator == Operator.IN) + { + if (inValues == null) + return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), inMarker.prepare(keyspace, valueSpec)); + List terms = new ArrayList<>(inValues.size()); + for (Term.Raw value : inValues) + terms.add(value.prepare(keyspace, valueSpec)); + return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), terms); + } + else + { + return ColumnCondition.condition(receiver, collectionElement.prepare(keyspace, elementSpec), value.prepare(keyspace, valueSpec), operator); + } + } + } +} diff --git a/cql3/ColumnIdentifier.java b/cql3/ColumnIdentifier.java new file mode 100644 index 0000000000..467b214ff6 --- /dev/null +++ b/cql3/ColumnIdentifier.java @@ -0,0 +1,182 @@ +/* + * 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.List; +import java.util.Locale; +import java.nio.ByteBuffer; + +import org.apache.cassandra.cache.IMeasurableMemory; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.selection.Selectable; +import org.apache.cassandra.cql3.selection.Selector; +import org.apache.cassandra.cql3.selection.SimpleSelector; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.memory.AbstractAllocator; + +/** + * Represents an identifer for a CQL column definition. + * TODO : should support light-weight mode without text representation for when not interned + */ +public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements IMeasurableMemory +{ + public final ByteBuffer bytes; + private final String text; + + private static final long EMPTY_SIZE = ObjectSizes.measure(new ColumnIdentifier("", true)); + + public ColumnIdentifier(String rawText, boolean keepCase) + { + this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US); + this.bytes = ByteBufferUtil.bytes(this.text); + } + + public ColumnIdentifier(ByteBuffer bytes, AbstractType type) + { + this.bytes = bytes; + this.text = type.getString(bytes); + } + + public ColumnIdentifier(ByteBuffer bytes, String text) + { + this.bytes = bytes; + this.text = text; + } + + @Override + public final int hashCode() + { + return bytes.hashCode(); + } + + @Override + public final boolean equals(Object o) + { + // Note: it's worth checking for reference equality since we intern those + // in SparseCellNameType + if (this == o) + return true; + + if(!(o instanceof ColumnIdentifier)) + return false; + ColumnIdentifier that = (ColumnIdentifier)o; + return bytes.equals(that.bytes); + } + + @Override + public String toString() + { + return text; + } + + public long unsharedHeapSize() + { + return EMPTY_SIZE + + ObjectSizes.sizeOnHeapOf(bytes) + + ObjectSizes.sizeOf(text); + } + + public long unsharedHeapSizeExcludingData() + { + return EMPTY_SIZE + + ObjectSizes.sizeOnHeapExcludingData(bytes) + + ObjectSizes.sizeOf(text); + } + + public ColumnIdentifier clone(AbstractAllocator allocator) + { + return new ColumnIdentifier(allocator.clone(bytes), text); + } + + public Selector.Factory newSelectorFactory(CFMetaData cfm, List defs) throws InvalidRequestException + { + ColumnDefinition def = cfm.getColumnDefinition(this); + if (def == null) + throw new InvalidRequestException(String.format("Undefined name %s in selection clause", this)); + + return SimpleSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), def.type); + } + + /** + * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until + * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted to a real ColumnIdentifier + * once the comparator is known with prepare(). This should only be used with identifiers that are actual + * column names. See CASSANDRA-8178 for more background. + */ + public static class Raw implements Selectable.Raw + { + private final String rawText; + private final String text; + + public Raw(String rawText, boolean keepCase) + { + this.rawText = rawText; + this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US); + } + + public ColumnIdentifier prepare(CFMetaData cfm) + { + AbstractType comparator = cfm.comparator.asAbstractType(); + if (cfm.getIsDense() || comparator instanceof CompositeType || comparator instanceof UTF8Type) + return new ColumnIdentifier(text, true); + + // We have a Thrift-created table with a non-text comparator. We need to parse column names with the comparator + // to get the correct ByteBuffer representation. However, this doesn't apply to key aliases, so we need to + // make a special check for those and treat them normally. See CASSANDRA-8178. + ByteBuffer bufferName = ByteBufferUtil.bytes(text); + for (ColumnDefinition def : cfm.partitionKeyColumns()) + { + if (def.name.bytes.equals(bufferName)) + return new ColumnIdentifier(text, true); + } + return new ColumnIdentifier(comparator.fromString(rawText), text); + } + + public boolean processesSelection() + { + return false; + } + + @Override + public final int hashCode() + { + return text.hashCode(); + } + + @Override + public final boolean equals(Object o) + { + if(!(o instanceof ColumnIdentifier.Raw)) + return false; + ColumnIdentifier.Raw that = (ColumnIdentifier.Raw)o; + return text.equals(that.text); + } + + @Override + public String toString() + { + return text; + } + } +} diff --git a/cql3/ColumnSpecification.java b/cql3/ColumnSpecification.java new file mode 100644 index 0000000000..cc54375574 --- /dev/null +++ b/cql3/ColumnSpecification.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 org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ReversedType; + +public class ColumnSpecification +{ + public final String ksName; + public final String cfName; + public final ColumnIdentifier name; + public final AbstractType type; + + public ColumnSpecification(String ksName, String cfName, ColumnIdentifier name, AbstractType type) + { + this.ksName = ksName; + this.cfName = cfName; + this.name = name; + this.type = type; + } + + /** + * Returns a new ColumnSpecification for the same column but with the specified alias. + * + * @param alias the column alias + * @return a new ColumnSpecification for the same column but with the specified alias. + */ + public ColumnSpecification withAlias(ColumnIdentifier alias) + { + return new ColumnSpecification(ksName, cfName, alias, type); + } + + public boolean isReversedType() + { + return type instanceof ReversedType; + } +} diff --git a/cql3/Constants.java b/cql3/Constants.java new file mode 100644 index 0000000000..1a257d110e --- /dev/null +++ b/cql3/Constants.java @@ -0,0 +1,378 @@ +/* + * 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.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.CounterColumnType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * Static helper methods and classes for constants. + */ +public abstract class Constants +{ + private static final Logger logger = LoggerFactory.getLogger(Constants.class); + + public enum Type + { + STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX; + } + + public static final Term.Raw NULL_LITERAL = new Term.Raw() + { + private final Term.Terminal NULL_VALUE = new Value(null) + { + @Override + public Terminal bind(QueryOptions options) + { + // We return null because that makes life easier for collections + return null; + } + + @Override + public String toString() + { + return "null"; + } + }; + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!testAssignment(keyspace, receiver).isAssignable()) + throw new InvalidRequestException("Invalid null value for counter increment/decrement"); + + return NULL_VALUE; + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + return receiver.type instanceof CounterColumnType + ? AssignmentTestable.TestResult.NOT_ASSIGNABLE + : AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + + @Override + public String toString() + { + return "null"; + } + }; + + public static class Literal implements Term.Raw + { + private final Type type; + private final String text; + + private Literal(Type type, String text) + { + assert type != null && text != null; + this.type = type; + this.text = text; + } + + public static Literal string(String text) + { + return new Literal(Type.STRING, text); + } + + public static Literal integer(String text) + { + return new Literal(Type.INTEGER, text); + } + + public static Literal floatingPoint(String text) + { + return new Literal(Type.FLOAT, text); + } + + public static Literal uuid(String text) + { + return new Literal(Type.UUID, text); + } + + public static Literal bool(String text) + { + return new Literal(Type.BOOLEAN, text); + } + + public static Literal hex(String text) + { + return new Literal(Type.HEX, text); + } + + public Value prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!testAssignment(keyspace, receiver).isAssignable()) + throw new InvalidRequestException(String.format("Invalid %s constant (%s) for \"%s\" of type %s", type, text, receiver.name, receiver.type.asCQL3Type())); + + return new Value(parsedValue(receiver.type)); + } + + private ByteBuffer parsedValue(AbstractType validator) throws InvalidRequestException + { + if (validator instanceof ReversedType) + validator = ((ReversedType) validator).baseType; + try + { + // BytesType doesn't want it's input prefixed by '0x'. + if (type == Type.HEX && validator instanceof BytesType) + return validator.fromString(text.substring(2)); + if (validator instanceof CounterColumnType) + return LongType.instance.fromString(text); + return validator.fromString(text); + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public String getRawText() + { + return text; + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + CQL3Type receiverType = receiver.type.asCQL3Type(); + if (receiverType.isCollection()) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + + if (!(receiverType instanceof CQL3Type.Native)) + // Skip type validation for custom types. May or may not be a good idea + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + + CQL3Type.Native nt = (CQL3Type.Native)receiverType; + switch (type) + { + case STRING: + switch (nt) + { + case ASCII: + case TEXT: + case INET: + case VARCHAR: + case TIMESTAMP: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + case INTEGER: + switch (nt) + { + case BIGINT: + case COUNTER: + case DECIMAL: + case DOUBLE: + case FLOAT: + case INT: + case TIMESTAMP: + case VARINT: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + case UUID: + switch (nt) + { + case UUID: + case TIMEUUID: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + case FLOAT: + switch (nt) + { + case DECIMAL: + case DOUBLE: + case FLOAT: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + case BOOLEAN: + switch (nt) + { + case BOOLEAN: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + case HEX: + switch (nt) + { + case BLOB: + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + break; + } + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + + @Override + public String toString() + { + return type == Type.STRING ? String.format("'%s'", text) : text; + } + } + + /** + * A constant value, i.e. a ByteBuffer. + */ + public static class Value extends Term.Terminal + { + public final ByteBuffer bytes; + + public Value(ByteBuffer bytes) + { + this.bytes = bytes; + } + + public ByteBuffer get(QueryOptions options) + { + return bytes; + } + + @Override + public ByteBuffer bindAndGet(QueryOptions options) + { + return bytes; + } + + @Override + public String toString() + { + return ByteBufferUtil.bytesToHex(bytes); + } + } + + public static class Marker extends AbstractMarker + { + protected Marker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + assert !receiver.type.isCollection(); + } + + @Override + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + try + { + ByteBuffer value = options.getValues().get(bindIndex); + if (value != null) + receiver.type.validate(value); + return value; + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer bytes = bindAndGet(options); + return bytes == null ? null : new Constants.Value(bytes); + } + } + + public static class Setter extends Operation + { + public Setter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + CellName cname = cf.getComparator().create(prefix, column); + ByteBuffer value = t.bindAndGet(params.options); + cf.addColumn(value == null ? params.makeTombstone(cname) : params.makeColumn(cname, value)); + } + } + + public static class Adder extends Operation + { + public Adder(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + ByteBuffer bytes = t.bindAndGet(params.options); + if (bytes == null) + throw new InvalidRequestException("Invalid null value for counter increment"); + long increment = ByteBufferUtil.toLong(bytes); + CellName cname = cf.getComparator().create(prefix, column); + cf.addColumn(params.makeCounter(cname, increment)); + } + } + + public static class Substracter extends Operation + { + public Substracter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + ByteBuffer bytes = t.bindAndGet(params.options); + if (bytes == null) + throw new InvalidRequestException("Invalid null value for counter increment"); + + long increment = ByteBufferUtil.toLong(bytes); + if (increment == Long.MIN_VALUE) + throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)"); + + CellName cname = cf.getComparator().create(prefix, column); + cf.addColumn(params.makeCounter(cname, -increment)); + } + } + + // This happens to also handle collection because it doesn't felt worth + // duplicating this further + public static class Deleter extends Operation + { + public Deleter(ColumnDefinition column) + { + super(column, null); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + CellName cname = cf.getComparator().create(prefix, column); + if (column.type.isMultiCell()) + cf.addAtom(params.makeRangeTombstone(cname.slice())); + else + cf.addColumn(params.makeTombstone(cname)); + } + }; +} diff --git a/cql3/Cql.g b/cql3/Cql.g new file mode 100644 index 0000000000..2a6e6d02c3 --- /dev/null +++ b/cql3/Cql.g @@ -0,0 +1,1581 @@ +/* + * 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. + */ + +grammar Cql; + +options { + language = Java; +} + +@header { + package org.apache.cassandra.cql3; + + import java.util.ArrayList; + import java.util.Arrays; + import java.util.Collections; + import java.util.EnumSet; + import java.util.HashSet; + import java.util.HashMap; + import java.util.LinkedHashMap; + import java.util.List; + import java.util.Map; + import java.util.Set; + + import org.apache.cassandra.auth.Permission; + import org.apache.cassandra.auth.DataResource; + import org.apache.cassandra.auth.IResource; + import org.apache.cassandra.cql3.*; + import org.apache.cassandra.cql3.statements.*; + import org.apache.cassandra.cql3.selection.*; + import org.apache.cassandra.cql3.functions.*; + import org.apache.cassandra.db.marshal.CollectionType; + import org.apache.cassandra.exceptions.ConfigurationException; + import org.apache.cassandra.exceptions.InvalidRequestException; + import org.apache.cassandra.exceptions.SyntaxException; + import org.apache.cassandra.utils.Pair; +} + +@members { + private final List listeners = new ArrayList(); + private final List bindVariables = new ArrayList(); + + public static final Set reservedTypeNames = new HashSet() + {{ + add("byte"); + add("smallint"); + add("complex"); + add("enum"); + add("date"); + add("interval"); + add("macaddr"); + add("bitstring"); + }}; + + public AbstractMarker.Raw newBindVariables(ColumnIdentifier name) + { + AbstractMarker.Raw marker = new AbstractMarker.Raw(bindVariables.size()); + bindVariables.add(name); + return marker; + } + + public AbstractMarker.INRaw newINBindVariables(ColumnIdentifier name) + { + AbstractMarker.INRaw marker = new AbstractMarker.INRaw(bindVariables.size()); + bindVariables.add(name); + return marker; + } + + public Tuples.Raw newTupleBindVariables(ColumnIdentifier name) + { + Tuples.Raw marker = new Tuples.Raw(bindVariables.size()); + bindVariables.add(name); + return marker; + } + + public Tuples.INRaw newTupleINBindVariables(ColumnIdentifier name) + { + Tuples.INRaw marker = new Tuples.INRaw(bindVariables.size()); + bindVariables.add(name); + return marker; + } + + public void addErrorListener(ErrorListener listener) + { + this.listeners.add(listener); + } + + public void removeErrorListener(ErrorListener listener) + { + this.listeners.remove(listener); + } + + public void displayRecognitionError(String[] tokenNames, RecognitionException e) + { + for (int i = 0, m = listeners.size(); i < m; i++) + listeners.get(i).syntaxError(this, tokenNames, e); + } + + private void addRecognitionError(String msg) + { + for (int i = 0, m = listeners.size(); i < m; i++) + listeners.get(i).syntaxError(this, msg); + } + + public Map convertPropertyMap(Maps.Literal map) + { + if (map == null || map.entries == null || map.entries.isEmpty()) + return Collections.emptyMap(); + + Map res = new HashMap(map.entries.size()); + + for (Pair entry : map.entries) + { + // Because the parser tries to be smart and recover on error (to + // allow displaying more than one error I suppose), we have null + // entries in there. Just skip those, a proper error will be thrown in the end. + if (entry.left == null || entry.right == null) + break; + + if (!(entry.left instanceof Constants.Literal)) + { + String msg = "Invalid property name: " + entry.left; + if (entry.left instanceof AbstractMarker.Raw) + msg += " (bind variables are not supported in DDL queries)"; + addRecognitionError(msg); + break; + } + if (!(entry.right instanceof Constants.Literal)) + { + String msg = "Invalid property value: " + entry.right + " for property: " + entry.left; + if (entry.right instanceof AbstractMarker.Raw) + msg += " (bind variables are not supported in DDL queries)"; + addRecognitionError(msg); + break; + } + + res.put(((Constants.Literal)entry.left).getRawText(), ((Constants.Literal)entry.right).getRawText()); + } + + return res; + } + + public void addRawUpdate(List> operations, ColumnIdentifier.Raw key, Operation.RawUpdate update) + { + for (Pair p : operations) + { + if (p.left.equals(key) && !p.right.isCompatibleWith(update)) + addRecognitionError("Multiple incompatible setting of column " + key); + } + operations.add(Pair.create(key, update)); + } +} + +@lexer::header { + package org.apache.cassandra.cql3; + + import org.apache.cassandra.exceptions.SyntaxException; +} + +@lexer::members { + List tokens = new ArrayList(); + + public void emit(Token token) + { + state.token = token; + tokens.add(token); + } + + public Token nextToken() + { + super.nextToken(); + if (tokens.size() == 0) + return new CommonToken(Token.EOF); + return tokens.remove(0); + } + + private final List listeners = new ArrayList(); + + public void addErrorListener(ErrorListener listener) + { + this.listeners.add(listener); + } + + public void removeErrorListener(ErrorListener listener) + { + this.listeners.remove(listener); + } + + public void displayRecognitionError(String[] tokenNames, RecognitionException e) + { + for (int i = 0, m = listeners.size(); i < m; i++) + listeners.get(i).syntaxError(this, tokenNames, e); + } +} + +/** STATEMENTS **/ + +query returns [ParsedStatement stmnt] + : st=cqlStatement (';')* EOF { $stmnt = st; } + ; + +cqlStatement returns [ParsedStatement stmt] + @after{ if (stmt != null) stmt.setBoundVariables(bindVariables); } + : st1= selectStatement { $stmt = st1; } + | st2= insertStatement { $stmt = st2; } + | st3= updateStatement { $stmt = st3; } + | st4= batchStatement { $stmt = st4; } + | st5= deleteStatement { $stmt = st5; } + | st6= useStatement { $stmt = st6; } + | st7= truncateStatement { $stmt = st7; } + | st8= createKeyspaceStatement { $stmt = st8; } + | st9= createTableStatement { $stmt = st9; } + | st10=createIndexStatement { $stmt = st10; } + | st11=dropKeyspaceStatement { $stmt = st11; } + | st12=dropTableStatement { $stmt = st12; } + | st13=dropIndexStatement { $stmt = st13; } + | st14=alterTableStatement { $stmt = st14; } + | st15=alterKeyspaceStatement { $stmt = st15; } + | st16=grantStatement { $stmt = st16; } + | st17=revokeStatement { $stmt = st17; } + | st18=listPermissionsStatement { $stmt = st18; } + | st19=createUserStatement { $stmt = st19; } + | st20=alterUserStatement { $stmt = st20; } + | st21=dropUserStatement { $stmt = st21; } + | st22=listUsersStatement { $stmt = st22; } + | st23=createTriggerStatement { $stmt = st23; } + | st24=dropTriggerStatement { $stmt = st24; } + | st25=createTypeStatement { $stmt = st25; } + | st26=alterTypeStatement { $stmt = st26; } + | st27=dropTypeStatement { $stmt = st27; } + | st28=createFunctionStatement { $stmt = st28; } + | st29=dropFunctionStatement { $stmt = st29; } + | st30=createAggregateStatement { $stmt = st30; } + | st31=dropAggregateStatement { $stmt = st31; } + ; + +/* + * USE ; + */ +useStatement returns [UseStatement stmt] + : K_USE ks=keyspaceName { $stmt = new UseStatement(ks); } + ; + +/** + * SELECT + * FROM + * WHERE KEY = "key1" AND COL > 1 AND COL < 100 + * LIMIT ; + */ +selectStatement returns [SelectStatement.RawStatement expr] + @init { + boolean isDistinct = false; + Term.Raw limit = null; + Map orderings = new LinkedHashMap(); + boolean allowFiltering = false; + } + : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause + | sclause=selectCountClause ) + K_FROM cf=columnFamilyName + ( K_WHERE wclause=whereClause )? + ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )? + ( K_LIMIT rows=intValue { limit = rows; } )? + ( K_ALLOW K_FILTERING { allowFiltering = true; } )? + { + SelectStatement.Parameters params = new SelectStatement.Parameters(orderings, + isDistinct, + allowFiltering); + $expr = new SelectStatement.RawStatement(cf, params, sclause, wclause, limit); + } + ; + +selectClause returns [List expr] + : t1=selector { $expr = new ArrayList(); $expr.add(t1); } (',' tN=selector { $expr.add(tN); })* + | '\*' { $expr = Collections.emptyList();} + ; + +selector returns [RawSelector s] + @init{ ColumnIdentifier alias = null; } + : us=unaliasedSelector (K_AS c=ident { alias = c; })? { $s = new RawSelector(us, alias); } + ; + +unaliasedSelector returns [Selectable.Raw s] + @init { Selectable.Raw tmp = null; } + : ( c=cident { tmp = c; } + | K_WRITETIME '(' c=cident ')' { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); } + | K_TTL '(' c=cident ')' { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); } + | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); } + ) ( '.' fi=cident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; } + ; + +selectionFunctionArgs returns [List a] + : '(' ')' { $a = Collections.emptyList(); } + | '(' s1=unaliasedSelector { List args = new ArrayList(); args.add(s1); } + ( ',' sn=unaliasedSelector { args.add(sn); } )* + ')' { $a = args; } + ; + +selectCountClause returns [List expr] + @init{ ColumnIdentifier alias = new ColumnIdentifier("count", false); } + : K_COUNT '(' countArgument ')' (K_AS c=ident { alias = c; })? { $expr = new ArrayList(); $expr.add( new RawSelector(new Selectable.WithFunction.Raw(FunctionName.nativeFunction("countRows"), Collections.emptyList()), alias));} + ; + +countArgument + : '\*' + | i=INTEGER { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")");} + ; + +whereClause returns [List clause] + @init{ $clause = new ArrayList(); } + : relation[$clause] (K_AND relation[$clause])* + ; + +orderByClause[Map orderings] + @init{ + boolean reversed = false; + } + : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); } + ; + +/** + * INSERT INTO (, , , ...) + * VALUES (, , , ...) + * USING TIMESTAMP ; + * + */ +insertStatement returns [UpdateStatement.ParsedInsert expr] + @init { + Attributes.Raw attrs = new Attributes.Raw(); + List columnNames = new ArrayList(); + List values = new ArrayList(); + boolean ifNotExists = false; + } + : K_INSERT K_INTO cf=columnFamilyName + '(' c1=cident { columnNames.add(c1); } ( ',' cn=cident { columnNames.add(cn); } )* ')' + K_VALUES + '(' v1=term { values.add(v1); } ( ',' vn=term { values.add(vn); } )* ')' + + ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + ( usingClause[attrs] )? + { + $expr = new UpdateStatement.ParsedInsert(cf, + attrs, + columnNames, + values, + ifNotExists); + } + ; + +usingClause[Attributes.Raw attrs] + : K_USING usingClauseObjective[attrs] ( K_AND usingClauseObjective[attrs] )* + ; + +usingClauseObjective[Attributes.Raw attrs] + : K_TIMESTAMP ts=intValue { attrs.timestamp = ts; } + | K_TTL t=intValue { attrs.timeToLive = t; } + ; + +/** + * UPDATE + * USING TIMESTAMP + * SET name1 = value1, name2 = value2 + * WHERE key = value; + */ +updateStatement returns [UpdateStatement.ParsedUpdate expr] + @init { + Attributes.Raw attrs = new Attributes.Raw(); + List> operations = new ArrayList>(); + } + : K_UPDATE cf=columnFamilyName + ( usingClause[attrs] )? + K_SET columnOperation[operations] (',' columnOperation[operations])* + K_WHERE wclause=whereClause + ( K_IF conditions=updateConditions )? + { + return new UpdateStatement.ParsedUpdate(cf, + attrs, + operations, + wclause, + conditions == null ? Collections.>emptyList() : conditions); + } + ; + +updateConditions returns [List> conditions] + @init { conditions = new ArrayList>(); } + : columnCondition[conditions] ( K_AND columnCondition[conditions] )* + ; + + +/** + * DELETE name1, name2 + * FROM + * USING TIMESTAMP + * WHERE KEY = keyname + [IF (EXISTS | name = value, ...)]; + */ +deleteStatement returns [DeleteStatement.Parsed expr] + @init { + Attributes.Raw attrs = new Attributes.Raw(); + List columnDeletions = Collections.emptyList(); + boolean ifExists = false; + } + : K_DELETE ( dels=deleteSelection { columnDeletions = dels; } )? + K_FROM cf=columnFamilyName + ( usingClauseDelete[attrs] )? + K_WHERE wclause=whereClause + ( K_IF ( K_EXISTS { ifExists = true; } | conditions=updateConditions ))? + { + return new DeleteStatement.Parsed(cf, + attrs, + columnDeletions, + wclause, + conditions == null ? Collections.>emptyList() : conditions, + ifExists); + } + ; + +deleteSelection returns [List operations] + : { $operations = new ArrayList(); } + t1=deleteOp { $operations.add(t1); } + (',' tN=deleteOp { $operations.add(tN); })* + ; + +deleteOp returns [Operation.RawDeletion op] + : c=cident { $op = new Operation.ColumnDeletion(c); } + | c=cident '[' t=term ']' { $op = new Operation.ElementDeletion(c, t); } + ; + +usingClauseDelete[Attributes.Raw attrs] + : K_USING K_TIMESTAMP ts=intValue { attrs.timestamp = ts; } + ; + +/** + * BEGIN BATCH + * UPDATE SET name1 = value1 WHERE KEY = keyname1; + * UPDATE SET name2 = value2 WHERE KEY = keyname2; + * UPDATE SET name3 = value3 WHERE KEY = keyname3; + * ... + * APPLY BATCH + * + * OR + * + * BEGIN BATCH + * INSERT INTO (KEY, ) VALUES ('', ''); + * INSERT INTO (KEY, ) VALUES ('', ''); + * ... + * APPLY BATCH + * + * OR + * + * BEGIN BATCH + * DELETE name1, name2 FROM WHERE key = + * DELETE name3, name4 FROM WHERE key = + * ... + * APPLY BATCH + */ +batchStatement returns [BatchStatement.Parsed expr] + @init { + BatchStatement.Type type = BatchStatement.Type.LOGGED; + List statements = new ArrayList(); + Attributes.Raw attrs = new Attributes.Raw(); + } + : K_BEGIN + ( K_UNLOGGED { type = BatchStatement.Type.UNLOGGED; } | K_COUNTER { type = BatchStatement.Type.COUNTER; } )? + K_BATCH ( usingClause[attrs] )? + ( s=batchStatementObjective ';'? { statements.add(s); } )* + K_APPLY K_BATCH + { + return new BatchStatement.Parsed(type, attrs, statements); + } + ; + +batchStatementObjective returns [ModificationStatement.Parsed statement] + : i=insertStatement { $statement = i; } + | u=updateStatement { $statement = u; } + | d=deleteStatement { $statement = d; } + ; + +createAggregateStatement returns [CreateAggregateStatement expr] + @init { + boolean orReplace = false; + boolean ifNotExists = false; + + List argsTypes = new ArrayList<>(); + } + : K_CREATE (K_OR K_REPLACE { orReplace = true; })? + K_AGGREGATE + (K_IF K_NOT K_EXISTS { ifNotExists = true; })? + fn=functionName + '(' + ( + v=comparatorType { argsTypes.add(v); } + ( ',' v=comparatorType { argsTypes.add(v); } )* + )? + ')' + K_SFUNC sfunc = allowedFunctionName + K_STYPE stype = comparatorType + ( + K_FINALFUNC ffunc = allowedFunctionName + )? + ( + K_INITCOND ival = term + )? + { $expr = new CreateAggregateStatement(fn, argsTypes, sfunc, stype, ffunc, ival, orReplace, ifNotExists); } + ; + +dropAggregateStatement returns [DropAggregateStatement expr] + @init { + boolean ifExists = false; + List argsTypes = new ArrayList<>(); + boolean argsPresent = false; + } + : K_DROP K_AGGREGATE + (K_IF K_EXISTS { ifExists = true; } )? + fn=functionName + ( + '(' + ( + v=comparatorType { argsTypes.add(v); } + ( ',' v=comparatorType { argsTypes.add(v); } )* + )? + ')' + { argsPresent = true; } + )? + { $expr = new DropAggregateStatement(fn, argsTypes, argsPresent, ifExists); } + ; + +createFunctionStatement returns [CreateFunctionStatement expr] + @init { + boolean orReplace = false; + boolean ifNotExists = false; + + boolean deterministic = true; + List argsNames = new ArrayList<>(); + List argsTypes = new ArrayList<>(); + } + : K_CREATE (K_OR K_REPLACE { orReplace = true; })? + ((K_NON { deterministic = false; })? K_DETERMINISTIC)? + K_FUNCTION + (K_IF K_NOT K_EXISTS { ifNotExists = true; })? + fn=functionName + '(' + ( + k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } + ( ',' k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )* + )? + ')' + K_RETURNS rt = comparatorType + K_LANGUAGE language = IDENT + K_AS body = STRING_LITERAL + { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text, deterministic, argsNames, argsTypes, rt, orReplace, ifNotExists); } + ; + +dropFunctionStatement returns [DropFunctionStatement expr] + @init { + boolean ifExists = false; + List argsTypes = new ArrayList<>(); + boolean argsPresent = false; + } + : K_DROP K_FUNCTION + (K_IF K_EXISTS { ifExists = true; } )? + fn=functionName + ( + '(' + ( + v=comparatorType { argsTypes.add(v); } + ( ',' v=comparatorType { argsTypes.add(v); } )* + )? + ')' + { argsPresent = true; } + )? + { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); } + ; + +/** + * CREATE KEYSPACE [IF NOT EXISTS] WITH attr1 = value1 AND attr2 = value2; + */ +createKeyspaceStatement returns [CreateKeyspaceStatement expr] + @init { + KSPropDefs attrs = new KSPropDefs(); + boolean ifNotExists = false; + } + : K_CREATE K_KEYSPACE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? ks=keyspaceName + K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs, ifNotExists); } + ; + +/** + * CREATE COLUMNFAMILY [IF NOT EXISTS] ( + * , + * , + * + * ) WITH = AND ...; + */ +createTableStatement returns [CreateTableStatement.RawStatement expr] + @init { boolean ifNotExists = false; } + : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + cf=columnFamilyName { $expr = new CreateTableStatement.RawStatement(cf, ifNotExists); } + cfamDefinition[expr] + ; + +cfamDefinition[CreateTableStatement.RawStatement expr] + : '(' cfamColumns[expr] ( ',' cfamColumns[expr]? )* ')' + ( K_WITH cfamProperty[expr] ( K_AND cfamProperty[expr] )*)? + ; + +cfamColumns[CreateTableStatement.RawStatement expr] + : k=ident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); } + (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })? + | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=ident { $expr.addColumnAlias(c); } )* ')' + ; + +pkDef[CreateTableStatement.RawStatement expr] + : k=ident { $expr.addKeyAliases(Collections.singletonList(k)); } + | '(' { List l = new ArrayList(); } k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); } + ; + +cfamProperty[CreateTableStatement.RawStatement expr] + : property[expr.properties] + | K_COMPACT K_STORAGE { $expr.setCompactStorage(); } + | K_CLUSTERING K_ORDER K_BY '(' cfamOrdering[expr] (',' cfamOrdering[expr])* ')' + ; + +cfamOrdering[CreateTableStatement.RawStatement expr] + @init{ boolean reversed=false; } + : k=ident (K_ASC | K_DESC { reversed=true;} ) { $expr.setOrdering(k, reversed); } + ; + + +/** + * CREATE TYPE foo ( + * , + * , + * .... + * ) + */ +createTypeStatement returns [CreateTypeStatement expr] + @init { boolean ifNotExists = false; } + : K_CREATE K_TYPE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + tn=userTypeName { $expr = new CreateTypeStatement(tn, ifNotExists); } + '(' typeColumns[expr] ( ',' typeColumns[expr]? )* ')' + ; + +typeColumns[CreateTypeStatement expr] + : k=ident v=comparatorType { $expr.addDefinition(k, v); } + ; + + +/** + * CREATE INDEX [IF NOT EXISTS] [indexName] ON (); + * CREATE CUSTOM INDEX [IF NOT EXISTS] [indexName] ON () USING ; + */ +createIndexStatement returns [CreateIndexStatement expr] + @init { + IndexPropDefs props = new IndexPropDefs(); + boolean ifNotExists = false; + } + : K_CREATE (K_CUSTOM { props.isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + (idxName=IDENT)? K_ON cf=columnFamilyName '(' id=indexIdent ')' + (K_USING cls=STRING_LITERAL { props.customClass = $cls.text; })? + (K_WITH properties[props])? + { $expr = new CreateIndexStatement(cf, $idxName.text, id, props, ifNotExists); } + ; + +indexIdent returns [IndexTarget.Raw id] + : c=cident { $id = IndexTarget.Raw.valuesOf(c); } + | K_KEYS '(' c=cident ')' { $id = IndexTarget.Raw.keysOf(c); } + | K_ENTRIES '(' c=cident ')' { $id = IndexTarget.Raw.keysAndValuesOf(c); } + | K_FULL '(' c=cident ')' { $id = IndexTarget.Raw.fullCollection(c); } + ; + + +/** + * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass'; + */ +createTriggerStatement returns [CreateTriggerStatement expr] + @init { + boolean ifNotExists = false; + } + : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident) + K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL + { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); } + ; + +/** + * DROP TRIGGER [IF EXISTS] triggerName ON columnFamily; + */ +dropTriggerStatement returns [DropTriggerStatement expr] + @init { boolean ifExists = false; } + : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName + { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); } + ; + +/** + * ALTER KEYSPACE WITH = ; + */ +alterKeyspaceStatement returns [AlterKeyspaceStatement expr] + @init { KSPropDefs attrs = new KSPropDefs(); } + : K_ALTER K_KEYSPACE ks=keyspaceName + K_WITH properties[attrs] { $expr = new AlterKeyspaceStatement(ks, attrs); } + ; + + +/** + * ALTER COLUMN FAMILY ALTER TYPE ; + * ALTER COLUMN FAMILY ADD ; + * ALTER COLUMN FAMILY DROP ; + * ALTER COLUMN FAMILY WITH = ; + * ALTER COLUMN FAMILY RENAME TO ; + */ +alterTableStatement returns [AlterTableStatement expr] + @init { + AlterTableStatement.Type type = null; + CFPropDefs props = new CFPropDefs(); + Map renames = new HashMap(); + boolean isStatic = false; + } + : K_ALTER K_COLUMNFAMILY cf=columnFamilyName + ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; } + | K_ADD id=cident v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; } + | K_DROP id=cident { type = AlterTableStatement.Type.DROP; } + | K_WITH properties[props] { type = AlterTableStatement.Type.OPTS; } + | K_RENAME { type = AlterTableStatement.Type.RENAME; } + id1=cident K_TO toId1=cident { renames.put(id1, toId1); } + ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )* + ) + { + $expr = new AlterTableStatement(cf, type, id, v, props, renames, isStatic); + } + ; + +/** + * ALTER TYPE ALTER TYPE ; + * ALTER TYPE ADD ; + * ALTER TYPE RENAME TO AND ...; + */ +alterTypeStatement returns [AlterTypeStatement expr] + : K_ALTER K_TYPE name=userTypeName + ( K_ALTER f=ident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); } + | K_ADD f=ident v=comparatorType { $expr = AlterTypeStatement.addition(name, f, v); } + | K_RENAME + { Map renames = new HashMap(); } + id1=ident K_TO toId1=ident { renames.put(id1, toId1); } + ( K_AND idn=ident K_TO toIdn=ident { renames.put(idn, toIdn); } )* + { $expr = AlterTypeStatement.renames(name, renames); } + ) + ; + + +/** + * DROP KEYSPACE [IF EXISTS] ; + */ +dropKeyspaceStatement returns [DropKeyspaceStatement ksp] + @init { boolean ifExists = false; } + : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks, ifExists); } + ; + +/** + * DROP COLUMNFAMILY [IF EXISTS] ; + */ +dropTableStatement returns [DropTableStatement stmt] + @init { boolean ifExists = false; } + : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName { $stmt = new DropTableStatement(cf, ifExists); } + ; + +/** + * DROP TYPE ; + */ +dropTypeStatement returns [DropTypeStatement stmt] + @init { boolean ifExists = false; } + : K_DROP K_TYPE (K_IF K_EXISTS { ifExists = true; } )? name=userTypeName { $stmt = new DropTypeStatement(name, ifExists); } + ; + +/** + * DROP INDEX [IF EXISTS] + */ +dropIndexStatement returns [DropIndexStatement expr] + @init { boolean ifExists = false; } + : K_DROP K_INDEX (K_IF K_EXISTS { ifExists = true; } )? index=indexName + { $expr = new DropIndexStatement(index, ifExists); } + ; + +/** + * TRUNCATE ; + */ +truncateStatement returns [TruncateStatement stmt] + : K_TRUNCATE cf=columnFamilyName { $stmt = new TruncateStatement(cf); } + ; + +/** + * GRANT ON TO + */ +grantStatement returns [GrantStatement stmt] + : K_GRANT + permissionOrAll + K_ON + resource + K_TO + username + { $stmt = new GrantStatement($permissionOrAll.perms, $resource.res, $username.text); } + ; + +/** + * REVOKE ON FROM + */ +revokeStatement returns [RevokeStatement stmt] + : K_REVOKE + permissionOrAll + K_ON + resource + K_FROM + username + { $stmt = new RevokeStatement($permissionOrAll.perms, $resource.res, $username.text); } + ; + +listPermissionsStatement returns [ListPermissionsStatement stmt] + @init { + IResource resource = null; + String username = null; + boolean recursive = true; + } + : K_LIST + permissionOrAll + ( K_ON resource { resource = $resource.res; } )? + ( K_OF username { username = $username.text; } )? + ( K_NORECURSIVE { recursive = false; } )? + { $stmt = new ListPermissionsStatement($permissionOrAll.perms, resource, username, recursive); } + ; + +permission returns [Permission perm] + : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE) + { $perm = Permission.valueOf($p.text.toUpperCase()); } + ; + +permissionOrAll returns [Set perms] + : K_ALL ( K_PERMISSIONS )? { $perms = Permission.ALL_DATA; } + | p=permission ( K_PERMISSION )? { $perms = EnumSet.of($p.perm); } + ; + +resource returns [IResource res] + : r=dataResource { $res = $r.res; } + ; + +dataResource returns [DataResource res] + : K_ALL K_KEYSPACES { $res = DataResource.root(); } + | K_KEYSPACE ks = keyspaceName { $res = DataResource.keyspace($ks.id); } + | ( K_COLUMNFAMILY )? cf = columnFamilyName + { $res = DataResource.columnFamily($cf.name.getKeyspace(), $cf.name.getColumnFamily()); } + ; + +/** + * CREATE USER [IF NOT EXISTS] [WITH PASSWORD ] [SUPERUSER|NOSUPERUSER] + */ +createUserStatement returns [CreateUserStatement stmt] + @init { + UserOptions opts = new UserOptions(); + boolean superuser = false; + boolean ifNotExists = false; + } + : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? username + ( K_WITH userOptions[opts] )? + ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )? + { $stmt = new CreateUserStatement($username.text, opts, superuser, ifNotExists); } + ; + +/** + * ALTER USER [WITH PASSWORD ] [SUPERUSER|NOSUPERUSER] + */ +alterUserStatement returns [AlterUserStatement stmt] + @init { + UserOptions opts = new UserOptions(); + Boolean superuser = null; + } + : K_ALTER K_USER username + ( K_WITH userOptions[opts] )? + ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )? + { $stmt = new AlterUserStatement($username.text, opts, superuser); } + ; + +/** + * DROP USER [IF EXISTS] + */ +dropUserStatement returns [DropUserStatement stmt] + @init { boolean ifExists = false; } + : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? username { $stmt = new DropUserStatement($username.text, ifExists); } + ; + +/** + * LIST USERS + */ +listUsersStatement returns [ListUsersStatement stmt] + : K_LIST K_USERS { $stmt = new ListUsersStatement(); } + ; + +userOptions[UserOptions opts] + : userOption[opts] + ; + +userOption[UserOptions opts] + : k=K_PASSWORD v=STRING_LITERAL { opts.put($k.text, $v.text); } + ; + +/** DEFINITIONS **/ + +// Column Identifiers. These need to be treated differently from other +// identifiers because the underlying comparator is not necessarily text. See +// CASSANDRA-8178 for details. +cident returns [ColumnIdentifier.Raw id] + : t=IDENT { $id = new ColumnIdentifier.Raw($t.text, false); } + | t=QUOTED_NAME { $id = new ColumnIdentifier.Raw($t.text, true); } + | k=unreserved_keyword { $id = new ColumnIdentifier.Raw(k, false); } + ; + +// Identifiers that do not refer to columns or where the comparator is known to be text +ident returns [ColumnIdentifier id] + : t=IDENT { $id = new ColumnIdentifier($t.text, false); } + | t=QUOTED_NAME { $id = new ColumnIdentifier($t.text, true); } + | k=unreserved_keyword { $id = new ColumnIdentifier(k, false); } + ; + +// Keyspace & Column family names +keyspaceName returns [String id] + @init { CFName name = new CFName(); } + : cfOrKsName[name, true] { $id = name.getKeyspace(); } + ; + +indexName returns [IndexName name] + @init { $name = new IndexName(); } + : (idxOrKsName[name, true] '.')? idxOrKsName[name, false] + ; + +idxOrKsName[IndexName name, boolean isKs] + : t=IDENT { if (isKs) $name.setKeyspace($t.text, false); else $name.setIndex($t.text, false); } + | t=QUOTED_NAME { if (isKs) $name.setKeyspace($t.text, true); else $name.setIndex($t.text, true); } + | k=unreserved_keyword { if (isKs) $name.setKeyspace(k, false); else $name.setIndex(k, false); } + ; + +columnFamilyName returns [CFName name] + @init { $name = new CFName(); } + : (cfOrKsName[name, true] '.')? cfOrKsName[name, false] + ; + +userTypeName returns [UTName name] + : (ks=ident '.')? ut=non_type_ident { return new UTName(ks, ut); } + ; + +cfOrKsName[CFName name, boolean isKs] + : t=IDENT { if (isKs) $name.setKeyspace($t.text, false); else $name.setColumnFamily($t.text, false); } + | t=QUOTED_NAME { if (isKs) $name.setKeyspace($t.text, true); else $name.setColumnFamily($t.text, true); } + | k=unreserved_keyword { if (isKs) $name.setKeyspace(k, false); else $name.setColumnFamily(k, false); } + | QMARK {addRecognitionError("Bind variables cannot be used for keyspace or table names");} + ; + +constant returns [Constants.Literal constant] + : t=STRING_LITERAL { $constant = Constants.Literal.string($t.text); } + | t=INTEGER { $constant = Constants.Literal.integer($t.text); } + | t=FLOAT { $constant = Constants.Literal.floatingPoint($t.text); } + | t=BOOLEAN { $constant = Constants.Literal.bool($t.text); } + | t=UUID { $constant = Constants.Literal.uuid($t.text); } + | t=HEXNUMBER { $constant = Constants.Literal.hex($t.text); } + | { String sign=""; } ('-' {sign = "-"; } )? t=(K_NAN | K_INFINITY) { $constant = Constants.Literal.floatingPoint(sign + $t.text); } + ; + +mapLiteral returns [Maps.Literal map] + : '{' { List> m = new ArrayList>(); } + ( k1=term ':' v1=term { m.add(Pair.create(k1, v1)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* )? + '}' { $map = new Maps.Literal(m); } + ; + +setOrMapLiteral[Term.Raw t] returns [Term.Raw value] + : ':' v=term { List> m = new ArrayList>(); m.add(Pair.create(t, v)); } + ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* + { $value = new Maps.Literal(m); } + | { List s = new ArrayList(); s.add(t); } + ( ',' tn=term { s.add(tn); } )* + { $value = new Sets.Literal(s); } + ; + +collectionLiteral returns [Term.Raw value] + : '[' { List l = new ArrayList(); } + ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )? + ']' { $value = new Lists.Literal(l); } + | '{' t=term v=setOrMapLiteral[t] { $value = v; } '}' + // Note that we have an ambiguity between maps and set for "{}". So we force it to a set literal, + // and deal with it later based on the type of the column (SetLiteral.java). + | '{' '}' { $value = new Sets.Literal(Collections.emptyList()); } + ; + +usertypeLiteral returns [UserTypes.Literal ut] + @init{ Map m = new HashMap(); } + @after{ $ut = new UserTypes.Literal(m); } + // We don't allow empty literals because that conflicts with sets/maps and is currently useless since we don't allow empty user types + : '{' k1=ident ':' v1=term { m.put(k1, v1); } ( ',' kn=ident ':' vn=term { m.put(kn, vn); } )* '}' + ; + +tupleLiteral returns [Tuples.Literal tt] + @init{ List l = new ArrayList(); } + @after{ $tt = new Tuples.Literal(l); } + : '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* ')' + ; + +value returns [Term.Raw value] + : c=constant { $value = c; } + | l=collectionLiteral { $value = l; } + | u=usertypeLiteral { $value = u; } + | t=tupleLiteral { $value = t; } + | K_NULL { $value = Constants.NULL_LITERAL; } + | ':' id=ident { $value = newBindVariables(id); } + | QMARK { $value = newBindVariables(null); } + ; + +intValue returns [Term.Raw value] + : + | t=INTEGER { $value = Constants.Literal.integer($t.text); } + | ':' id=ident { $value = newBindVariables(id); } + | QMARK { $value = newBindVariables(null); } + ; + +functionName returns [FunctionName s] + : (ks=keyspaceName '.')? f=allowedFunctionName { $s = new FunctionName(ks, f); } + ; + +allowedFunctionName returns [String s] + : f=IDENT { $s = $f.text.toLowerCase(); } + | f=QUOTED_NAME { $s = $f.text; } + | u=unreserved_function_keyword { $s = u; } + | K_TOKEN { $s = "token"; } + | K_COUNT { $s = "count"; } + ; + +functionArgs returns [List a] + : '(' ')' { $a = Collections.emptyList(); } + | '(' t1=term { List args = new ArrayList(); args.add(t1); } + ( ',' tn=term { args.add(tn); } )* + ')' { $a = args; } + ; + +term returns [Term.Raw term] + : v=value { $term = v; } + | f=functionName args=functionArgs { $term = new FunctionCall.Raw(f, args); } + | '(' c=comparatorType ')' t=term { $term = new TypeCast(c, t); } + ; + +columnOperation[List> operations] + : key=cident columnOperationDifferentiator[operations, key] + ; + +columnOperationDifferentiator[List> operations, ColumnIdentifier.Raw key] + : '=' normalColumnOperation[operations, key] + | '[' k=term ']' specializedColumnOperation[operations, key, k] + ; + +normalColumnOperation[List> operations, ColumnIdentifier.Raw key] + : t=term ('+' c=cident )? + { + if (c == null) + { + addRawUpdate(operations, key, new Operation.SetValue(t)); + } + else + { + if (!key.equals(c)) + addRecognitionError("Only expressions of the form X = + X are supported."); + addRawUpdate(operations, key, new Operation.Prepend(t)); + } + } + | c=cident sig=('+' | '-') t=term + { + if (!key.equals(c)) + addRecognitionError("Only expressions of the form X = X " + $sig.text + " are supported."); + addRawUpdate(operations, key, $sig.text.equals("+") ? new Operation.Addition(t) : new Operation.Substraction(t)); + } + | c=cident i=INTEGER + { + // Note that this production *is* necessary because X = X - 3 will in fact be lexed as [ X, '=', X, INTEGER]. + if (!key.equals(c)) + // We don't yet allow a '+' in front of an integer, but we could in the future really, so let's be future-proof in our error message + addRecognitionError("Only expressions of the form X = X " + ($i.text.charAt(0) == '-' ? '-' : '+') + " are supported."); + addRawUpdate(operations, key, new Operation.Addition(Constants.Literal.integer($i.text))); + } + ; + +specializedColumnOperation[List> operations, ColumnIdentifier.Raw key, Term.Raw k] + : '=' t=term + { + addRawUpdate(operations, key, new Operation.SetElement(k, t)); + } + ; + +columnCondition[List> conditions] + // Note: we'll reject duplicates later + : key=cident + ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); } + | K_IN + ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(values))); } + | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(marker))); } + ) + | '[' element=term ']' + ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionCondition(t, element, op))); } + | K_IN + ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, values))); } + | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, marker))); } + ) + ) + ) + ; + +properties[PropertyDefinitions props] + : property[props] (K_AND property[props])* + ; + +property[PropertyDefinitions props] + : k=ident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } } + | map=mapLiteral { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }) + ; + +propertyValue returns [String str] + : c=constant { $str = c.getRawText(); } + | u=unreserved_keyword { $str = u; } + ; + +relationType returns [Operator op] + : '=' { $op = Operator.EQ; } + | '<' { $op = Operator.LT; } + | '<=' { $op = Operator.LTE; } + | '>' { $op = Operator.GT; } + | '>=' { $op = Operator.GTE; } + | '!=' { $op = Operator.NEQ; } + ; + +relation[List clauses] + : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); } + | K_TOKEN l=tupleOfIdentifiers type=relationType t=term + { $clauses.add(new TokenRelation(l, type, t)); } + | name=cident K_IN marker=inMarker + { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); } + | name=cident K_IN inValues=singleColumnInValues + { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); } + | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })? + t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); } + | name=cident '[' key=term ']' type=relationType t=term { $clauses.add(new SingleColumnRelation(name, key, type, t)); } + | ids=tupleOfIdentifiers + ( K_IN + ( '(' ')' + { $clauses.add(MultiColumnRelation.createInRelation(ids, new ArrayList())); } + | tupleInMarker=inMarkerForTuple /* (a, b, c) IN ? */ + { $clauses.add(MultiColumnRelation.createSingleMarkerInRelation(ids, tupleInMarker)); } + | literals=tupleOfTupleLiterals /* (a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) */ + { + $clauses.add(MultiColumnRelation.createInRelation(ids, literals)); + } + | markers=tupleOfMarkersForTuples /* (a, b, c) IN (?, ?, ...) */ + { $clauses.add(MultiColumnRelation.createInRelation(ids, markers)); } + ) + | type=relationType literal=tupleLiteral /* (a, b, c) > (1, 2, 3) or (a, b, c) > (?, ?, ?) */ + { + $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, literal)); + } + | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */ + { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); } + ) + | '(' relation[$clauses] ')' + ; + +inMarker returns [AbstractMarker.INRaw marker] + : QMARK { $marker = newINBindVariables(null); } + | ':' name=ident { $marker = newINBindVariables(name); } + ; + +tupleOfIdentifiers returns [List ids] + @init { $ids = new ArrayList(); } + : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')' + ; + +singleColumnInValues returns [List terms] + @init { $terms = new ArrayList(); } + : '(' ( t1 = term { $terms.add(t1); } (',' ti=term { $terms.add(ti); })* )? ')' + ; + +tupleOfTupleLiterals returns [List literals] + @init { $literals = new ArrayList<>(); } + : '(' t1=tupleLiteral { $literals.add(t1); } (',' ti=tupleLiteral { $literals.add(ti); })* ')' + ; + +markerForTuple returns [Tuples.Raw marker] + : QMARK { $marker = newTupleBindVariables(null); } + | ':' name=ident { $marker = newTupleBindVariables(name); } + ; + +tupleOfMarkersForTuples returns [List markers] + @init { $markers = new ArrayList(); } + : '(' m1=markerForTuple { $markers.add(m1); } (',' mi=markerForTuple { $markers.add(mi); })* ')' + ; + +inMarkerForTuple returns [Tuples.INRaw marker] + : QMARK { $marker = newTupleINBindVariables(null); } + | ':' name=ident { $marker = newTupleINBindVariables(name); } + ; + +comparatorType returns [CQL3Type.Raw t] + : n=native_type { $t = CQL3Type.Raw.from(n); } + | c=collection_type { $t = c; } + | tt=tuple_type { $t = tt; } + | id=userTypeName { $t = CQL3Type.Raw.userType(id); } + | K_FROZEN '<' f=comparatorType '>' + { + try { + $t = CQL3Type.Raw.frozen(f); + } catch (InvalidRequestException e) { + addRecognitionError(e.getMessage()); + } + } + | s=STRING_LITERAL + { + try { + $t = CQL3Type.Raw.from(new CQL3Type.Custom($s.text)); + } catch (SyntaxException e) { + addRecognitionError("Cannot parse type " + $s.text + ": " + e.getMessage()); + } catch (ConfigurationException e) { + addRecognitionError("Error setting type " + $s.text + ": " + e.getMessage()); + } + } + ; + +native_type returns [CQL3Type t] + : K_ASCII { $t = CQL3Type.Native.ASCII; } + | K_BIGINT { $t = CQL3Type.Native.BIGINT; } + | K_BLOB { $t = CQL3Type.Native.BLOB; } + | K_BOOLEAN { $t = CQL3Type.Native.BOOLEAN; } + | K_COUNTER { $t = CQL3Type.Native.COUNTER; } + | K_DECIMAL { $t = CQL3Type.Native.DECIMAL; } + | K_DOUBLE { $t = CQL3Type.Native.DOUBLE; } + | K_FLOAT { $t = CQL3Type.Native.FLOAT; } + | K_INET { $t = CQL3Type.Native.INET;} + | K_INT { $t = CQL3Type.Native.INT; } + | K_TEXT { $t = CQL3Type.Native.TEXT; } + | K_TIMESTAMP { $t = CQL3Type.Native.TIMESTAMP; } + | K_UUID { $t = CQL3Type.Native.UUID; } + | K_VARCHAR { $t = CQL3Type.Native.VARCHAR; } + | K_VARINT { $t = CQL3Type.Native.VARINT; } + | K_TIMEUUID { $t = CQL3Type.Native.TIMEUUID; } + ; + +collection_type returns [CQL3Type.Raw pt] + : K_MAP '<' t1=comparatorType ',' t2=comparatorType '>' + { + // if we can't parse either t1 or t2, antlr will "recover" and we may have t1 or t2 null. + if (t1 != null && t2 != null) + $pt = CQL3Type.Raw.map(t1, t2); + } + | K_LIST '<' t=comparatorType '>' + { if (t != null) $pt = CQL3Type.Raw.list(t); } + | K_SET '<' t=comparatorType '>' + { if (t != null) $pt = CQL3Type.Raw.set(t); } + ; + +tuple_type returns [CQL3Type.Raw t] + : K_TUPLE '<' { List types = new ArrayList<>(); } + t1=comparatorType { types.add(t1); } (',' tn=comparatorType { types.add(tn); })* + '>' { $t = CQL3Type.Raw.tuple(types); } + ; + +username + : IDENT + | STRING_LITERAL + ; + +// Basically the same as cident, but we need to exlude existing CQL3 types +// (which for some reason are not reserved otherwise) +non_type_ident returns [ColumnIdentifier id] + : t=IDENT { if (reservedTypeNames.contains($t.text)) addRecognitionError("Invalid (reserved) user type name " + $t.text); $id = new ColumnIdentifier($t.text, false); } + | t=QUOTED_NAME { $id = new ColumnIdentifier($t.text, true); } + | k=basic_unreserved_keyword { $id = new ColumnIdentifier(k, false); } + | kk=K_KEY { $id = new ColumnIdentifier($kk.text, false); } + ; + +unreserved_keyword returns [String str] + : u=unreserved_function_keyword { $str = u; } + | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY) { $str = $k.text; } + ; + +unreserved_function_keyword returns [String str] + : u=basic_unreserved_keyword { $str = u; } + | t=native_type { $str = t.toString(); } + ; + +basic_unreserved_keyword returns [String str] + : k=( K_KEYS + | K_AS + | K_CLUSTERING + | K_COMPACT + | K_STORAGE + | K_TYPE + | K_VALUES + | K_MAP + | K_LIST + | K_FILTERING + | K_PERMISSION + | K_PERMISSIONS + | K_KEYSPACES + | K_ALL + | K_USER + | K_USERS + | K_SUPERUSER + | K_NOSUPERUSER + | K_PASSWORD + | K_EXISTS + | K_CUSTOM + | K_TRIGGER + | K_DISTINCT + | K_CONTAINS + | K_STATIC + | K_FUNCTION + | K_AGGREGATE + | K_SFUNC + | K_STYPE + | K_FINALFUNC + | K_INITCOND + | K_RETURNS + | K_LANGUAGE + | K_NON + | K_DETERMINISTIC + ) { $str = $k.text; } + ; + +// Case-insensitive keywords +K_SELECT: S E L E C T; +K_FROM: F R O M; +K_AS: A S; +K_WHERE: W H E R E; +K_AND: A N D; +K_KEY: K E Y; +K_KEYS: K E Y S; +K_ENTRIES: E N T R I E S; +K_FULL: F U L L; +K_INSERT: I N S E R T; +K_UPDATE: U P D A T E; +K_WITH: W I T H; +K_LIMIT: L I M I T; +K_USING: U S I N G; +K_USE: U S E; +K_DISTINCT: D I S T I N C T; +K_COUNT: C O U N T; +K_SET: S E T; +K_BEGIN: B E G I N; +K_UNLOGGED: U N L O G G E D; +K_BATCH: B A T C H; +K_APPLY: A P P L Y; +K_TRUNCATE: T R U N C A T E; +K_DELETE: D E L E T E; +K_IN: I N; +K_CREATE: C R E A T E; +K_KEYSPACE: ( K E Y S P A C E + | S C H E M A ); +K_KEYSPACES: K E Y S P A C E S; +K_COLUMNFAMILY:( C O L U M N F A M I L Y + | T A B L E ); +K_INDEX: I N D E X; +K_CUSTOM: C U S T O M; +K_ON: O N; +K_TO: T O; +K_DROP: D R O P; +K_PRIMARY: P R I M A R Y; +K_INTO: I N T O; +K_VALUES: V A L U E S; +K_TIMESTAMP: T I M E S T A M P; +K_TTL: T T L; +K_ALTER: A L T E R; +K_RENAME: R E N A M E; +K_ADD: A D D; +K_TYPE: T Y P E; +K_COMPACT: C O M P A C T; +K_STORAGE: S T O R A G E; +K_ORDER: O R D E R; +K_BY: B Y; +K_ASC: A S C; +K_DESC: D E S C; +K_ALLOW: A L L O W; +K_FILTERING: F I L T E R I N G; +K_IF: I F; +K_CONTAINS: C O N T A I N S; + +K_GRANT: G R A N T; +K_ALL: A L L; +K_PERMISSION: P E R M I S S I O N; +K_PERMISSIONS: P E R M I S S I O N S; +K_OF: O F; +K_REVOKE: R E V O K E; +K_MODIFY: M O D I F Y; +K_AUTHORIZE: A U T H O R I Z E; +K_NORECURSIVE: N O R E C U R S I V E; + +K_USER: U S E R; +K_USERS: U S E R S; +K_SUPERUSER: S U P E R U S E R; +K_NOSUPERUSER: N O S U P E R U S E R; +K_PASSWORD: P A S S W O R D; + +K_CLUSTERING: C L U S T E R I N G; +K_ASCII: A S C I I; +K_BIGINT: B I G I N T; +K_BLOB: B L O B; +K_BOOLEAN: B O O L E A N; +K_COUNTER: C O U N T E R; +K_DECIMAL: D E C I M A L; +K_DOUBLE: D O U B L E; +K_FLOAT: F L O A T; +K_INET: I N E T; +K_INT: I N T; +K_TEXT: T E X T; +K_UUID: U U I D; +K_VARCHAR: V A R C H A R; +K_VARINT: V A R I N T; +K_TIMEUUID: T I M E U U I D; +K_TOKEN: T O K E N; +K_WRITETIME: W R I T E T I M E; + +K_NULL: N U L L; +K_NOT: N O T; +K_EXISTS: E X I S T S; + +K_MAP: M A P; +K_LIST: L I S T; +K_NAN: N A N; +K_INFINITY: I N F I N I T Y; +K_TUPLE: T U P L E; + +K_TRIGGER: T R I G G E R; +K_STATIC: S T A T I C; +K_FROZEN: F R O Z E N; + +K_FUNCTION: F U N C T I O N; +K_AGGREGATE: A G G R E G A T E; +K_SFUNC: S F U N C; +K_STYPE: S T Y P E; +K_FINALFUNC: F I N A L F U N C; +K_INITCOND: I N I T C O N D; +K_RETURNS: R E T U R N S; +K_LANGUAGE: L A N G U A G E; +K_NON: N O N; +K_OR: O R; +K_REPLACE: R E P L A C E; +K_DETERMINISTIC: D E T E R M I N I S T I C; + +// Case-insensitive alpha characters +fragment A: ('a'|'A'); +fragment B: ('b'|'B'); +fragment C: ('c'|'C'); +fragment D: ('d'|'D'); +fragment E: ('e'|'E'); +fragment F: ('f'|'F'); +fragment G: ('g'|'G'); +fragment H: ('h'|'H'); +fragment I: ('i'|'I'); +fragment J: ('j'|'J'); +fragment K: ('k'|'K'); +fragment L: ('l'|'L'); +fragment M: ('m'|'M'); +fragment N: ('n'|'N'); +fragment O: ('o'|'O'); +fragment P: ('p'|'P'); +fragment Q: ('q'|'Q'); +fragment R: ('r'|'R'); +fragment S: ('s'|'S'); +fragment T: ('t'|'T'); +fragment U: ('u'|'U'); +fragment V: ('v'|'V'); +fragment W: ('w'|'W'); +fragment X: ('x'|'X'); +fragment Y: ('y'|'Y'); +fragment Z: ('z'|'Z'); + +STRING_LITERAL + @init{ + StringBuilder txt = new StringBuilder(); // temporary to build pg-style-string + } + @after{ setText(txt.toString()); } + : + /* pg-style string literal */ + ( + '\$' '\$' + ( /* collect all input until '$$' is reached again */ + { (input.size() - input.index() > 1) + && !"$$".equals(input.substring(input.index(), input.index() + 1)) }? + => c=. { txt.appendCodePoint(c); } + )* + '\$' '\$' + ) + | + /* conventional quoted string literal */ + ( + '\'' (c=~('\'') { txt.appendCodePoint(c);} | '\'' '\'' { txt.appendCodePoint('\''); })* '\'' + ) + ; + +QUOTED_NAME + @init{ StringBuilder b = new StringBuilder(); } + @after{ setText(b.toString()); } + : '\"' (c=~('\"') { b.appendCodePoint(c); } | '\"' '\"' { b.appendCodePoint('\"'); })+ '\"' + ; + +fragment DIGIT + : '0'..'9' + ; + +fragment LETTER + : ('A'..'Z' | 'a'..'z') + ; + +fragment HEX + : ('A'..'F' | 'a'..'f' | '0'..'9') + ; + +fragment EXPONENT + : E ('+' | '-')? DIGIT+ + ; + +INTEGER + : '-'? DIGIT+ + ; + +QMARK + : '?' + ; + +/* + * Normally a lexer only emits one token at a time, but ours is tricked out + * to support multiple (see @lexer::members near the top of the grammar). + */ +FLOAT + : INTEGER EXPONENT + | INTEGER '.' DIGIT* EXPONENT? + ; + +/* + * This has to be before IDENT so it takes precendence over it. + */ +BOOLEAN + : T R U E | F A L S E + ; + +IDENT + : LETTER (LETTER | DIGIT | '_')* + ; + +HEXNUMBER + : '0' X HEX* + ; + +UUID + : HEX HEX HEX HEX HEX HEX HEX HEX '-' + HEX HEX HEX HEX '-' + HEX HEX HEX HEX '-' + HEX HEX HEX HEX '-' + HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX + ; + +WS + : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; } + ; + +COMMENT + : ('--' | '//') .* ('\n'|'\r') { $channel = HIDDEN; } + ; + +MULTILINE_COMMENT + : '/*' .* '*/' { $channel = HIDDEN; } + ; diff --git a/cql3/ErrorCollector.java b/cql3/ErrorCollector.java new file mode 100644 index 0000000000..f49cca41d1 --- /dev/null +++ b/cql3/ErrorCollector.java @@ -0,0 +1,290 @@ +/* + * 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.LinkedList; + +import org.antlr.runtime.BaseRecognizer; +import org.antlr.runtime.Parser; +import org.antlr.runtime.RecognitionException; +import org.antlr.runtime.Token; +import org.antlr.runtime.TokenStream; +import org.apache.cassandra.exceptions.SyntaxException; + +/** + * ErrorListener that collect and enhance the errors send by the CQL lexer and parser. + */ +public final class ErrorCollector implements ErrorListener +{ + /** + * The offset of the first token of the snippet. + */ + private static final int FIRST_TOKEN_OFFSET = 10; + + /** + * The offset of the last token of the snippet. + */ + private static final int LAST_TOKEN_OFFSET = 2; + + /** + * The CQL query. + */ + private final String query; + + /** + * The error messages. + */ + private final LinkedList errorMsgs = new LinkedList<>(); + + /** + * Creates a new ErrorCollector instance to collect the syntax errors associated to the specified CQL + * query. + * + * @param query the CQL query that will be parsed + */ + public ErrorCollector(String query) + { + this.query = query; + } + + /** + * {@inheritDoc} + */ + @Override + public void syntaxError(BaseRecognizer recognizer, String[] tokenNames, RecognitionException e) + { + String hdr = recognizer.getErrorHeader(e); + String msg = recognizer.getErrorMessage(e, tokenNames); + + StringBuilder builder = new StringBuilder().append(hdr) + .append(' ') + .append(msg); + + if (recognizer instanceof Parser) + appendQuerySnippet((Parser) recognizer, builder); + + errorMsgs.add(builder.toString()); + } + + /** + * {@inheritDoc} + */ + @Override + public void syntaxError(BaseRecognizer recognizer, String errorMsg) + { + errorMsgs.add(errorMsg); + } + + /** + * Throws the first syntax error found by the lexer or the parser if it exists. + * + * @throws SyntaxException the syntax error. + */ + public void throwFirstSyntaxError() throws SyntaxException + { + if (!errorMsgs.isEmpty()) + throw new SyntaxException(errorMsgs.getFirst()); + } + + /** + * Appends a query snippet to the message to help the user to understand the problem. + * + * @param parser the parser used to parse the query + * @param builder the StringBuilder used to build the error message + */ + private void appendQuerySnippet(Parser parser, StringBuilder builder) + { + TokenStream tokenStream = parser.getTokenStream(); + int index = tokenStream.index(); + int size = tokenStream.size(); + + Token from = tokenStream.get(getSnippetFirstTokenIndex(index)); + Token to = tokenStream.get(getSnippetLastTokenIndex(index, size)); + Token offending = tokenStream.get(getOffendingTokenIndex(index, size)); + + appendSnippet(builder, from, to, offending); + } + + /** + * Appends a query snippet to the message to help the user to understand the problem. + * + * @param from the first token to include within the snippet + * @param to the last token to include within the snippet + * @param offending the token which is responsible for the error + */ + final void appendSnippet(StringBuilder builder, + Token from, + Token to, + Token offending) + { + if (!areTokensValid(from, to, offending)) + return; + + String[] lines = query.split("\n"); + + boolean includeQueryStart = (from.getLine() == 1) && (from.getCharPositionInLine() == 0); + boolean includeQueryEnd = (to.getLine() == lines.length) + && (getLastCharPositionInLine(to) == lines[lines.length - 1].length()); + + builder.append(" ("); + + if (!includeQueryStart) + builder.append("..."); + + String toLine = lines[lineIndex(to)]; + int toEnd = getLastCharPositionInLine(to); + lines[lineIndex(to)] = toEnd >= toLine.length() ? toLine : toLine.substring(0, toEnd); + lines[lineIndex(offending)] = highlightToken(lines[lineIndex(offending)], offending); + lines[lineIndex(from)] = lines[lineIndex(from)].substring(from.getCharPositionInLine()); + + for (int i = lineIndex(from), m = lineIndex(to); i <= m; i++) + builder.append(lines[i]); + + if (!includeQueryEnd) + builder.append("..."); + + builder.append(")"); + } + + /** + * Checks if the specified tokens are valid. + * + * @param tokens the tokens to check + * @return true if all the specified tokens are valid ones, + * false otherwise. + */ + private static boolean areTokensValid(Token... tokens) + { + for (Token token : tokens) + { + if (!isTokenValid(token)) + return false; + } + return true; + } + + /** + * Checks that the specified token is valid. + * + * @param token the token to check + * @return true if it is considered as valid, false otherwise. + */ + private static boolean isTokenValid(Token token) + { + return token.getLine() > 0 && token.getCharPositionInLine() >= 0; + } + + /** + * Returns the index of the offending token.

In the case where the offending token is an extra + * character at the end, the index returned by the TokenStream might be after the last token. + * To avoid that problem we need to make sure that the index of the offending token is a valid index + * (one for which a token exist).

+ * + * @param index the token index returned by the TokenStream + * @param size the TokenStream size + * @return the valid index of the offending token + */ + private static int getOffendingTokenIndex(int index, int size) + { + return Math.min(index, size - 1); + } + + /** + * Puts the specified token within square brackets. + * + * @param line the line containing the token + * @param token the token to put within square brackets + */ + private static String highlightToken(String line, Token token) + { + String newLine = insertChar(line, getLastCharPositionInLine(token), ']'); + return insertChar(newLine, token.getCharPositionInLine(), '['); + } + + /** + * Returns the index of the last character relative to the beginning of the line 0..n-1 + * + * @param token the token + * @return the index of the last character relative to the beginning of the line 0..n-1 + */ + private static int getLastCharPositionInLine(Token token) + { + return token.getCharPositionInLine() + getLength(token); + } + + /** + * Return the token length. + * + * @param token the token + * @return the token length + */ + private static int getLength(Token token) + { + return token.getText().length(); + } + + /** + * Inserts a character at a given position within a String. + * + * @param s the String in which the character must be inserted + * @param index the position where the character must be inserted + * @param c the character to insert + * @return the modified String + */ + private static String insertChar(String s, int index, char c) + { + return new StringBuilder().append(s.substring(0, index)) + .append(c) + .append(s.substring(index)) + .toString(); + } + + /** + * Returns the index of the line number on which this token was matched; index=0..n-1 + * + * @param token the token + * @return the index of the line number on which this token was matched; index=0..n-1 + */ + private static int lineIndex(Token token) + { + return token.getLine() - 1; + } + + /** + * Returns the index of the last token which is part of the snippet. + * + * @param index the index of the token causing the error + * @param size the total number of tokens + * @return the index of the last token which is part of the snippet. + */ + private static int getSnippetLastTokenIndex(int index, int size) + { + return Math.min(size - 1, index + LAST_TOKEN_OFFSET); + } + + /** + * Returns the index of the first token which is part of the snippet. + * + * @param index the index of the token causing the error + * @return the index of the first token which is part of the snippet. + */ + private static int getSnippetFirstTokenIndex(int index) + { + return Math.max(0, index - FIRST_TOKEN_OFFSET); + } +} diff --git a/cql3/ErrorListener.java b/cql3/ErrorListener.java new file mode 100644 index 0000000000..0bf891a71c --- /dev/null +++ b/cql3/ErrorListener.java @@ -0,0 +1,44 @@ +/* + * 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 org.antlr.runtime.BaseRecognizer; +import org.antlr.runtime.RecognitionException; + +/** + * Listener used to collect the syntax errors emitted by the Lexer and Parser. + */ +public interface ErrorListener +{ + /** + * Invoked when a syntax error occurs. + * + * @param recognizer the parser or lexer that emitted the error + * @param tokenNames the token names + * @param e the exception + */ + void syntaxError(BaseRecognizer recognizer, String[] tokenNames, RecognitionException e); + + /** + * Invoked when a syntax error with a specified message occurs. + * + * @param recognizer the parser or lexer that emitted the error + * @param errorMsg the error message + */ + void syntaxError(BaseRecognizer recognizer, String errorMsg); +} diff --git a/cql3/IndexName.java b/cql3/IndexName.java new file mode 100644 index 0000000000..ded86e4c71 --- /dev/null +++ b/cql3/IndexName.java @@ -0,0 +1,65 @@ +/* + * 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.Locale; + +public class IndexName +{ + private String ksName; + private String idxName; + + public void setKeyspace(String ks, boolean keepCase) + { + ksName = keepCase ? ks : ks.toLowerCase(Locale.US); + } + + public void setIndex(String idx, boolean keepCase) + { + idxName = keepCase ? idx : idx.toLowerCase(Locale.US); + } + + public boolean hasKeyspace() + { + return ksName != null; + } + + public String getKeyspace() + { + return ksName; + } + + public String getIdx() + { + return idxName; + } + + public CFName getCfName() + { + CFName cfName = new CFName(); + if (hasKeyspace()) + cfName.setKeyspace(ksName, true); + return cfName; + } + + @Override + public String toString() + { + return (hasKeyspace() ? (ksName + ".") : "") + idxName; + } +} diff --git a/cql3/Lists.java b/cql3/Lists.java new file mode 100644 index 0000000000..891b980f61 --- /dev/null +++ b/cql3/Lists.java @@ -0,0 +1,508 @@ +/* + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.Cell; +import org.apache.cassandra.db.ColumnFamily; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.CollectionSerializer; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.UUIDGen; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Static helper methods and classes for lists. + */ +public abstract class Lists +{ + private static final Logger logger = LoggerFactory.getLogger(Lists.class); + + private Lists() {} + + public static ColumnSpecification indexSpecOf(ColumnSpecification column) + { + return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("idx(" + column.name + ")", true), Int32Type.instance); + } + + public static ColumnSpecification valueSpecOf(ColumnSpecification column) + { + return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((ListType)column.type).getElementsType()); + } + + public static class Literal implements Term.Raw + { + private final List elements; + + public Literal(List elements) + { + this.elements = elements; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + validateAssignableTo(keyspace, receiver); + + ColumnSpecification valueSpec = Lists.valueSpecOf(receiver); + List values = new ArrayList(elements.size()); + boolean allTerminal = true; + for (Term.Raw rt : elements) + { + Term t = rt.prepare(keyspace, valueSpec); + + if (t.containsBindMarker()) + throw new InvalidRequestException(String.format("Invalid list literal for %s: bind variables are not supported inside collection literals", receiver.name)); + + if (t instanceof Term.NonTerminal) + allTerminal = false; + + values.add(t); + } + DelayedValue value = new DelayedValue(values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof ListType)) + throw new InvalidRequestException(String.format("Invalid list literal for %s of type %s", receiver.name, receiver.type.asCQL3Type())); + + ColumnSpecification valueSpec = Lists.valueSpecOf(receiver); + for (Term.Raw rt : elements) + { + if (!rt.testAssignment(keyspace, valueSpec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid list literal for %s: value %s is not of type %s", receiver.name, rt, valueSpec.type.asCQL3Type())); + } + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + if (!(receiver.type instanceof ListType)) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + + // If there is no elements, we can't say it's an exact match (an empty list if fundamentally polymorphic). + if (elements.isEmpty()) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + + ColumnSpecification valueSpec = Lists.valueSpecOf(receiver); + return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements); + } + + @Override + public String toString() + { + return elements.toString(); + } + } + + public static class Value extends Term.MultiItemTerminal implements Term.CollectionTerminal + { + public final List elements; + + public Value(List elements) + { + this.elements = elements; + } + + public static Value fromSerialized(ByteBuffer value, ListType type, int version) throws InvalidRequestException + { + try + { + // Collections have this small hack that validate cannot be called on a serialized object, + // but compose does the validation (so we're fine). + List l = (List)type.getSerializer().deserializeForNativeProtocol(value, version); + List elements = new ArrayList(l.size()); + for (Object element : l) + // elements can be null in lists that represent a set of IN values + elements.add(element == null ? null : type.getElementsType().decompose(element)); + return new Value(elements); + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public ByteBuffer get(QueryOptions options) + { + return getWithProtocolVersion(options.getProtocolVersion()); + } + + public ByteBuffer getWithProtocolVersion(int protocolVersion) + { + return CollectionSerializer.pack(elements, elements.size(), protocolVersion); + } + + public boolean equals(ListType lt, Value v) + { + if (elements.size() != v.elements.size()) + return false; + + for (int i = 0; i < elements.size(); i++) + if (lt.getElementsType().compare(elements.get(i), v.elements.get(i)) != 0) + return false; + + return true; + } + + public List getElements() + { + return elements; + } + } + + /** + * Basically similar to a Value, but with some non-pure function (that need + * to be evaluated at execution time) in it. + * + * Note: this would also work for a list with bind markers, but we don't support + * that because 1) it's not excessively useful and 2) we wouldn't have a good + * column name to return in the ColumnSpecification for those markers (not a + * blocker per-se but we don't bother due to 1)). + */ + public static class DelayedValue extends Term.NonTerminal + { + private final List elements; + + public DelayedValue(List elements) + { + this.elements = elements; + } + + public boolean containsBindMarker() + { + // False since we don't support them in collection + return false; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + List buffers = new ArrayList(elements.size()); + for (Term t : elements) + { + ByteBuffer bytes = t.bindAndGet(options); + + if (bytes == null) + throw new InvalidRequestException("null is not supported inside collections"); + + // We don't support value > 64K because the serialization format encode the length as an unsigned short. + if (bytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("List value is too long. List values are limited to %d bytes but %d bytes value provided", + FBUtilities.MAX_UNSIGNED_SHORT, + bytes.remaining())); + + buffers.add(bytes); + } + return new Value(buffers); + } + } + + /** + * A marker for List values and IN relations + */ + public static class Marker extends AbstractMarker + { + protected Marker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + assert receiver.type instanceof ListType; + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer value = options.getValues().get(bindIndex); + return value == null ? null : Value.fromSerialized(value, (ListType)receiver.type, options.getProtocolVersion()); + } + } + + /* + * For prepend, we need to be able to generate unique but decreasing time + * UUID, which is a bit challenging. To do that, given a time in milliseconds, + * we adds a number representing the 100-nanoseconds precision and make sure + * that within the same millisecond, that number is always decreasing. We + * do rely on the fact that the user will only provide decreasing + * milliseconds timestamp for that purpose. + */ + private static class PrecisionTime + { + // Our reference time (1 jan 2010, 00:00:00) in milliseconds. + private static final long REFERENCE_TIME = 1262304000000L; + private static final AtomicReference last = new AtomicReference(new PrecisionTime(Long.MAX_VALUE, 0)); + + public final long millis; + public final int nanos; + + PrecisionTime(long millis, int nanos) + { + this.millis = millis; + this.nanos = nanos; + } + + static PrecisionTime getNext(long millis) + { + while (true) + { + PrecisionTime current = last.get(); + + assert millis <= current.millis; + PrecisionTime next = millis < current.millis + ? new PrecisionTime(millis, 9999) + : new PrecisionTime(millis, Math.max(0, current.nanos - 1)); + + if (last.compareAndSet(current, next)) + return next; + } + } + } + + public static class Setter extends Operation + { + public Setter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + if (column.type.isMultiCell()) + { + // delete + append + CellName name = cf.getComparator().create(prefix, column); + cf.addAtom(params.makeTombstoneForOverwrite(name.slice())); + } + Appender.doAppend(t, cf, prefix, column, params); + } + } + + public static class SetterByIndex extends Operation + { + private final Term idx; + + public SetterByIndex(ColumnDefinition column, Term idx, Term t) + { + super(column, t); + this.idx = idx; + } + + @Override + public boolean requiresRead() + { + return true; + } + + @Override + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + super.collectMarkerSpecification(boundNames); + idx.collectMarkerSpecification(boundNames); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + // we should not get here for frozen lists + assert column.type.isMultiCell() : "Attempted to set an individual element on a frozen list"; + + ByteBuffer index = idx.bindAndGet(params.options); + ByteBuffer value = t.bindAndGet(params.options); + + if (index == null) + throw new InvalidRequestException("Invalid null value for list index"); + + List existingList = params.getPrefetchedList(rowKey, column.name); + int idx = ByteBufferUtil.toInt(index); + if (idx < 0 || idx >= existingList.size()) + throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingList.size())); + + CellName elementName = existingList.get(idx).name(); + if (value == null) + { + cf.addColumn(params.makeTombstone(elementName)); + } + else + { + // We don't support value > 64K because the serialization format encode the length as an unsigned short. + if (value.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("List value is too long. List values are limited to %d bytes but %d bytes value provided", + FBUtilities.MAX_UNSIGNED_SHORT, + value.remaining())); + + cf.addColumn(params.makeColumn(elementName, value)); + } + } + } + + public static class Appender extends Operation + { + public Appender(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to append to a frozen list"; + doAppend(t, cf, prefix, column, params); + } + + static void doAppend(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException + { + Term.Terminal value = t.bind(params.options); + Lists.Value listValue = (Lists.Value)value; + if (column.type.isMultiCell()) + { + // If we append null, do nothing. Note that for Setter, we've + // already removed the previous value so we're good here too + if (value == null) + return; + + List toAdd = listValue.elements; + for (int i = 0; i < toAdd.size(); i++) + { + ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()); + cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), toAdd.get(i))); + } + } + else + { + // for frozen lists, we're overwriting the whole cell value + CellName name = cf.getComparator().create(prefix, column); + if (value == null) + cf.addAtom(params.makeTombstone(name)); + else + cf.addColumn(params.makeColumn(name, listValue.getWithProtocolVersion(Server.CURRENT_VERSION))); + } + } + } + + public static class Prepender extends Operation + { + public Prepender(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to prepend to a frozen list"; + Term.Terminal value = t.bind(params.options); + if (value == null) + return; + + assert value instanceof Lists.Value; + long time = PrecisionTime.REFERENCE_TIME - (System.currentTimeMillis() - PrecisionTime.REFERENCE_TIME); + + List toAdd = ((Lists.Value)value).elements; + for (int i = 0; i < toAdd.size(); i++) + { + PrecisionTime pt = PrecisionTime.getNext(time); + ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(pt.millis, pt.nanos)); + cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), toAdd.get(i))); + } + } + } + + public static class Discarder extends Operation + { + public Discarder(ColumnDefinition column, Term t) + { + super(column, t); + } + + @Override + public boolean requiresRead() + { + return true; + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to delete from a frozen list"; + List existingList = params.getPrefetchedList(rowKey, column.name); + // We want to call bind before possibly returning to reject queries where the value provided is not a list. + Term.Terminal value = t.bind(params.options); + + if (existingList.isEmpty()) + return; + + if (value == null) + return; + + assert value instanceof Lists.Value; + + // Note: below, we will call 'contains' on this toDiscard list for each element of existingList. + // Meaning that if toDiscard is big, converting it to a HashSet might be more efficient. However, + // the read-before-write this operation requires limits its usefulness on big lists, so in practice + // toDiscard will be small and keeping a list will be more efficient. + List toDiscard = ((Lists.Value)value).elements; + for (Cell cell : existingList) + { + if (toDiscard.contains(cell.value())) + cf.addColumn(params.makeTombstone(cell.name())); + } + } + } + + public static class DiscarderByIndex extends Operation + { + public DiscarderByIndex(ColumnDefinition column, Term idx) + { + super(column, idx); + } + + @Override + public boolean requiresRead() + { + return true; + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to delete an item by index from a frozen list"; + Term.Terminal index = t.bind(params.options); + if (index == null) + throw new InvalidRequestException("Invalid null value for list index"); + + assert index instanceof Constants.Value; + + List existingList = params.getPrefetchedList(rowKey, column.name); + int idx = ByteBufferUtil.toInt(((Constants.Value)index).bytes); + if (idx < 0 || idx >= existingList.size()) + throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingList.size())); + + CellName elementName = existingList.get(idx).name(); + cf.addColumn(params.makeTombstone(elementName)); + } + } +} diff --git a/cql3/Maps.java b/cql3/Maps.java new file mode 100644 index 0000000000..3a346ff199 --- /dev/null +++ b/cql3/Maps.java @@ -0,0 +1,399 @@ +/* + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.ColumnFamily; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.CollectionSerializer; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Pair; + +/** + * Static helper methods and classes for maps. + */ +public abstract class Maps +{ + private Maps() {} + + public static ColumnSpecification keySpecOf(ColumnSpecification column) + { + return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("key(" + column.name + ")", true), ((MapType)column.type).getKeysType()); + } + + public static ColumnSpecification valueSpecOf(ColumnSpecification column) + { + return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((MapType)column.type).getValuesType()); + } + + public static class Literal implements Term.Raw + { + public final List> entries; + + public Literal(List> entries) + { + this.entries = entries; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + validateAssignableTo(keyspace, receiver); + + ColumnSpecification keySpec = Maps.keySpecOf(receiver); + ColumnSpecification valueSpec = Maps.valueSpecOf(receiver); + Map values = new HashMap(entries.size()); + boolean allTerminal = true; + for (Pair entry : entries) + { + Term k = entry.left.prepare(keyspace, keySpec); + Term v = entry.right.prepare(keyspace, valueSpec); + + if (k.containsBindMarker() || v.containsBindMarker()) + throw new InvalidRequestException(String.format("Invalid map literal for %s: bind variables are not supported inside collection literals", receiver.name)); + + if (k instanceof Term.NonTerminal || v instanceof Term.NonTerminal) + allTerminal = false; + + values.put(k, v); + } + DelayedValue value = new DelayedValue(((MapType)receiver.type).getKeysType(), values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof MapType)) + throw new InvalidRequestException(String.format("Invalid map literal for %s of type %s", receiver.name, receiver.type.asCQL3Type())); + + ColumnSpecification keySpec = Maps.keySpecOf(receiver); + ColumnSpecification valueSpec = Maps.valueSpecOf(receiver); + for (Pair entry : entries) + { + if (!entry.left.testAssignment(keyspace, keySpec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid map literal for %s: key %s is not of type %s", receiver.name, entry.left, keySpec.type.asCQL3Type())); + if (!entry.right.testAssignment(keyspace, valueSpec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid map literal for %s: value %s is not of type %s", receiver.name, entry.right, valueSpec.type.asCQL3Type())); + } + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + if (!(receiver.type instanceof MapType)) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + + // If there is no elements, we can't say it's an exact match (an empty map if fundamentally polymorphic). + if (entries.isEmpty()) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + + ColumnSpecification keySpec = Maps.keySpecOf(receiver); + ColumnSpecification valueSpec = Maps.valueSpecOf(receiver); + // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable. + AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH; + for (Pair entry : entries) + { + AssignmentTestable.TestResult t1 = entry.left.testAssignment(keyspace, keySpec); + AssignmentTestable.TestResult t2 = entry.right.testAssignment(keyspace, valueSpec); + if (t1 == AssignmentTestable.TestResult.NOT_ASSIGNABLE || t2 == AssignmentTestable.TestResult.NOT_ASSIGNABLE) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + if (t1 != AssignmentTestable.TestResult.EXACT_MATCH || t2 != AssignmentTestable.TestResult.EXACT_MATCH) + res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + return res; + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + for (int i = 0; i < entries.size(); i++) + { + if (i > 0) sb.append(", "); + sb.append(entries.get(i).left).append(":").append(entries.get(i).right); + } + sb.append("}"); + return sb.toString(); + } + } + + public static class Value extends Term.Terminal implements Term.CollectionTerminal + { + public final Map map; + + public Value(Map map) + { + this.map = map; + } + + public static Value fromSerialized(ByteBuffer value, MapType type, int version) throws InvalidRequestException + { + try + { + // Collections have this small hack that validate cannot be called on a serialized object, + // but compose does the validation (so we're fine). + Map m = (Map)type.getSerializer().deserializeForNativeProtocol(value, version); + Map map = new LinkedHashMap(m.size()); + for (Map.Entry entry : m.entrySet()) + map.put(type.getKeysType().decompose(entry.getKey()), type.getValuesType().decompose(entry.getValue())); + return new Value(map); + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public ByteBuffer get(QueryOptions options) + { + return getWithProtocolVersion(options.getProtocolVersion()); + } + + public ByteBuffer getWithProtocolVersion(int protocolVersion) + { + List buffers = new ArrayList<>(2 * map.size()); + for (Map.Entry entry : map.entrySet()) + { + buffers.add(entry.getKey()); + buffers.add(entry.getValue()); + } + return CollectionSerializer.pack(buffers, map.size(), protocolVersion); + } + + public boolean equals(MapType mt, Value v) + { + if (map.size() != v.map.size()) + return false; + + // We use the fact that we know the maps iteration will both be in comparator order + Iterator> thisIter = map.entrySet().iterator(); + Iterator> thatIter = v.map.entrySet().iterator(); + while (thisIter.hasNext()) + { + Map.Entry thisEntry = thisIter.next(); + Map.Entry thatEntry = thatIter.next(); + if (mt.getKeysType().compare(thisEntry.getKey(), thatEntry.getKey()) != 0 || mt.getValuesType().compare(thisEntry.getValue(), thatEntry.getValue()) != 0) + return false; + } + + return true; + } + } + + // See Lists.DelayedValue + public static class DelayedValue extends Term.NonTerminal + { + private final Comparator comparator; + private final Map elements; + + public DelayedValue(Comparator comparator, Map elements) + { + this.comparator = comparator; + this.elements = elements; + } + + public boolean containsBindMarker() + { + // False since we don't support them in collection + return false; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + Map buffers = new TreeMap(comparator); + for (Map.Entry entry : elements.entrySet()) + { + // We don't support values > 64K because the serialization format encode the length as an unsigned short. + ByteBuffer keyBytes = entry.getKey().bindAndGet(options); + if (keyBytes == null) + throw new InvalidRequestException("null is not supported inside collections"); + if (keyBytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("Map key is too long. Map keys are limited to %d bytes but %d bytes keys provided", + FBUtilities.MAX_UNSIGNED_SHORT, + keyBytes.remaining())); + + ByteBuffer valueBytes = entry.getValue().bindAndGet(options); + if (valueBytes == null) + throw new InvalidRequestException("null is not supported inside collections"); + if (valueBytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("Map value is too long. Map values are limited to %d bytes but %d bytes value provided", + FBUtilities.MAX_UNSIGNED_SHORT, + valueBytes.remaining())); + + buffers.put(keyBytes, valueBytes); + } + return new Value(buffers); + } + } + + public static class Marker extends AbstractMarker + { + protected Marker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + assert receiver.type instanceof MapType; + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer value = options.getValues().get(bindIndex); + return value == null ? null : Value.fromSerialized(value, (MapType)receiver.type, options.getProtocolVersion()); + } + } + + public static class Setter extends Operation + { + public Setter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + if (column.type.isMultiCell()) + { + // delete + put + CellName name = cf.getComparator().create(prefix, column); + cf.addAtom(params.makeTombstoneForOverwrite(name.slice())); + } + Putter.doPut(t, cf, prefix, column, params); + } + } + + public static class SetterByKey extends Operation + { + private final Term k; + + public SetterByKey(ColumnDefinition column, Term k, Term t) + { + super(column, t); + this.k = k; + } + + @Override + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + super.collectMarkerSpecification(boundNames); + k.collectMarkerSpecification(boundNames); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to set a value for a single key on a frozen map"; + ByteBuffer key = k.bindAndGet(params.options); + ByteBuffer value = t.bindAndGet(params.options); + if (key == null) + throw new InvalidRequestException("Invalid null map key"); + + CellName cellName = cf.getComparator().create(prefix, column, key); + + if (value == null) + { + cf.addColumn(params.makeTombstone(cellName)); + } + else + { + // We don't support value > 64K because the serialization format encode the length as an unsigned short. + if (value.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("Map value is too long. Map values are limited to %d bytes but %d bytes value provided", + FBUtilities.MAX_UNSIGNED_SHORT, + value.remaining())); + + cf.addColumn(params.makeColumn(cellName, value)); + } + } + } + + public static class Putter extends Operation + { + public Putter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to add items to a frozen map"; + doPut(t, cf, prefix, column, params); + } + + static void doPut(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException + { + Term.Terminal value = t.bind(params.options); + Maps.Value mapValue = (Maps.Value) value; + if (column.type.isMultiCell()) + { + if (value == null) + return; + + for (Map.Entry entry : mapValue.map.entrySet()) + { + CellName cellName = cf.getComparator().create(prefix, column, entry.getKey()); + cf.addColumn(params.makeColumn(cellName, entry.getValue())); + } + } + else + { + // for frozen maps, we're overwriting the whole cell + CellName cellName = cf.getComparator().create(prefix, column); + if (value == null) + cf.addAtom(params.makeTombstone(cellName)); + else + cf.addColumn(params.makeColumn(cellName, mapValue.getWithProtocolVersion(Server.CURRENT_VERSION))); + } + } + } + + public static class DiscarderByKey extends Operation + { + public DiscarderByKey(ColumnDefinition column, Term k) + { + super(column, k); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to delete a single key in a frozen map"; + Term.Terminal key = t.bind(params.options); + if (key == null) + throw new InvalidRequestException("Invalid null map key"); + assert key instanceof Constants.Value; + + CellName cellName = cf.getComparator().create(prefix, column, ((Constants.Value)key).bytes); + cf.addColumn(params.makeTombstone(cellName)); + } + } +} diff --git a/cql3/MultiColumnRelation.java b/cql3/MultiColumnRelation.java new file mode 100644 index 0000000000..d754968332 --- /dev/null +++ b/cql3/MultiColumnRelation.java @@ -0,0 +1,216 @@ +/* + * 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.ArrayList; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.Term.MultiColumnRaw; +import org.apache.cassandra.cql3.Term.Raw; +import org.apache.cassandra.cql3.restrictions.MultiColumnRestriction; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue; +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +/** + * A relation using the tuple notation, which typically affects multiple columns. + * Examples: + * - SELECT ... WHERE (a, b, c) > (1, 'a', 10) + * - SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6)) + * - SELECT ... WHERE (a, b) < ? + * - SELECT ... WHERE (a, b) IN ? + */ +public class MultiColumnRelation extends Relation +{ + private final List entities; + + /** A Tuples.Literal or Tuples.Raw marker */ + private final Term.MultiColumnRaw valuesOrMarker; + + /** A list of Tuples.Literal or Tuples.Raw markers */ + private final List inValues; + + private final Tuples.INRaw inMarker; + + private MultiColumnRelation(List entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List inValues, Tuples.INRaw inMarker) + { + this.entities = entities; + this.relationType = relationType; + this.valuesOrMarker = valuesOrMarker; + + this.inValues = inValues; + this.inMarker = inMarker; + } + + /** + * Creates a multi-column EQ, LT, LTE, GT, or GTE relation. + * For example: "SELECT ... WHERE (a, b) > (0, 1)" + * @param entities the columns on the LHS of the relation + * @param relationType the relation operator + * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker + * @return a new MultiColumnRelation instance + */ + public static MultiColumnRelation createNonInRelation(List entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker) + { + assert relationType != Operator.IN; + return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null); + } + + /** + * Creates a multi-column IN relation with a list of IN values or markers. + * For example: "SELECT ... WHERE (a, b) IN ((0, 1), (2, 3))" + * @param entities the columns on the LHS of the relation + * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers + * @return a new MultiColumnRelation instance + */ + public static MultiColumnRelation createInRelation(List entities, List inValues) + { + return new MultiColumnRelation(entities, Operator.IN, null, inValues, null); + } + + /** + * Creates a multi-column IN relation with a marker for the IN values. + * For example: "SELECT ... WHERE (a, b) IN ?" + * @param entities the columns on the LHS of the relation + * @param inMarker a single IN marker + * @return a new MultiColumnRelation instance + */ + public static MultiColumnRelation createSingleMarkerInRelation(List entities, Tuples.INRaw inMarker) + { + return new MultiColumnRelation(entities, Operator.IN, null, null, inMarker); + } + + public List getEntities() + { + return entities; + } + + /** + * For non-IN relations, returns the Tuples.Literal or Tuples.Raw marker for a single tuple. + * @return a Tuples.Literal for non-IN relations or Tuples.Raw marker for a single tuple. + */ + private Term.MultiColumnRaw getValue() + { + return relationType == Operator.IN ? inMarker : valuesOrMarker; + } + + @Override + public boolean isMultiColumn() + { + return true; + } + + @Override + protected Restriction newEQRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException + { + List receivers = receivers(cfm); + Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames); + return new MultiColumnRestriction.EQ(cfm.comparator, receivers, term); + } + + @Override + protected Restriction newINRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException + { + List receivers = receivers(cfm); + List terms = toTerms(receivers, inValues, cfm.ksName, boundNames); + if (terms == null) + { + Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames); + return new MultiColumnRestriction.InWithMarker(cfm.comparator, receivers, (AbstractMarker) term); + } + return new MultiColumnRestriction.InWithValues(cfm.comparator, receivers, terms); + } + + @Override + protected Restriction newSliceRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + Bound bound, + boolean inclusive) throws InvalidRequestException + { + List receivers = receivers(cfm); + Term term = toTerm(receivers(cfm), getValue(), cfm.ksName, boundNames); + return new MultiColumnRestriction.Slice(cfm.comparator, receivers, bound, inclusive, term); + } + + @Override + protected Restriction newContainsRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + boolean isKey) throws InvalidRequestException + { + throw invalidRequest("%s cannot be used for Multi-column relations", operator()); + } + + @Override + protected Term toTerm(List receivers, + Raw raw, + String keyspace, + VariableSpecifications boundNames) throws InvalidRequestException + { + Term term = ((MultiColumnRaw) raw).prepare(keyspace, receivers); + term.collectMarkerSpecification(boundNames); + return term; + } + + protected List receivers(CFMetaData cfm) throws InvalidRequestException + { + List names = new ArrayList<>(getEntities().size()); + int previousPosition = -1; + for (ColumnIdentifier.Raw raw : getEntities()) + { + ColumnDefinition def = toColumnDefinition(cfm, raw); + checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name); + checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this); + + // check that no clustering columns were skipped + if (def.position() != previousPosition + 1) + { + checkFalse(previousPosition == -1, "Clustering columns may not be skipped in multi-column relations. " + + "They should appear in the PRIMARY KEY order. Got %s", this); + throw invalidRequest("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", this); + } + names.add(def); + previousPosition = def.position(); + } + return names; + } + + @Override + public String toString() + { + StringBuilder builder = new StringBuilder(Tuples.tupleToString(entities)); + if (isIN()) + { + return builder.append(" IN ") + .append(inMarker != null ? '?' : Tuples.tupleToString(inValues)) + .toString(); + } + return builder.append(" ") + .append(relationType) + .append(" ") + .append(valuesOrMarker) + .toString(); + } +} \ No newline at end of file diff --git a/cql3/Operation.java b/cql3/Operation.java new file mode 100644 index 0000000000..583158b4fc --- /dev/null +++ b/cql3/Operation.java @@ -0,0 +1,429 @@ +/* + * 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.nio.ByteBuffer; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.ColumnFamily; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * An UPDATE or DELETE operation. + * + * For UPDATE this includes: + * - setting a constant + * - counter operations + * - collections operations + * and for DELETE: + * - deleting a column + * - deleting an element of collection column + * + * Fine grained operation are obtained from their raw counterpart (Operation.Raw, which + * correspond to a parsed, non-checked operation) by provided the receiver for the operation. + */ +public abstract class Operation +{ + // the column the operation applies to + public final ColumnDefinition column; + + // Term involved in the operation. In theory this should not be here since some operation + // may require none of more than one term, but most need 1 so it simplify things a bit. + protected final Term t; + + protected Operation(ColumnDefinition column, Term t) + { + assert column != null; + this.column = column; + this.t = t; + } + + public boolean usesFunction(String ksName, String functionName) + { + return t != null && t.usesFunction(ksName, functionName); + } + + /** + * @return whether the operation requires a read of the previous value to be executed + * (only lists setterByIdx, discard and discardByIdx requires that). + */ + public boolean requiresRead() + { + return false; + } + + /** + * Collects the column specification for the bind variables of this operation. + * + * @param boundNames the list of column specification where to collect the + * bind variables of this term in. + */ + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + if (t != null) + t.collectMarkerSpecification(boundNames); + } + + /** + * Execute the operation. + * + * @param rowKey row key for the update. + * @param cf the column family to which to add the updates generated by this operation. + * @param prefix the prefix that identify the CQL3 row this operation applies to. + * @param params parameters of the update. + */ + public abstract void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException; + + /** + * A parsed raw UPDATE operation. + * + * This can be one of: + * - Setting a value: c = v + * - Setting an element of a collection: c[x] = v + * - An addition/subtraction to a variable: c = c +/- v (where v can be a collection literal) + * - An prepend operation: c = v + c + */ + public interface RawUpdate + { + /** + * This method validates the operation (i.e. validate it is well typed) + * based on the specification of the receiver of the operation. + * + * It returns an Operation which can be though as post-preparation well-typed + * Operation. + * + * @param receiver the "column" this operation applies to. Note that + * contrarly to the method of same name in Term.Raw, the receiver should always + * be a true column. + * @return the prepared update operation. + */ + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException; + + /** + * @return whether this operation can be applied alongside the {@code + * other} update (in the same UPDATE statement for the same column). + */ + public boolean isCompatibleWith(RawUpdate other); + } + + /** + * A parsed raw DELETE operation. + * + * This can be one of: + * - Deleting a column + * - Deleting an element of a collection + */ + public interface RawDeletion + { + /** + * The name of the column affected by this delete operation. + */ + public ColumnIdentifier.Raw affectedColumn(); + + /** + * This method validates the operation (i.e. validate it is well typed) + * based on the specification of the column affected by the operation (i.e the + * one returned by affectedColumn()). + * + * It returns an Operation which can be though as post-preparation well-typed + * Operation. + * + * @param receiver the "column" this operation applies to. + * @return the prepared delete operation. + */ + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException; + } + + public static class SetValue implements RawUpdate + { + private final Term.Raw value; + + public SetValue(Term.Raw value) + { + this.value = value; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + Term v = value.prepare(keyspace, receiver); + + if (receiver.type instanceof CounterColumnType) + throw new InvalidRequestException(String.format("Cannot set the value of counter column %s (counters can only be incremented/decremented, not set)", receiver.name)); + + if (!(receiver.type.isCollection())) + return new Constants.Setter(receiver, v); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: + return new Lists.Setter(receiver, v); + case SET: + return new Sets.Setter(receiver, v); + case MAP: + return new Maps.Setter(receiver, v); + } + throw new AssertionError(); + } + + protected String toString(ColumnSpecification column) + { + return String.format("%s = %s", column, value); + } + + public boolean isCompatibleWith(RawUpdate other) + { + // We don't allow setting multiple time the same column, because 1) + // it's stupid and 2) the result would seem random to the user. + return false; + } + } + + public static class SetElement implements RawUpdate + { + private final Term.Raw selector; + private final Term.Raw value; + + public SetElement(Term.Raw selector, Term.Raw value) + { + this.selector = selector; + this.value = value; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof CollectionType)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non collection column %s", toString(receiver), receiver.name)); + else if (!(receiver.type.isMultiCell())) + throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen collection column %s", toString(receiver), receiver.name)); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: + Term idx = selector.prepare(keyspace, Lists.indexSpecOf(receiver)); + Term lval = value.prepare(keyspace, Lists.valueSpecOf(receiver)); + return new Lists.SetterByIndex(receiver, idx, lval); + case SET: + throw new InvalidRequestException(String.format("Invalid operation (%s) for set column %s", toString(receiver), receiver.name)); + case MAP: + Term key = selector.prepare(keyspace, Maps.keySpecOf(receiver)); + Term mval = value.prepare(keyspace, Maps.valueSpecOf(receiver)); + return new Maps.SetterByKey(receiver, key, mval); + } + throw new AssertionError(); + } + + protected String toString(ColumnSpecification column) + { + return String.format("%s[%s] = %s", column.name, selector, value); + } + + public boolean isCompatibleWith(RawUpdate other) + { + // TODO: we could check that the other operation is not setting the same element + // too (but since the index/key set may be a bind variables we can't always do it at this point) + return !(other instanceof SetValue); + } + } + + public static class Addition implements RawUpdate + { + private final Term.Raw value; + + public Addition(Term.Raw value) + { + this.value = value; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + Term v = value.prepare(keyspace, receiver); + + if (!(receiver.type instanceof CollectionType)) + { + if (!(receiver.type instanceof CounterColumnType)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver.name)); + return new Constants.Adder(receiver, v); + } + else if (!(receiver.type.isMultiCell())) + throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen collection column %s", toString(receiver), receiver.name)); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: + return new Lists.Appender(receiver, v); + case SET: + return new Sets.Adder(receiver, v); + case MAP: + return new Maps.Putter(receiver, v); + } + throw new AssertionError(); + } + + protected String toString(ColumnSpecification column) + { + return String.format("%s = %s + %s", column.name, column.name, value); + } + + public boolean isCompatibleWith(RawUpdate other) + { + return !(other instanceof SetValue); + } + } + + public static class Substraction implements RawUpdate + { + private final Term.Raw value; + + public Substraction(Term.Raw value) + { + this.value = value; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof CollectionType)) + { + if (!(receiver.type instanceof CounterColumnType)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver.name)); + return new Constants.Substracter(receiver, value.prepare(keyspace, receiver)); + } + else if (!(receiver.type.isMultiCell())) + throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen collection column %s", toString(receiver), receiver.name)); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: + return new Lists.Discarder(receiver, value.prepare(keyspace, receiver)); + case SET: + return new Sets.Discarder(receiver, value.prepare(keyspace, receiver)); + case MAP: + // The value for a map subtraction is actually a set + ColumnSpecification vr = new ColumnSpecification(receiver.ksName, + receiver.cfName, + receiver.name, + SetType.getInstance(((MapType)receiver.type).getKeysType(), false)); + return new Sets.Discarder(receiver, value.prepare(keyspace, vr)); + } + throw new AssertionError(); + } + + protected String toString(ColumnSpecification column) + { + return String.format("%s = %s - %s", column.name, column.name, value); + } + + public boolean isCompatibleWith(RawUpdate other) + { + return !(other instanceof SetValue); + } + } + + public static class Prepend implements RawUpdate + { + private final Term.Raw value; + + public Prepend(Term.Raw value) + { + this.value = value; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + Term v = value.prepare(keyspace, receiver); + + if (!(receiver.type instanceof ListType)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non list column %s", toString(receiver), receiver.name)); + else if (!(receiver.type.isMultiCell())) + throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen list column %s", toString(receiver), receiver.name)); + + return new Lists.Prepender(receiver, v); + } + + protected String toString(ColumnSpecification column) + { + return String.format("%s = %s - %s", column.name, value, column.name); + } + + public boolean isCompatibleWith(RawUpdate other) + { + return !(other instanceof SetValue); + } + } + + public static class ColumnDeletion implements RawDeletion + { + private final ColumnIdentifier.Raw id; + + public ColumnDeletion(ColumnIdentifier.Raw id) + { + this.id = id; + } + + public ColumnIdentifier.Raw affectedColumn() + { + return id; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + // No validation, deleting a column is always "well typed" + return new Constants.Deleter(receiver); + } + } + + public static class ElementDeletion implements RawDeletion + { + private final ColumnIdentifier.Raw id; + private final Term.Raw element; + + public ElementDeletion(ColumnIdentifier.Raw id, Term.Raw element) + { + this.id = id; + this.element = element; + } + + public ColumnIdentifier.Raw affectedColumn() + { + return id; + } + + public Operation prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException + { + if (!(receiver.type.isCollection())) + throw new InvalidRequestException(String.format("Invalid deletion operation for non collection column %s", receiver.name)); + else if (!(receiver.type.isMultiCell())) + throw new InvalidRequestException(String.format("Invalid deletion operation for frozen collection column %s", receiver.name)); + + switch (((CollectionType)receiver.type).kind) + { + case LIST: + Term idx = element.prepare(keyspace, Lists.indexSpecOf(receiver)); + return new Lists.DiscarderByIndex(receiver, idx); + case SET: + Term elt = element.prepare(keyspace, Sets.valueSpecOf(receiver)); + return new Sets.Discarder(receiver, elt); + case MAP: + Term key = element.prepare(keyspace, Maps.keySpecOf(receiver)); + return new Maps.DiscarderByKey(receiver, key); + } + throw new AssertionError(); + } + } +} diff --git a/cql3/Operator.java b/cql3/Operator.java new file mode 100644 index 0000000000..86bcbd38e8 --- /dev/null +++ b/cql3/Operator.java @@ -0,0 +1,170 @@ +/* + * 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.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public enum Operator +{ + EQ(0) + { + @Override + public String toString() + { + return "="; + } + }, + LT(4) + { + @Override + public String toString() + { + return "<"; + } + + @Override + public Operator reverse() + { + return GT; + } + }, + LTE(3) + { + @Override + public String toString() + { + return "<="; + } + + @Override + public Operator reverse() + { + return GTE; + } + }, + GTE(1) + { + @Override + public String toString() + { + return ">="; + } + + @Override + public Operator reverse() + { + return LTE; + } + }, + GT(2) + { + @Override + public String toString() + { + return ">"; + } + + @Override + public Operator reverse() + { + return LT; + } + }, + IN(7) + { + }, + CONTAINS(5) + { + }, + CONTAINS_KEY(6) + { + @Override + public String toString() + { + return "CONTAINS KEY"; + } + }, + NEQ(8) + { + @Override + public String toString() + { + return "!="; + } + }; + + /** + * The binary representation of this Enum value. + */ + private final int b; + + /** + * Creates a new Operator with the specified binary representation. + * @param b the binary representation of this Enum value + */ + private Operator(int b) + { + this.b = b; + } + + /** + * Write the serialized version of this Operator to the specified output. + * + * @param output the output to write to + * @throws IOException if an I/O problem occurs while writing to the specified output + */ + public void writeTo(DataOutput output) throws IOException + { + output.writeInt(b); + } + + /** + * Deserializes a Operator instance from the specified input. + * + * @param input the input to read from + * @return the Operator instance deserialized + * @throws IOException if a problem occurs while deserializing the Type instance. + */ + public static Operator readFrom(DataInput input) throws IOException + { + int b = input.readInt(); + for (Operator operator : values()) + if (operator.b == b) + return operator; + + throw new IOException(String.format("Cannot resolve Relation.Type from binary representation: %s", b)); + } + + @Override + public String toString() + { + return this.name(); + } + + /** + * Returns the reverse operator if this one. + * + * @return the reverse operator of this one. + */ + public Operator reverse() + { + return this; + } +} diff --git a/cql3/QueryHandler.java b/cql3/QueryHandler.java new file mode 100644 index 0000000000..d42d90ea99 --- /dev/null +++ b/cql3/QueryHandler.java @@ -0,0 +1,36 @@ +/* + * 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 org.apache.cassandra.cql3.statements.BatchStatement; +import org.apache.cassandra.cql3.statements.ParsedStatement; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.MD5Digest; + +public interface QueryHandler +{ + public ResultMessage process(String query, QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException; + public ResultMessage.Prepared prepare(String query, QueryState state) throws RequestValidationException; + public ParsedStatement.Prepared getPrepared(MD5Digest id); + public ParsedStatement.Prepared getPreparedForThrift(Integer id); + public ResultMessage processPrepared(CQLStatement statement, QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException; + public ResultMessage processBatch(BatchStatement statement, QueryState state, BatchQueryOptions options) throws RequestExecutionException, RequestValidationException; +} diff --git a/cql3/QueryOptions.java b/cql3/QueryOptions.java new file mode 100644 index 0000000000..b2569e74e8 --- /dev/null +++ b/cql3/QueryOptions.java @@ -0,0 +1,410 @@ +/* + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; + +import io.netty.buffer.ByteBuf; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.pager.PagingState; +import org.apache.cassandra.transport.CBCodec; +import org.apache.cassandra.transport.CBUtil; +import org.apache.cassandra.transport.ProtocolException; +import org.apache.cassandra.utils.Pair; + +/** + * Options for a query. + */ +public abstract class QueryOptions +{ + public static final QueryOptions DEFAULT = new DefaultQueryOptions(ConsistencyLevel.ONE, + Collections.emptyList(), + false, + SpecificOptions.DEFAULT, + 3); + + public static final CBCodec codec = new Codec(); + + public static QueryOptions fromProtocolV1(ConsistencyLevel consistency, List values) + { + return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 1); + } + + public static QueryOptions fromProtocolV2(ConsistencyLevel consistency, List values) + { + return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 2); + } + + public static QueryOptions forInternalCalls(ConsistencyLevel consistency, List values) + { + return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 3); + } + + public static QueryOptions forInternalCalls(List values) + { + return new DefaultQueryOptions(ConsistencyLevel.ONE, values, false, SpecificOptions.DEFAULT, 3); + } + + public static QueryOptions fromPreV3Batch(ConsistencyLevel consistency) + { + return new DefaultQueryOptions(consistency, Collections.emptyList(), false, SpecificOptions.DEFAULT, 2); + } + + public static QueryOptions create(ConsistencyLevel consistency, List values, boolean skipMetadata, int pageSize, PagingState pagingState, ConsistencyLevel serialConsistency) + { + return new DefaultQueryOptions(consistency, values, skipMetadata, new SpecificOptions(pageSize, pagingState, serialConsistency, -1L), 0); + } + + public abstract ConsistencyLevel getConsistency(); + public abstract List getValues(); + public abstract boolean skipMetadata(); + + /** The pageSize for this query. Will be <= 0 if not relevant for the query. */ + public int getPageSize() + { + return getSpecificOptions().pageSize; + } + + /** The paging state for this query, or null if not relevant. */ + public PagingState getPagingState() + { + return getSpecificOptions().state; + } + + /** Serial consistency for conditional updates. */ + public ConsistencyLevel getSerialConsistency() + { + return getSpecificOptions().serialConsistency; + } + + public long getTimestamp(QueryState state) + { + long tstamp = getSpecificOptions().timestamp; + return tstamp != Long.MIN_VALUE ? tstamp : state.getTimestamp(); + } + + /** + * The protocol version for the query. Will be 3 if the object don't come from + * a native protocol request (i.e. it's been allocated locally or by CQL-over-thrift). + */ + public abstract int getProtocolVersion(); + + // Mainly for the sake of BatchQueryOptions + abstract SpecificOptions getSpecificOptions(); + + public QueryOptions prepare(List specs) + { + return this; + } + + static class DefaultQueryOptions extends QueryOptions + { + private final ConsistencyLevel consistency; + private final List values; + private final boolean skipMetadata; + + private final SpecificOptions options; + + private final transient int protocolVersion; + + DefaultQueryOptions(ConsistencyLevel consistency, List values, boolean skipMetadata, SpecificOptions options, int protocolVersion) + { + this.consistency = consistency; + this.values = values; + this.skipMetadata = skipMetadata; + this.options = options; + this.protocolVersion = protocolVersion; + } + + public ConsistencyLevel getConsistency() + { + return consistency; + } + + public List getValues() + { + return values; + } + + public boolean skipMetadata() + { + return skipMetadata; + } + + public int getProtocolVersion() + { + return protocolVersion; + } + + SpecificOptions getSpecificOptions() + { + return options; + } + } + + static abstract class QueryOptionsWrapper extends QueryOptions + { + protected final QueryOptions wrapped; + + QueryOptionsWrapper(QueryOptions wrapped) + { + this.wrapped = wrapped; + } + + public ConsistencyLevel getConsistency() + { + return wrapped.getConsistency(); + } + + public boolean skipMetadata() + { + return wrapped.skipMetadata(); + } + + public int getProtocolVersion() + { + return wrapped.getProtocolVersion(); + } + + SpecificOptions getSpecificOptions() + { + return wrapped.getSpecificOptions(); + } + + @Override + public QueryOptions prepare(List specs) + { + wrapped.prepare(specs); + return this; + } + } + + static class OptionsWithNames extends QueryOptionsWrapper + { + private final List names; + private List orderedValues; + + OptionsWithNames(DefaultQueryOptions wrapped, List names) + { + super(wrapped); + this.names = names; + } + + @Override + public QueryOptions prepare(List specs) + { + super.prepare(specs); + + orderedValues = new ArrayList(specs.size()); + for (int i = 0; i < specs.size(); i++) + { + String name = specs.get(i).name.toString(); + for (int j = 0; j < names.size(); j++) + { + if (name.equals(names.get(j))) + { + orderedValues.add(wrapped.getValues().get(j)); + break; + } + } + } + return this; + } + + public List getValues() + { + assert orderedValues != null; // We should have called prepare first! + return orderedValues; + } + } + + // Options that are likely to not be present in most queries + static class SpecificOptions + { + private static final SpecificOptions DEFAULT = new SpecificOptions(-1, null, null, Long.MIN_VALUE); + + private final int pageSize; + private final PagingState state; + private final ConsistencyLevel serialConsistency; + private final long timestamp; + + private SpecificOptions(int pageSize, PagingState state, ConsistencyLevel serialConsistency, long timestamp) + { + this.pageSize = pageSize; + this.state = state; + this.serialConsistency = serialConsistency == null ? ConsistencyLevel.SERIAL : serialConsistency; + this.timestamp = timestamp; + } + } + + private static class Codec implements CBCodec + { + private static enum Flag + { + // The order of that enum matters!! + VALUES, + SKIP_METADATA, + PAGE_SIZE, + PAGING_STATE, + SERIAL_CONSISTENCY, + TIMESTAMP, + NAMES_FOR_VALUES; + + private static final Flag[] ALL_VALUES = values(); + + public static EnumSet deserialize(int flags) + { + EnumSet set = EnumSet.noneOf(Flag.class); + for (int n = 0; n < ALL_VALUES.length; n++) + { + if ((flags & (1 << n)) != 0) + set.add(ALL_VALUES[n]); + } + return set; + } + + public static int serialize(EnumSet flags) + { + int i = 0; + for (Flag flag : flags) + i |= 1 << flag.ordinal(); + return i; + } + } + + public QueryOptions decode(ByteBuf body, int version) + { + assert version >= 2; + + ConsistencyLevel consistency = CBUtil.readConsistencyLevel(body); + EnumSet flags = Flag.deserialize((int)body.readByte()); + + List values = Collections.emptyList(); + List names = null; + if (flags.contains(Flag.VALUES)) + { + if (flags.contains(Flag.NAMES_FOR_VALUES)) + { + Pair, List> namesAndValues = CBUtil.readNameAndValueList(body); + names = namesAndValues.left; + values = namesAndValues.right; + } + else + { + values = CBUtil.readValueList(body); + } + } + + boolean skipMetadata = flags.contains(Flag.SKIP_METADATA); + flags.remove(Flag.VALUES); + flags.remove(Flag.SKIP_METADATA); + + SpecificOptions options = SpecificOptions.DEFAULT; + if (!flags.isEmpty()) + { + int pageSize = flags.contains(Flag.PAGE_SIZE) ? body.readInt() : -1; + PagingState pagingState = flags.contains(Flag.PAGING_STATE) ? PagingState.deserialize(CBUtil.readValue(body)) : null; + ConsistencyLevel serialConsistency = flags.contains(Flag.SERIAL_CONSISTENCY) ? CBUtil.readConsistencyLevel(body) : ConsistencyLevel.SERIAL; + long timestamp = Long.MIN_VALUE; + if (flags.contains(Flag.TIMESTAMP)) + { + long ts = body.readLong(); + if (ts == Long.MIN_VALUE) + throw new ProtocolException(String.format("Out of bound timestamp, must be in [%d, %d] (got %d)", Long.MIN_VALUE + 1, Long.MAX_VALUE, ts)); + timestamp = ts; + } + + options = new SpecificOptions(pageSize, pagingState, serialConsistency, timestamp); + } + DefaultQueryOptions opts = new DefaultQueryOptions(consistency, values, skipMetadata, options, version); + return names == null ? opts : new OptionsWithNames(opts, names); + } + + public void encode(QueryOptions options, ByteBuf dest, int version) + { + assert version >= 2; + + CBUtil.writeConsistencyLevel(options.getConsistency(), dest); + + EnumSet flags = gatherFlags(options); + dest.writeByte((byte)Flag.serialize(flags)); + + if (flags.contains(Flag.VALUES)) + CBUtil.writeValueList(options.getValues(), dest); + if (flags.contains(Flag.PAGE_SIZE)) + dest.writeInt(options.getPageSize()); + if (flags.contains(Flag.PAGING_STATE)) + CBUtil.writeValue(options.getPagingState().serialize(), dest); + if (flags.contains(Flag.SERIAL_CONSISTENCY)) + CBUtil.writeConsistencyLevel(options.getSerialConsistency(), dest); + if (flags.contains(Flag.TIMESTAMP)) + dest.writeLong(options.getSpecificOptions().timestamp); + + // Note that we don't really have to bother with NAMES_FOR_VALUES server side, + // and in fact we never really encode QueryOptions, only decode them, so we + // don't bother. + } + + public int encodedSize(QueryOptions options, int version) + { + int size = 0; + + size += CBUtil.sizeOfConsistencyLevel(options.getConsistency()); + + EnumSet flags = gatherFlags(options); + size += 1; + + if (flags.contains(Flag.VALUES)) + size += CBUtil.sizeOfValueList(options.getValues()); + if (flags.contains(Flag.PAGE_SIZE)) + size += 4; + if (flags.contains(Flag.PAGING_STATE)) + size += CBUtil.sizeOfValue(options.getPagingState().serialize()); + if (flags.contains(Flag.SERIAL_CONSISTENCY)) + size += CBUtil.sizeOfConsistencyLevel(options.getSerialConsistency()); + if (flags.contains(Flag.TIMESTAMP)) + size += 8; + + return size; + } + + private EnumSet gatherFlags(QueryOptions options) + { + EnumSet flags = EnumSet.noneOf(Flag.class); + if (options.getValues().size() > 0) + flags.add(Flag.VALUES); + if (options.skipMetadata()) + flags.add(Flag.SKIP_METADATA); + if (options.getPageSize() >= 0) + flags.add(Flag.PAGE_SIZE); + if (options.getPagingState() != null) + flags.add(Flag.PAGING_STATE); + if (options.getSerialConsistency() != ConsistencyLevel.SERIAL) + flags.add(Flag.SERIAL_CONSISTENCY); + if (options.getSpecificOptions().timestamp != Long.MIN_VALUE) + flags.add(Flag.TIMESTAMP); + return flags; + } + } +} diff --git a/cql3/QueryProcessor.java b/cql3/QueryProcessor.java new file mode 100644 index 0000000000..ae099724bf --- /dev/null +++ b/cql3/QueryProcessor.java @@ -0,0 +1,662 @@ +/* + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.primitives.Ints; +import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; +import com.googlecode.concurrentlinkedhashmap.EntryWeigher; +import com.googlecode.concurrentlinkedhashmap.EvictionListener; + +import org.antlr.runtime.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.cql3.functions.*; + +import org.apache.cassandra.cql3.statements.*; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CType; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.CellNameType; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.SyntaxException; +import org.apache.cassandra.metrics.CQLMetrics; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.IMigrationListener; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.pager.QueryPager; +import org.apache.cassandra.service.pager.QueryPagers; +import org.apache.cassandra.thrift.ThriftClientState; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MD5Digest; +import org.apache.cassandra.utils.SemanticVersion; +import org.github.jamm.MemoryMeter; + +public class QueryProcessor implements QueryHandler +{ + public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.2.0"); + + public static final QueryProcessor instance = new QueryProcessor(); + + private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class); + private static final MemoryMeter meter = new MemoryMeter().withGuessing(MemoryMeter.Guess.FALLBACK_BEST).ignoreKnownSingletons(); + private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256; + + private static EntryWeigher cqlMemoryUsageWeigher = new EntryWeigher() + { + @Override + public int weightOf(MD5Digest key, ParsedStatement.Prepared value) + { + return Ints.checkedCast(measure(key) + measure(value.statement) + measure(value.boundNames)); + } + }; + + private static EntryWeigher thriftMemoryUsageWeigher = new EntryWeigher() + { + @Override + public int weightOf(Integer key, ParsedStatement.Prepared value) + { + return Ints.checkedCast(measure(key) + measure(value.statement) + measure(value.boundNames)); + } + }; + + private static final ConcurrentLinkedHashMap preparedStatements; + private static final ConcurrentLinkedHashMap thriftPreparedStatements; + + // A map for prepared statements used internally (which we don't want to mix with user statement, in particular we don't + // bother with expiration on those. + private static final ConcurrentMap internalStatements = new ConcurrentHashMap<>(); + + // Direct calls to processStatement do not increment the preparedStatementsExecuted/regularStatementsExecuted + // counters. Callers of processStatement are responsible for correctly notifying metrics + public static final CQLMetrics metrics = new CQLMetrics(); + + private static final AtomicInteger lastMinuteEvictionsCount = new AtomicInteger(0); + + static + { + preparedStatements = new ConcurrentLinkedHashMap.Builder() + .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) + .weigher(cqlMemoryUsageWeigher) + .listener(new EvictionListener() + { + public void onEviction(MD5Digest md5Digest, ParsedStatement.Prepared prepared) + { + metrics.preparedStatementsEvicted.inc(); + lastMinuteEvictionsCount.incrementAndGet(); + } + }).build(); + + thriftPreparedStatements = new ConcurrentLinkedHashMap.Builder() + .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) + .weigher(thriftMemoryUsageWeigher) + .listener(new EvictionListener() + { + public void onEviction(Integer integer, ParsedStatement.Prepared prepared) + { + metrics.preparedStatementsEvicted.inc(); + lastMinuteEvictionsCount.incrementAndGet(); + } + }) + .build(); + + ScheduledExecutors.scheduledTasks.scheduleAtFixedRate(new Runnable() + { + public void run() + { + long count = lastMinuteEvictionsCount.getAndSet(0); + if (count > 0) + logger.info("{} prepared statements discarded in the last minute because cache limit reached ({} bytes)", + count, + MAX_CACHE_PREPARED_MEMORY); + } + }, 1, 1, TimeUnit.MINUTES); + } + + public static int preparedStatementsCount() + { + return preparedStatements.size() + thriftPreparedStatements.size(); + } + + // Work around initialization dependency + private static enum InternalStateInstance + { + INSTANCE; + + private final QueryState queryState; + + InternalStateInstance() + { + ClientState state = ClientState.forInternalCalls(); + try + { + state.setKeyspace(SystemKeyspace.NAME); + } + catch (InvalidRequestException e) + { + throw new RuntimeException(); + } + this.queryState = new QueryState(state); + } + } + + private static QueryState internalQueryState() + { + return InternalStateInstance.INSTANCE.queryState; + } + + private QueryProcessor() + { + MigrationManager.instance.register(new MigrationSubscriber()); + } + + public ParsedStatement.Prepared getPrepared(MD5Digest id) + { + return preparedStatements.get(id); + } + + public ParsedStatement.Prepared getPreparedForThrift(Integer id) + { + return thriftPreparedStatements.get(id); + } + + public static void validateKey(ByteBuffer key) throws InvalidRequestException + { + if (key == null || key.remaining() == 0) + { + throw new InvalidRequestException("Key may not be empty"); + } + + // check that key can be handled by FBUtilities.writeShortByteArray + if (key.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + { + throw new InvalidRequestException("Key length of " + key.remaining() + + " is longer than maximum of " + FBUtilities.MAX_UNSIGNED_SHORT); + } + } + + public static void validateCellNames(Iterable cellNames, CellNameType type) throws InvalidRequestException + { + for (CellName name : cellNames) + validateCellName(name, type); + } + + public static void validateCellName(CellName name, CellNameType type) throws InvalidRequestException + { + validateComposite(name, type); + if (name.isEmpty()) + throw new InvalidRequestException("Invalid empty value for clustering column of COMPACT TABLE"); + } + + public static void validateComposite(Composite name, CType type) throws InvalidRequestException + { + long serializedSize = type.serializer().serializedSize(name, TypeSizes.NATIVE); + if (serializedSize > Cell.MAX_NAME_LENGTH) + throw new InvalidRequestException(String.format("The sum of all clustering columns is too long (%s > %s)", + serializedSize, + Cell.MAX_NAME_LENGTH)); + } + + public ResultMessage processStatement(CQLStatement statement, QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + logger.trace("Process {} @CL.{}", statement, options.getConsistency()); + ClientState clientState = queryState.getClientState(); + statement.checkAccess(clientState); + statement.validate(clientState); + + ResultMessage result = statement.execute(queryState, options); + return result == null ? new ResultMessage.Void() : result; + } + + public static ResultMessage process(String queryString, ConsistencyLevel cl, QueryState queryState) + throws RequestExecutionException, RequestValidationException + { + return instance.process(queryString, queryState, QueryOptions.forInternalCalls(cl, Collections.emptyList())); + } + + public ResultMessage process(String queryString, QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + ParsedStatement.Prepared p = getStatement(queryString, queryState.getClientState()); + options.prepare(p.boundNames); + CQLStatement prepared = p.statement; + if (prepared.getBoundTerms() != options.getValues().size()) + throw new InvalidRequestException("Invalid amount of bind variables"); + + if (!queryState.getClientState().isInternal) + metrics.regularStatementsExecuted.inc(); + + return processStatement(prepared, queryState, options); + } + + public static ParsedStatement.Prepared parseStatement(String queryStr, QueryState queryState) throws RequestValidationException + { + return getStatement(queryStr, queryState.getClientState()); + } + + public static UntypedResultSet process(String query, ConsistencyLevel cl) throws RequestExecutionException + { + try + { + ResultMessage result = instance.process(query, QueryState.forInternalCalls(), QueryOptions.forInternalCalls(cl, Collections.emptyList())); + if (result instanceof ResultMessage.Rows) + return UntypedResultSet.create(((ResultMessage.Rows)result).result); + else + return null; + } + catch (RequestValidationException e) + { + throw new RuntimeException(e); + } + } + + private static QueryOptions makeInternalOptions(ParsedStatement.Prepared prepared, Object[] values) + { + if (prepared.boundNames.size() != values.length) + throw new IllegalArgumentException(String.format("Invalid number of values. Expecting %d but got %d", prepared.boundNames.size(), values.length)); + + List boundValues = new ArrayList(values.length); + for (int i = 0; i < values.length; i++) + { + Object value = values[i]; + AbstractType type = prepared.boundNames.get(i).type; + boundValues.add(value instanceof ByteBuffer || value == null ? (ByteBuffer)value : type.decompose(value)); + } + return QueryOptions.forInternalCalls(boundValues); + } + + private static ParsedStatement.Prepared prepareInternal(String query) throws RequestValidationException + { + ParsedStatement.Prepared prepared = internalStatements.get(query); + if (prepared != null) + return prepared; + + // Note: if 2 threads prepare the same query, we'll live so don't bother synchronizing + prepared = parseStatement(query, internalQueryState()); + prepared.statement.validate(internalQueryState().getClientState()); + internalStatements.putIfAbsent(query, prepared); + return prepared; + } + + public static UntypedResultSet executeInternal(String query, Object... values) + { + try + { + ParsedStatement.Prepared prepared = prepareInternal(query); + ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values)); + if (result instanceof ResultMessage.Rows) + return UntypedResultSet.create(((ResultMessage.Rows)result).result); + else + return null; + } + catch (RequestExecutionException e) + { + throw new RuntimeException(e); + } + catch (RequestValidationException e) + { + throw new RuntimeException("Error validating " + query, e); + } + } + + public static UntypedResultSet executeInternalWithPaging(String query, int pageSize, Object... values) + { + try + { + ParsedStatement.Prepared prepared = prepareInternal(query); + if (!(prepared.statement instanceof SelectStatement)) + throw new IllegalArgumentException("Only SELECTs can be paged"); + + SelectStatement select = (SelectStatement)prepared.statement; + QueryPager pager = QueryPagers.localPager(select.getPageableCommand(makeInternalOptions(prepared, values))); + return UntypedResultSet.create(select, pager, pageSize); + } + catch (RequestValidationException e) + { + throw new RuntimeException("Error validating query" + e); + } + } + + /** + * Same than executeInternal, but to use for queries we know are only executed once so that the + * created statement object is not cached. + */ + public static UntypedResultSet executeOnceInternal(String query, Object... values) + { + try + { + ParsedStatement.Prepared prepared = parseStatement(query, internalQueryState()); + prepared.statement.validate(internalQueryState().getClientState()); + ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values)); + if (result instanceof ResultMessage.Rows) + return UntypedResultSet.create(((ResultMessage.Rows)result).result); + else + return null; + } + catch (RequestExecutionException e) + { + throw new RuntimeException(e); + } + catch (RequestValidationException e) + { + throw new RuntimeException("Error validating query " + query, e); + } + } + + public static UntypedResultSet resultify(String query, Row row) + { + return resultify(query, Collections.singletonList(row)); + } + + public static UntypedResultSet resultify(String query, List rows) + { + try + { + SelectStatement ss = (SelectStatement) getStatement(query, null).statement; + ResultSet cqlRows = ss.process(rows); + return UntypedResultSet.create(cqlRows); + } + catch (RequestValidationException e) + { + throw new AssertionError(e); + } + } + + public ResultMessage.Prepared prepare(String queryString, QueryState queryState) + throws RequestValidationException + { + ClientState cState = queryState.getClientState(); + return prepare(queryString, cState, cState instanceof ThriftClientState); + } + + public static ResultMessage.Prepared prepare(String queryString, ClientState clientState, boolean forThrift) + throws RequestValidationException + { + ResultMessage.Prepared existing = getStoredPreparedStatement(queryString, clientState.getRawKeyspace(), forThrift); + if (existing != null) + return existing; + + ParsedStatement.Prepared prepared = getStatement(queryString, clientState); + int boundTerms = prepared.statement.getBoundTerms(); + if (boundTerms > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("Too many markers(?). %d markers exceed the allowed maximum of %d", boundTerms, FBUtilities.MAX_UNSIGNED_SHORT)); + assert boundTerms == prepared.boundNames.size(); + + return storePreparedStatement(queryString, clientState.getRawKeyspace(), prepared, forThrift); + } + + private static MD5Digest computeId(String queryString, String keyspace) + { + String toHash = keyspace == null ? queryString : keyspace + queryString; + return MD5Digest.compute(toHash); + } + + private static Integer computeThriftId(String queryString, String keyspace) + { + String toHash = keyspace == null ? queryString : keyspace + queryString; + return toHash.hashCode(); + } + + private static ResultMessage.Prepared getStoredPreparedStatement(String queryString, String keyspace, boolean forThrift) + throws InvalidRequestException + { + if (forThrift) + { + Integer thriftStatementId = computeThriftId(queryString, keyspace); + ParsedStatement.Prepared existing = thriftPreparedStatements.get(thriftStatementId); + return existing == null ? null : ResultMessage.Prepared.forThrift(thriftStatementId, existing.boundNames); + } + else + { + MD5Digest statementId = computeId(queryString, keyspace); + ParsedStatement.Prepared existing = preparedStatements.get(statementId); + return existing == null ? null : new ResultMessage.Prepared(statementId, existing); + } + } + + private static ResultMessage.Prepared storePreparedStatement(String queryString, String keyspace, ParsedStatement.Prepared prepared, boolean forThrift) + throws InvalidRequestException + { + // Concatenate the current keyspace so we don't mix prepared statements between keyspace (#5352). + // (if the keyspace is null, queryString has to have a fully-qualified keyspace so it's fine. + long statementSize = measure(prepared.statement); + // don't execute the statement if it's bigger than the allowed threshold + if (statementSize > MAX_CACHE_PREPARED_MEMORY) + throw new InvalidRequestException(String.format("Prepared statement of size %d bytes is larger than allowed maximum of %d bytes.", + statementSize, + MAX_CACHE_PREPARED_MEMORY)); + if (forThrift) + { + Integer statementId = computeThriftId(queryString, keyspace); + thriftPreparedStatements.put(statementId, prepared); + return ResultMessage.Prepared.forThrift(statementId, prepared.boundNames); + } + else + { + MD5Digest statementId = computeId(queryString, keyspace); + preparedStatements.put(statementId, prepared); + return new ResultMessage.Prepared(statementId, prepared); + } + } + + public ResultMessage processPrepared(CQLStatement statement, QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + List variables = options.getValues(); + // Check to see if there are any bound variables to verify + if (!(variables.isEmpty() && (statement.getBoundTerms() == 0))) + { + if (variables.size() != statement.getBoundTerms()) + throw new InvalidRequestException(String.format("there were %d markers(?) in CQL but %d bound variables", + statement.getBoundTerms(), + variables.size())); + + // at this point there is a match in count between markers and variables that is non-zero + + if (logger.isTraceEnabled()) + for (int i = 0; i < variables.size(); i++) + logger.trace("[{}] '{}'", i+1, variables.get(i)); + } + + metrics.preparedStatementsExecuted.inc(); + return processStatement(statement, queryState, options); + } + + public ResultMessage processBatch(BatchStatement batch, QueryState queryState, BatchQueryOptions options) + throws RequestExecutionException, RequestValidationException + { + ClientState clientState = queryState.getClientState(); + batch.checkAccess(clientState); + batch.validate(); + batch.validate(clientState); + return batch.execute(queryState, options); + } + + public static ParsedStatement.Prepared getStatement(String queryStr, ClientState clientState) + throws RequestValidationException + { + Tracing.trace("Parsing {}", queryStr); + ParsedStatement statement = parseStatement(queryStr); + + // Set keyspace for statement that require login + if (statement instanceof CFStatement) + ((CFStatement)statement).prepareKeyspace(clientState); + + Tracing.trace("Preparing statement"); + return statement.prepare(); + } + + public static ParsedStatement parseStatement(String queryStr) throws SyntaxException + { + try + { + // Lexer and parser + ErrorCollector errorCollector = new ErrorCollector(queryStr); + CharStream stream = new ANTLRStringStream(queryStr); + CqlLexer lexer = new CqlLexer(stream); + lexer.addErrorListener(errorCollector); + + TokenStream tokenStream = new CommonTokenStream(lexer); + CqlParser parser = new CqlParser(tokenStream); + parser.addErrorListener(errorCollector); + + // Parse the query string to a statement instance + ParsedStatement statement = parser.query(); + + // The errorCollector has queue up any errors that the lexer and parser may have encountered + // along the way, if necessary, we turn the last error into exceptions here. + errorCollector.throwFirstSyntaxError(); + + return statement; + } + catch (RuntimeException re) + { + logger.error(String.format("The statement: [%s] could not be parsed.", queryStr), re); + throw new SyntaxException(String.format("Failed parsing statement: [%s] reason: %s %s", + queryStr, + re.getClass().getSimpleName(), + re.getMessage())); + } + catch (RecognitionException e) + { + throw new SyntaxException("Invalid or malformed CQL query string: " + e.getMessage()); + } + } + + private static long measure(Object key) + { + return meter.measureDeep(key); + } + + private static class MigrationSubscriber implements IMigrationListener + { + private void removeInvalidPreparedStatements(String ksName, String cfName) + { + removeInvalidPreparedStatements(preparedStatements.values().iterator(), ksName, cfName); + removeInvalidPreparedStatements(thriftPreparedStatements.values().iterator(), ksName, cfName); + } + + private void removeInvalidPreparedStatements(Iterator iterator, String ksName, String cfName) + { + while (iterator.hasNext()) + { + if (shouldInvalidate(ksName, cfName, iterator.next().statement)) + iterator.remove(); + } + } + + private boolean shouldInvalidate(String ksName, String cfName, CQLStatement statement) + { + String statementKsName; + String statementCfName; + + if (statement instanceof ModificationStatement) + { + ModificationStatement modificationStatement = ((ModificationStatement) statement); + statementKsName = modificationStatement.keyspace(); + statementCfName = modificationStatement.columnFamily(); + } + else if (statement instanceof SelectStatement) + { + SelectStatement selectStatement = ((SelectStatement) statement); + statementKsName = selectStatement.keyspace(); + statementCfName = selectStatement.columnFamily(); + } + else + { + return false; + } + + return ksName.equals(statementKsName) && (cfName == null || cfName.equals(statementCfName)); + } + + public void onCreateKeyspace(String ksName) { } + public void onCreateColumnFamily(String ksName, String cfName) { } + public void onCreateUserType(String ksName, String typeName) { } + public void onCreateFunction(String ksName, String functionName) { + if (Functions.getOverloadCount(new FunctionName(ksName, functionName)) > 1) + { + // in case there are other overloads, we have to remove all overloads since argument type + // matching may change (due to type casting) + removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, functionName); + removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, functionName); + } + } + public void onCreateAggregate(String ksName, String aggregateName) { + if (Functions.getOverloadCount(new FunctionName(ksName, aggregateName)) > 1) + { + // in case there are other overloads, we have to remove all overloads since argument type + // matching may change (due to type casting) + removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, aggregateName); + removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, aggregateName); + } + } + + public void onUpdateKeyspace(String ksName) { } + public void onUpdateColumnFamily(String ksName, String cfName) { } + public void onUpdateUserType(String ksName, String typeName) { } + public void onUpdateFunction(String ksName, String functionName) { } + public void onUpdateAggregate(String ksName, String aggregateName) { } + + public void onDropKeyspace(String ksName) + { + removeInvalidPreparedStatements(ksName, null); + } + + public void onDropColumnFamily(String ksName, String cfName) + { + removeInvalidPreparedStatements(ksName, cfName); + } + + public void onDropUserType(String ksName, String typeName) { } + public void onDropFunction(String ksName, String functionName) { + removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, functionName); + removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, functionName); + } + public void onDropAggregate(String ksName, String aggregateName) + { + removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, aggregateName); + removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, aggregateName); + } + + private void removeInvalidPreparedStatementsForFunction(Iterator iterator, + String ksName, String functionName) + { + while (iterator.hasNext()) + if (iterator.next().statement.usesFunction(ksName, functionName)) + iterator.remove(); + } + } +} diff --git a/cql3/Relation.java b/cql3/Relation.java new file mode 100644 index 0000000000..1337096b90 --- /dev/null +++ b/cql3/Relation.java @@ -0,0 +1,249 @@ +/* + * 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.ArrayList; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.UnrecognizedEntityException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +public abstract class Relation { + + protected Operator relationType; + + public Operator operator() + { + return relationType; + } + + /** + * Checks if this relation apply to multiple columns. + * + * @return true if this relation apply to multiple columns, false otherwise. + */ + public boolean isMultiColumn() + { + return false; + } + + /** + * Checks if this relation is a token relation (e.g.
token(a) = token(1)
). + * + * @return true if this relation is a token relation, false otherwise. + */ + public boolean onToken() + { + return false; + } + + /** + * Checks if the operator of this relation is a CONTAINS. + * @return true if the operator of this relation is a CONTAINS, false + * otherwise. + */ + public final boolean isContains() + { + return relationType == Operator.CONTAINS; + } + + /** + * Checks if the operator of this relation is a CONTAINS_KEY. + * @return true if the operator of this relation is a CONTAINS_KEY, false + * otherwise. + */ + public final boolean isContainsKey() + { + return relationType == Operator.CONTAINS_KEY; + } + + /** + * Checks if the operator of this relation is a IN. + * @return true if the operator of this relation is a IN, false + * otherwise. + */ + public final boolean isIN() + { + return relationType == Operator.IN; + } + + /** + * Checks if the operator of this relation is a EQ. + * @return true if the operator of this relation is a EQ, false + * otherwise. + */ + public final boolean isEQ() + { + return relationType == Operator.EQ; + } + + /** + * Checks if the operator of this relation is a Slice (GT, GTE, LTE, LT). + * + * @return true if the operator of this relation is a Slice, false otherwise. + */ + public final boolean isSlice() + { + return relationType == Operator.GT + || relationType == Operator.GTE + || relationType == Operator.LTE + || relationType == Operator.LT; + } + + /** + * Converts this Relation into a Restriction. + * + * @param cfm the Column Family meta data + * @param boundNames the variables specification where to collect the bind variables + * @return the Restriction corresponding to this Relation + * @throws InvalidRequestException if this Relation is not valid + */ + public final Restriction toRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException + { + switch (relationType) + { + case EQ: return newEQRestriction(cfm, boundNames); + case LT: return newSliceRestriction(cfm, boundNames, Bound.END, false); + case LTE: return newSliceRestriction(cfm, boundNames, Bound.END, true); + case GTE: return newSliceRestriction(cfm, boundNames, Bound.START, true); + case GT: return newSliceRestriction(cfm, boundNames, Bound.START, false); + case IN: return newINRestriction(cfm, boundNames); + case CONTAINS: return newContainsRestriction(cfm, boundNames, false); + case CONTAINS_KEY: return newContainsRestriction(cfm, boundNames, true); + default: throw invalidRequest("Unsupported \"!=\" relation: %s", this); + } + } + + /** + * Creates a new EQ restriction instance. + * + * @param cfm the Column Family meta data + * @param boundNames the variables specification where to collect the bind variables + * @return a new EQ restriction instance. + * @throws InvalidRequestException if the relation cannot be converted into an EQ restriction. + */ + protected abstract Restriction newEQRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException; + + /** + * Creates a new IN restriction instance. + * + * @param cfm the Column Family meta data + * @param boundNames the variables specification where to collect the bind variables + * @return a new IN restriction instance + * @throws InvalidRequestException if the relation cannot be converted into an IN restriction. + */ + protected abstract Restriction newINRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException; + + /** + * Creates a new Slice restriction instance. + * + * @param cfm the Column Family meta data + * @param boundNames the variables specification where to collect the bind variables + * @param bound the slice bound + * @param inclusive true if the bound is included. + * @return a new slice restriction instance + * @throws InvalidRequestException if the Relation is not valid + */ + protected abstract Restriction newSliceRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + Bound bound, + boolean inclusive) throws InvalidRequestException; + + /** + * Creates a new Contains restriction instance. + * + * @param cfm the Column Family meta data + * @param boundNames the variables specification where to collect the bind variables + * @param isKey true if the restriction to create is a CONTAINS KEY + * @return a new Contains Restriction instance + * @throws InvalidRequestException if the Relation is not valid + */ + protected abstract Restriction newContainsRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + boolean isKey) throws InvalidRequestException; + + /** + * Converts the specified Raw into a Term. + * @param receivers the columns to which the values must be associated at + * @param raw the raw term to convert + * @param keyspace the keyspace name + * @param boundNames the variables specification where to collect the bind variables + * + * @return the Term corresponding to the specified Raw + * @throws InvalidRequestException if the Raw term is not valid + */ + protected abstract Term toTerm(List receivers, + Term.Raw raw, + String keyspace, + VariableSpecifications boundNames) + throws InvalidRequestException; + + /** + * Converts the specified Raw terms into a Terms. + * @param receivers the columns to which the values must be associated at + * @param raws the raw terms to convert + * @param keyspace the keyspace name + * @param boundNames the variables specification where to collect the bind variables + * + * @return the Terms corresponding to the specified Raw terms + * @throws InvalidRequestException if the Raw terms are not valid + */ + protected final List toTerms(List receivers, + List raws, + String keyspace, + VariableSpecifications boundNames) throws InvalidRequestException + { + if (raws == null) + return null; + + List terms = new ArrayList<>(); + for (int i = 0, m = raws.size(); i < m; i++) + terms.add(toTerm(receivers, raws.get(i), keyspace, boundNames)); + + return terms; + } + + /** + * Converts the specified entity into a column definition. + * + * @param cfm the column family meta data + * @param entity the entity to convert + * @return the column definition corresponding to the specified entity + * @throws InvalidRequestException if the entity cannot be recognized + */ + protected final ColumnDefinition toColumnDefinition(CFMetaData cfm, + ColumnIdentifier.Raw entity) throws InvalidRequestException + { + ColumnIdentifier identifier = entity.prepare(cfm); + ColumnDefinition def = cfm.getColumnDefinition(identifier); + + if (def == null) + throw new UnrecognizedEntityException(identifier, this); + + return def; + } +} diff --git a/cql3/ResultSet.java b/cql3/ResultSet.java new file mode 100644 index 0000000000..c8c38698fb --- /dev/null +++ b/cql3/ResultSet.java @@ -0,0 +1,462 @@ +/* + * 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.nio.ByteBuffer; +import java.util.*; + +import io.netty.buffer.ByteBuf; + +import org.apache.cassandra.transport.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.thrift.Column; +import org.apache.cassandra.thrift.CqlMetadata; +import org.apache.cassandra.thrift.CqlResult; +import org.apache.cassandra.thrift.CqlResultType; +import org.apache.cassandra.thrift.CqlRow; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.service.pager.PagingState; + +public class ResultSet +{ + public static final Codec codec = new Codec(); + private static final ColumnIdentifier COUNT_COLUMN = new ColumnIdentifier("count", false); + + public final Metadata metadata; + public final List> rows; + + public ResultSet(List metadata) + { + this(new Metadata(metadata), new ArrayList>()); + } + + public ResultSet(Metadata metadata, List> rows) + { + this.metadata = metadata; + this.rows = rows; + } + + public int size() + { + return rows.size(); + } + + public boolean isEmpty() + { + return size() == 0; + } + + public void addRow(List row) + { + assert row.size() == metadata.valueCount(); + rows.add(row); + } + + public void addColumnValue(ByteBuffer value) + { + if (rows.isEmpty() || lastRow().size() == metadata.valueCount()) + rows.add(new ArrayList(metadata.valueCount())); + + lastRow().add(value); + } + + private List lastRow() + { + return rows.get(rows.size() - 1); + } + + public void reverse() + { + Collections.reverse(rows); + } + + public void trim(int limit) + { + int toRemove = rows.size() - limit; + if (toRemove > 0) + { + for (int i = 0; i < toRemove; i++) + rows.remove(rows.size() - 1); + } + } + + public CqlResult toThriftResult() + { + assert metadata.names != null; + + String UTF8 = "UTF8Type"; + CqlMetadata schema = new CqlMetadata(new HashMap(), + new HashMap(), + // The 2 following ones shouldn't be needed in CQL3 + UTF8, UTF8); + + for (int i = 0; i < metadata.columnCount; i++) + { + ColumnSpecification spec = metadata.names.get(i); + ByteBuffer colName = ByteBufferUtil.bytes(spec.name.toString()); + schema.name_types.put(colName, UTF8); + AbstractType normalizedType = spec.type instanceof ReversedType ? ((ReversedType)spec.type).baseType : spec.type; + schema.value_types.put(colName, normalizedType.toString()); + + } + + List cqlRows = new ArrayList(rows.size()); + for (List row : rows) + { + List thriftCols = new ArrayList(metadata.columnCount); + for (int i = 0; i < metadata.columnCount; i++) + { + Column col = new Column(ByteBufferUtil.bytes(metadata.names.get(i).name.toString())); + col.setValue(row.get(i)); + thriftCols.add(col); + } + // The key of CqlRow shoudn't be needed in CQL3 + cqlRows.add(new CqlRow(ByteBufferUtil.EMPTY_BYTE_BUFFER, thriftCols)); + } + CqlResult res = new CqlResult(CqlResultType.ROWS); + res.setRows(cqlRows).setSchema(schema); + return res; + } + + @Override + public String toString() + { + try + { + StringBuilder sb = new StringBuilder(); + sb.append(metadata).append('\n'); + for (List row : rows) + { + for (int i = 0; i < row.size(); i++) + { + ByteBuffer v = row.get(i); + if (v == null) + { + sb.append(" | null"); + } + else + { + sb.append(" | "); + if (metadata.flags.contains(Flag.NO_METADATA)) + sb.append("0x").append(ByteBufferUtil.bytesToHex(v)); + else + sb.append(metadata.names.get(i).type.getString(v)); + } + } + sb.append('\n'); + } + sb.append("---"); + return sb.toString(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + + public static class Codec implements CBCodec + { + /* + * Format: + * - metadata + * - rows count (4 bytes) + * - rows + */ + public ResultSet decode(ByteBuf body, int version) + { + Metadata m = Metadata.codec.decode(body, version); + int rowCount = body.readInt(); + ResultSet rs = new ResultSet(m, new ArrayList>(rowCount)); + + // rows + int totalValues = rowCount * m.columnCount; + for (int i = 0; i < totalValues; i++) + rs.addColumnValue(CBUtil.readValue(body)); + + return rs; + } + + public void encode(ResultSet rs, ByteBuf dest, int version) + { + Metadata.codec.encode(rs.metadata, dest, version); + dest.writeInt(rs.rows.size()); + for (List row : rs.rows) + { + // Note that we do only want to serialize only the first columnCount values, even if the row + // as more: see comment on Metadata.names field. + for (int i = 0; i < rs.metadata.columnCount; i++) + CBUtil.writeValue(row.get(i), dest); + } + } + + public int encodedSize(ResultSet rs, int version) + { + int size = Metadata.codec.encodedSize(rs.metadata, version) + 4; + for (List row : rs.rows) + { + for (int i = 0; i < rs.metadata.columnCount; i++) + size += CBUtil.sizeOfValue(row.get(i)); + } + return size; + } + } + + public static class Metadata + { + public static final CBCodec codec = new Codec(); + + public static final Metadata EMPTY = new Metadata(EnumSet.of(Flag.NO_METADATA), null, 0, null); + + private final EnumSet flags; + // Please note that columnCount can actually be smaller than names, even if names is not null. This is + // used to include columns in the resultSet that we need to do post-query re-orderings + // (SelectStatement.orderResults) but that shouldn't be sent to the user as they haven't been requested + // (CASSANDRA-4911). So the serialization code will exclude any columns in name whose index is >= columnCount. + public final List names; + private final int columnCount; + private PagingState pagingState; + + public Metadata(List names) + { + this(EnumSet.noneOf(Flag.class), names, names.size(), null); + if (!names.isEmpty() && allInSameCF()) + flags.add(Flag.GLOBAL_TABLES_SPEC); + } + + private Metadata(EnumSet flags, List names, int columnCount, PagingState pagingState) + { + this.flags = flags; + this.names = names; + this.columnCount = columnCount; + this.pagingState = pagingState; + } + + public Metadata copy() + { + return new Metadata(EnumSet.copyOf(flags), names, columnCount, pagingState); + } + + // The maximum number of values that the ResultSet can hold. This can be bigger than columnCount due to CASSANDRA-4911 + public int valueCount() + { + return names == null ? columnCount : names.size(); + } + + public void addNonSerializedColumn(ColumnSpecification name) + { + // See comment above. Because columnCount doesn't account the newly added name, it + // won't be serialized. + names.add(name); + } + + private boolean allInSameCF() + { + if (names == null) + return false; + + assert !names.isEmpty(); + + Iterator iter = names.iterator(); + ColumnSpecification first = iter.next(); + while (iter.hasNext()) + { + ColumnSpecification name = iter.next(); + if (!name.ksName.equals(first.ksName) || !name.cfName.equals(first.cfName)) + return false; + } + return true; + } + + public void setHasMorePages(PagingState pagingState) + { + if (pagingState == null) + return; + + flags.add(Flag.HAS_MORE_PAGES); + this.pagingState = pagingState; + } + + public void setSkipMetadata() + { + flags.add(Flag.NO_METADATA); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + + if (names == null) + { + sb.append("[").append(columnCount).append(" columns]"); + } + else + { + for (ColumnSpecification name : names) + { + sb.append("[").append(name.name); + sb.append("(").append(name.ksName).append(", ").append(name.cfName).append(")"); + sb.append(", ").append(name.type).append("]"); + } + } + if (flags.contains(Flag.HAS_MORE_PAGES)) + sb.append(" (to be continued)"); + return sb.toString(); + } + + private static class Codec implements CBCodec + { + public Metadata decode(ByteBuf body, int version) + { + // flags & column count + int iflags = body.readInt(); + int columnCount = body.readInt(); + + EnumSet flags = Flag.deserialize(iflags); + + PagingState state = null; + if (flags.contains(Flag.HAS_MORE_PAGES)) + state = PagingState.deserialize(CBUtil.readValue(body)); + + if (flags.contains(Flag.NO_METADATA)) + return new Metadata(flags, null, columnCount, state); + + boolean globalTablesSpec = flags.contains(Flag.GLOBAL_TABLES_SPEC); + + String globalKsName = null; + String globalCfName = null; + if (globalTablesSpec) + { + globalKsName = CBUtil.readString(body); + globalCfName = CBUtil.readString(body); + } + + // metadata (names/types) + List names = new ArrayList(columnCount); + for (int i = 0; i < columnCount; i++) + { + String ksName = globalTablesSpec ? globalKsName : CBUtil.readString(body); + String cfName = globalTablesSpec ? globalCfName : CBUtil.readString(body); + ColumnIdentifier colName = new ColumnIdentifier(CBUtil.readString(body), true); + AbstractType type = DataType.toType(DataType.codec.decodeOne(body, version)); + names.add(new ColumnSpecification(ksName, cfName, colName, type)); + } + return new Metadata(flags, names, names.size(), state); + } + + public void encode(Metadata m, ByteBuf dest, int version) + { + boolean noMetadata = m.flags.contains(Flag.NO_METADATA); + boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC); + boolean hasMorePages = m.flags.contains(Flag.HAS_MORE_PAGES); + + assert version > 1 || (!m.flags.contains(Flag.HAS_MORE_PAGES) && !noMetadata): "version = " + version + ", flags = " + m.flags; + + dest.writeInt(Flag.serialize(m.flags)); + dest.writeInt(m.columnCount); + + if (hasMorePages) + CBUtil.writeValue(m.pagingState.serialize(), dest); + + if (!noMetadata) + { + if (globalTablesSpec) + { + CBUtil.writeString(m.names.get(0).ksName, dest); + CBUtil.writeString(m.names.get(0).cfName, dest); + } + + for (int i = 0; i < m.columnCount; i++) + { + ColumnSpecification name = m.names.get(i); + if (!globalTablesSpec) + { + CBUtil.writeString(name.ksName, dest); + CBUtil.writeString(name.cfName, dest); + } + CBUtil.writeString(name.name.toString(), dest); + DataType.codec.writeOne(DataType.fromType(name.type, version), dest, version); + } + } + } + + public int encodedSize(Metadata m, int version) + { + boolean noMetadata = m.flags.contains(Flag.NO_METADATA); + boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC); + boolean hasMorePages = m.flags.contains(Flag.HAS_MORE_PAGES); + + int size = 8; + if (hasMorePages) + size += CBUtil.sizeOfValue(m.pagingState.serialize()); + + if (!noMetadata) + { + if (globalTablesSpec) + { + size += CBUtil.sizeOfString(m.names.get(0).ksName); + size += CBUtil.sizeOfString(m.names.get(0).cfName); + } + + for (int i = 0; i < m.columnCount; i++) + { + ColumnSpecification name = m.names.get(i); + if (!globalTablesSpec) + { + size += CBUtil.sizeOfString(name.ksName); + size += CBUtil.sizeOfString(name.cfName); + } + size += CBUtil.sizeOfString(name.name.toString()); + size += DataType.codec.oneSerializedSize(DataType.fromType(name.type, version), version); + } + } + return size; + } + } + } + + public static enum Flag + { + // The order of that enum matters!! + GLOBAL_TABLES_SPEC, + HAS_MORE_PAGES, + NO_METADATA; + + public static EnumSet deserialize(int flags) + { + EnumSet set = EnumSet.noneOf(Flag.class); + Flag[] values = Flag.values(); + for (int n = 0; n < values.length; n++) + { + if ((flags & (1 << n)) != 0) + set.add(values[n]); + } + return set; + } + + public static int serialize(EnumSet flags) + { + int i = 0; + for (Flag flag : flags) + i |= 1 << flag.ordinal(); + return i; + } + } +} diff --git a/cql3/Sets.java b/cql3/Sets.java new file mode 100644 index 0000000000..d7de9f457e --- /dev/null +++ b/cql3/Sets.java @@ -0,0 +1,334 @@ +/* + * 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.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Joiner; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.ColumnFamily; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.CollectionSerializer; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Static helper methods and classes for sets. + */ +public abstract class Sets +{ + private Sets() {} + + public static ColumnSpecification valueSpecOf(ColumnSpecification column) + { + return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((SetType)column.type).getElementsType()); + } + + public static class Literal implements Term.Raw + { + private final List elements; + + public Literal(List elements) + { + this.elements = elements; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + validateAssignableTo(keyspace, receiver); + + // We've parsed empty maps as a set literal to break the ambiguity so + // handle that case now + if (receiver.type instanceof MapType && elements.isEmpty()) + return new Maps.Value(Collections.emptyMap()); + + ColumnSpecification valueSpec = Sets.valueSpecOf(receiver); + Set values = new HashSet(elements.size()); + boolean allTerminal = true; + for (Term.Raw rt : elements) + { + Term t = rt.prepare(keyspace, valueSpec); + + if (t.containsBindMarker()) + throw new InvalidRequestException(String.format("Invalid set literal for %s: bind variables are not supported inside collection literals", receiver.name)); + + if (t instanceof Term.NonTerminal) + allTerminal = false; + + values.add(t); + } + DelayedValue value = new DelayedValue(((SetType)receiver.type).getElementsType(), values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof SetType)) + { + // We've parsed empty maps as a set literal to break the ambiguity so + // handle that case now + if ((receiver.type instanceof MapType) && elements.isEmpty()) + return; + + throw new InvalidRequestException(String.format("Invalid set literal for %s of type %s", receiver.name, receiver.type.asCQL3Type())); + } + + ColumnSpecification valueSpec = Sets.valueSpecOf(receiver); + for (Term.Raw rt : elements) + { + if (!rt.testAssignment(keyspace, valueSpec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid set literal for %s: value %s is not of type %s", receiver.name, rt, valueSpec.type.asCQL3Type())); + } + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + if (!(receiver.type instanceof SetType)) + { + // We've parsed empty maps as a set literal to break the ambiguity so handle that case now + if (receiver.type instanceof MapType && elements.isEmpty()) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + + // If there is no elements, we can't say it's an exact match (an empty set if fundamentally polymorphic). + if (elements.isEmpty()) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + + ColumnSpecification valueSpec = Sets.valueSpecOf(receiver); + return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements); + } + + @Override + public String toString() + { + return "{" + Joiner.on(", ").join(elements) + "}"; + } + } + + public static class Value extends Term.Terminal implements Term.CollectionTerminal + { + public final SortedSet elements; + + public Value(SortedSet elements) + { + this.elements = elements; + } + + public static Value fromSerialized(ByteBuffer value, SetType type, int version) throws InvalidRequestException + { + try + { + // Collections have this small hack that validate cannot be called on a serialized object, + // but compose does the validation (so we're fine). + Set s = (Set)type.getSerializer().deserializeForNativeProtocol(value, version); + SortedSet elements = new TreeSet(type.getElementsType()); + for (Object element : s) + elements.add(type.getElementsType().decompose(element)); + return new Value(elements); + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public ByteBuffer get(QueryOptions options) + { + return getWithProtocolVersion(options.getProtocolVersion()); + } + + public ByteBuffer getWithProtocolVersion(int protocolVersion) + { + return CollectionSerializer.pack(new ArrayList<>(elements), elements.size(), protocolVersion); + } + + public boolean equals(SetType st, Value v) + { + if (elements.size() != v.elements.size()) + return false; + + Iterator thisIter = elements.iterator(); + Iterator thatIter = v.elements.iterator(); + AbstractType elementsType = st.getElementsType(); + while (thisIter.hasNext()) + if (elementsType.compare(thisIter.next(), thatIter.next()) != 0) + return false; + + return true; + } + } + + // See Lists.DelayedValue + public static class DelayedValue extends Term.NonTerminal + { + private final Comparator comparator; + private final Set elements; + + public DelayedValue(Comparator comparator, Set elements) + { + this.comparator = comparator; + this.elements = elements; + } + + public boolean containsBindMarker() + { + // False since we don't support them in collection + return false; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + SortedSet buffers = new TreeSet<>(comparator); + for (Term t : elements) + { + ByteBuffer bytes = t.bindAndGet(options); + + if (bytes == null) + throw new InvalidRequestException("null is not supported inside collections"); + + // We don't support value > 64K because the serialization format encode the length as an unsigned short. + if (bytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + throw new InvalidRequestException(String.format("Set value is too long. Set values are limited to %d bytes but %d bytes value provided", + FBUtilities.MAX_UNSIGNED_SHORT, + bytes.remaining())); + + buffers.add(bytes); + } + return new Value(buffers); + } + } + + public static class Marker extends AbstractMarker + { + protected Marker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + assert receiver.type instanceof SetType; + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer value = options.getValues().get(bindIndex); + return value == null ? null : Value.fromSerialized(value, (SetType)receiver.type, options.getProtocolVersion()); + } + } + + public static class Setter extends Operation + { + public Setter(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + if (column.type.isMultiCell()) + { + // delete + add + CellName name = cf.getComparator().create(prefix, column); + cf.addAtom(params.makeTombstoneForOverwrite(name.slice())); + } + Adder.doAdd(t, cf, prefix, column, params); + } + } + + public static class Adder extends Operation + { + public Adder(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to add items to a frozen set"; + + doAdd(t, cf, prefix, column, params); + } + + static void doAdd(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException + { + Term.Terminal value = t.bind(params.options); + Sets.Value setValue = (Sets.Value)value; + if (column.type.isMultiCell()) + { + if (value == null) + return; + + Set toAdd = setValue.elements; + for (ByteBuffer bb : toAdd) + { + CellName cellName = cf.getComparator().create(prefix, column, bb); + cf.addColumn(params.makeColumn(cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER)); + } + } + else + { + // for frozen sets, we're overwriting the whole cell + CellName cellName = cf.getComparator().create(prefix, column); + if (value == null) + cf.addAtom(params.makeTombstone(cellName)); + else + cf.addColumn(params.makeColumn(cellName, ((Value) value).getWithProtocolVersion(Server.CURRENT_VERSION))); + } + } + } + + // Note that this is reused for Map subtraction too (we subtract a set from a map) + public static class Discarder extends Operation + { + public Discarder(ColumnDefinition column, Term t) + { + super(column, t); + } + + public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException + { + assert column.type.isMultiCell() : "Attempted to remove items from a frozen set"; + + Term.Terminal value = t.bind(params.options); + if (value == null) + return; + + // This can be either a set or a single element + Set toDiscard = value instanceof Constants.Value + ? Collections.singleton(((Constants.Value)value).bytes) + : ((Sets.Value)value).elements; + + for (ByteBuffer bb : toDiscard) + { + cf.addColumn(params.makeTombstone(cf.getComparator().create(prefix, column, bb))); + } + } + } +} diff --git a/cql3/SingleColumnRelation.java b/cql3/SingleColumnRelation.java new file mode 100644 index 0000000000..3db1195c2b --- /dev/null +++ b/cql3/SingleColumnRelation.java @@ -0,0 +1,297 @@ +/* + * 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.Collections; +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.Term.Raw; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue; + +/** + * Relations encapsulate the relationship between an entity of some kind, and + * a value (term). For example, > "start" or "colname1" = "somevalue". + * + */ +public final class SingleColumnRelation extends Relation +{ + private final ColumnIdentifier.Raw entity; + private final Term.Raw mapKey; + private final Term.Raw value; + private final List inValues; + + private SingleColumnRelation(ColumnIdentifier.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List inValues) + { + this.entity = entity; + this.mapKey = mapKey; + this.relationType = type; + this.value = value; + this.inValues = inValues; + } + + /** + * Creates a new relation. + * + * @param entity the kind of relation this is; what the term is being compared to. + * @param mapKey the key into the entity identifying the value the term is being compared to. + * @param type the type that describes how this entity relates to the value. + * @param value the value being compared. + */ + public SingleColumnRelation(ColumnIdentifier.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value) + { + this(entity, mapKey, type, value, null); + } + + /** + * Creates a new relation. + * + * @param entity the kind of relation this is; what the term is being compared to. + * @param type the type that describes how this entity relates to the value. + * @param value the value being compared. + */ + public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value) + { + this(entity, null, type, value); + } + + public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List inValues) + { + return new SingleColumnRelation(entity, null, Operator.IN, null, inValues); + } + + public ColumnIdentifier.Raw getEntity() + { + return entity; + } + + public Term.Raw getMapKey() + { + return mapKey; + } + + @Override + protected Term toTerm(List receivers, + Raw raw, + String keyspace, + VariableSpecifications boundNames) + throws InvalidRequestException + { + assert receivers.size() == 1; + + Term term = raw.prepare(keyspace, receivers.get(0)); + term.collectMarkerSpecification(boundNames); + return term; + } + + public SingleColumnRelation withNonStrictOperator() + { + switch (relationType) + { + case GT: return new SingleColumnRelation(entity, Operator.GTE, value); + case LT: return new SingleColumnRelation(entity, Operator.LTE, value); + default: return this; + } + } + + @Override + public String toString() + { + String entityAsString = entity.toString(); + if (mapKey != null) + entityAsString = String.format("%s[%s]", entityAsString, mapKey); + + if (isIN()) + return String.format("%s IN %s", entityAsString, inValues); + + return String.format("%s %s %s", entityAsString, relationType, value); + } + + @Override + protected Restriction newEQRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException + { + ColumnDefinition columnDef = toColumnDefinition(cfm, entity); + if (mapKey == null) + { + Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames); + return new SingleColumnRestriction.EQ(columnDef, term); + } + List receivers = toReceivers(cfm, columnDef); + Term entryKey = toTerm(Collections.singletonList(receivers.get(0)), mapKey, cfm.ksName, boundNames); + Term entryValue = toTerm(Collections.singletonList(receivers.get(1)), value, cfm.ksName, boundNames); + return new SingleColumnRestriction.Contains(columnDef, entryKey, entryValue); + } + + @Override + protected Restriction newINRestriction(CFMetaData cfm, + VariableSpecifications boundNames) throws InvalidRequestException + { + ColumnDefinition columnDef = cfm.getColumnDefinition(getEntity().prepare(cfm)); + List receivers = toReceivers(cfm, columnDef); + List terms = toTerms(receivers, inValues, cfm.ksName, boundNames); + if (terms == null) + { + Term term = toTerm(receivers, value, cfm.ksName, boundNames); + return new SingleColumnRestriction.InWithMarker(columnDef, (Lists.Marker) term); + } + return new SingleColumnRestriction.InWithValues(columnDef, terms); + } + + @Override + protected Restriction newSliceRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + Bound bound, + boolean inclusive) throws InvalidRequestException + { + ColumnDefinition columnDef = toColumnDefinition(cfm, entity); + Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames); + return new SingleColumnRestriction.Slice(columnDef, bound, inclusive, term); + } + + @Override + protected Restriction newContainsRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + boolean isKey) throws InvalidRequestException + { + ColumnDefinition columnDef = toColumnDefinition(cfm, entity); + Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames); + return new SingleColumnRestriction.Contains(columnDef, term, isKey); + } + + /** + * Returns the receivers for this relation. + * + * @param cfm the Column Family meta data + * @param columnDef the column definition + * @return the receivers for the specified relation. + * @throws InvalidRequestException if the relation is invalid + */ + private List toReceivers(CFMetaData cfm, ColumnDefinition columnDef) throws InvalidRequestException + { + ColumnSpecification receiver = columnDef; + + checkFalse(columnDef.isCompactValue(), + "Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", + columnDef.name); + + if (isIN()) + { + // For partition keys we only support IN for the last name so far + checkFalse(columnDef.isPartitionKey() && !isLastPartitionKey(cfm, columnDef), + "Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)", + columnDef.name); + + // We only allow IN on the row key and the clustering key so far, never on non-PK columns, and this even if + // there's an index + // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that + // slide. + checkFalse(!columnDef.isPrimaryKeyColumn() && !canHaveOnlyOneValue(), + "IN predicates on non-primary-key columns (%s) is not yet supported", columnDef.name); + } + else if (isSlice()) + { + // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those + // are ordered by partitioner). + // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would + // probably require some special casing + // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we + // lift the limitation for 2ndary + // index with filtering, we'll need to handle it though. + checkFalse(columnDef.isPartitionKey(), "Only EQ and IN relation are supported on the partition key (unless you use the token() function)"); + } + + checkFalse(isContainsKey() && !(receiver.type instanceof MapType), "Cannot use CONTAINS KEY on non-map column %s", receiver.name); + + if (mapKey != null) + { + checkFalse(receiver.type instanceof ListType, "Indexes on list entries (%s[index] = value) are not currently supported.", receiver.name); + checkTrue(receiver.type instanceof MapType, "Column %s cannot be used as a map", receiver.name); + checkTrue(receiver.type.isMultiCell(), "Map-entry equality predicates on frozen map column %s are not supported", receiver.name); + checkTrue(isEQ(), "Only EQ relations are supported on map entries"); + } + + if (receiver.type.isCollection()) + { + // We don't support relations against entire collections (unless they're frozen), like "numbers = {1, 2, 3}" + checkFalse(receiver.type.isMultiCell() && !isLegalRelationForNonFrozenCollection(), + "Collection column '%s' (%s) cannot be restricted by a '%s' relation", + receiver.name, + receiver.type.asCQL3Type(), + operator()); + + if (isContainsKey() || isContains()) + { + receiver = makeCollectionReceiver(receiver, isContainsKey()); + } + else if (receiver.type.isMultiCell() && mapKey != null && isEQ()) + { + List receivers = new ArrayList<>(2); + receivers.add(makeCollectionReceiver(receiver, true)); + receivers.add(makeCollectionReceiver(receiver, false)); + return receivers; + } + } + + return Collections.singletonList(receiver); + } + + private ColumnSpecification makeCollectionReceiver(ColumnSpecification receiver, boolean forKey) + { + return ((CollectionType) receiver.type).makeCollectionReceiver(receiver, forKey); + } + + private boolean isLegalRelationForNonFrozenCollection() + { + return isContainsKey() || isContains() || isMapEntryEquality(); + } + + private boolean isMapEntryEquality() + { + return mapKey != null && isEQ(); + } + + /** + * Checks if the specified column is the last column of the partition key. + * + * @param cfm the column family meta data + * @param columnDef the column to check + * @return true if the specified column is the last column of the partition key, false + * otherwise. + */ + private static boolean isLastPartitionKey(CFMetaData cfm, ColumnDefinition columnDef) + { + return columnDef.position() == cfm.partitionKeyColumns().size() - 1; + } + + private boolean canHaveOnlyOneValue() + { + return isEQ() || (isIN() && inValues != null && inValues.size() == 1); + } +} diff --git a/cql3/Term.java b/cql3/Term.java new file mode 100644 index 0000000000..7e20df8717 --- /dev/null +++ b/cql3/Term.java @@ -0,0 +1,177 @@ +/* + * 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.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A CQL3 term, i.e. a column value with or without bind variables. + * + * A Term can be either terminal or non terminal. A term object is one that is typed and is obtained + * from a raw term (Term.Raw) by poviding the actual receiver to which the term is supposed to be a + * value of. + */ +public interface Term +{ + /** + * Collects the column specification for the bind variables in this Term. + * This is obviously a no-op if the term is Terminal. + * + * @param boundNames the variables specification where to collect the + * bind variables of this term in. + */ + public void collectMarkerSpecification(VariableSpecifications boundNames); + + /** + * Bind the values in this term to the values contained in {@code values}. + * This is obviously a no-op if the term is Terminal. + * + * @param options the values to bind markers to. + * @return the result of binding all the variables of this NonTerminal (or + * 'this' if the term is terminal). + */ + public Terminal bind(QueryOptions options) throws InvalidRequestException; + + /** + * A shorter for bind(values).get(). + * We expose it mainly because for constants it can avoids allocating a temporary + * object between the bind and the get (note that we still want to be able + * to separate bind and get for collections). + */ + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException; + + /** + * Whether or not that term contains at least one bind marker. + * + * Note that this is slightly different from being or not a NonTerminal, + * because calls to non pure functions will be NonTerminal (see #5616) + * even if they don't have bind markers. + */ + public abstract boolean containsBindMarker(); + + boolean usesFunction(String ksName, String functionName); + + /** + * A parsed, non prepared (thus untyped) term. + * + * This can be one of: + * - a constant + * - a collection literal + * - a function call + * - a marker + */ + public interface Raw extends AssignmentTestable + { + /** + * This method validates this RawTerm is valid for provided column + * specification and "prepare" this RawTerm, returning the resulting + * prepared Term. + * + * @param receiver the "column" this RawTerm is supposed to be a value of. Note + * that the ColumnSpecification may not correspond to a real column in the + * case this RawTerm describe a list index or a map key, etc... + * @return the prepared term. + */ + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException; + } + + public interface MultiColumnRaw extends Raw + { + public Term prepare(String keyspace, List receiver) throws InvalidRequestException; + } + + /** + * A terminal term, one that can be reduced to a byte buffer directly. + * + * This includes most terms that don't have a bind marker (an exception + * being delayed call for non pure function that are NonTerminal even + * if they don't have bind markers). + * + * This can be only one of: + * - a constant value + * - a collection value + * + * Note that a terminal term will always have been type checked, and thus + * consumer can (and should) assume so. + */ + public abstract class Terminal implements Term + { + public void collectMarkerSpecification(VariableSpecifications boundNames) {} + public Terminal bind(QueryOptions options) { return this; } + + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + // While some NonTerminal may not have bind markers, no Term can be Terminal + // with a bind marker + public boolean containsBindMarker() + { + return false; + } + + /** + * @return the serialized value of this terminal. + */ + public abstract ByteBuffer get(QueryOptions options); + + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + return get(options); + } + } + + public abstract class MultiItemTerminal extends Terminal + { + public abstract List getElements(); + } + + public interface CollectionTerminal + { + /** Gets the value of the collection when serialized with the given protocol version format */ + public ByteBuffer getWithProtocolVersion(int protocolVersion); + } + + /** + * A non terminal term, i.e. a term that can only be reduce to a byte buffer + * at execution time. + * + * We have the following type of NonTerminal: + * - marker for a constant value + * - marker for a collection value (list, set, map) + * - a function having bind marker + * - a non pure function (even if it doesn't have bind marker - see #5616) + */ + public abstract class NonTerminal implements Term + { + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + Terminal t = bind(options); + return t == null ? null : t.get(options); + } + } +} diff --git a/cql3/TokenRelation.java b/cql3/TokenRelation.java new file mode 100644 index 0000000000..d1bd265148 --- /dev/null +++ b/cql3/TokenRelation.java @@ -0,0 +1,164 @@ +/* + * 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.ArrayList; +import java.util.Collections; +import java.util.List; + +import com.google.common.base.Joiner; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.Term.Raw; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.cql3.restrictions.TokenRestriction; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsNoDuplicates; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsOnly; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue; +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +/** + * A relation using the token function. + * Examples: + *
    + *
  • SELECT ... WHERE token(a) > token(1)
  • + *
  • SELECT ... WHERE token(a, b) > token(1, 3)
  • + *
+ */ +public final class TokenRelation extends Relation +{ + private final List entities; + + private final Term.Raw value; + + public TokenRelation(List entities, Operator type, Term.Raw value) + { + this.entities = entities; + this.relationType = type; + this.value = value; + } + + @Override + public boolean onToken() + { + return true; + } + + @Override + protected Restriction newEQRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException + { + List columnDefs = getColumnDefinitions(cfm); + Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames); + return new TokenRestriction.EQ(columnDefs, term); + } + + @Override + protected Restriction newINRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException + { + throw invalidRequest("%s cannot be used with the token function", operator()); + } + + @Override + protected Restriction newSliceRestriction(CFMetaData cfm, + VariableSpecifications boundNames, + Bound bound, + boolean inclusive) throws InvalidRequestException + { + List columnDefs = getColumnDefinitions(cfm); + Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames); + return new TokenRestriction.Slice(columnDefs, bound, inclusive, term); + } + + @Override + protected Restriction newContainsRestriction(CFMetaData cfm, VariableSpecifications boundNames, boolean isKey) throws InvalidRequestException + { + throw invalidRequest("%s cannot be used with the token function", operator()); + } + + @Override + protected Term toTerm(List receivers, + Raw raw, + String keyspace, + VariableSpecifications boundNames) throws InvalidRequestException + { + Term term = raw.prepare(keyspace, receivers.get(0)); + term.collectMarkerSpecification(boundNames); + return term; + } + + @Override + public String toString() + { + return String.format("token(%s) %s %s", Tuples.tupleToString(entities), relationType, value); + } + + /** + * Returns the definition of the columns to which apply the token restriction. + * + * @param cfm the column family metadata + * @return the definition of the columns to which apply the token restriction. + * @throws InvalidRequestException if the entity cannot be resolved + */ + private List getColumnDefinitions(CFMetaData cfm) throws InvalidRequestException + { + List columnDefs = new ArrayList<>(); + for ( ColumnIdentifier.Raw raw : entities) + { + columnDefs.add(toColumnDefinition(cfm, raw)); + } + return columnDefs; + } + + /** + * Returns the receivers for this relation. + * + * @param cfm the Column Family meta data + * @param columnDefs the column definitions + * @return the receivers for the specified relation. + * @throws InvalidRequestException if the relation is invalid + */ + private static List toReceivers(CFMetaData cfm, + List columnDefs) + throws InvalidRequestException + { + + if (!columnDefs.equals(cfm.partitionKeyColumns())) + { + checkTrue(columnDefs.containsAll(cfm.partitionKeyColumns()), + "The token() function must be applied to all partition key components or none of them"); + + checkContainsNoDuplicates(columnDefs, "The token() function contains duplicate partition key components"); + + checkContainsOnly(columnDefs, cfm.partitionKeyColumns(), "The token() function must contains only partition key components"); + + throw invalidRequest("The token function arguments must be in the partition key order: %s", + Joiner.on(", ").join(ColumnDefinition.toIdentifiers(cfm.partitionKeyColumns()))); + } + + ColumnDefinition firstColumn = columnDefs.get(0); + return Collections.singletonList(new ColumnSpecification(firstColumn.ksName, + firstColumn.cfName, + new ColumnIdentifier("partition key token", true), + StorageService.getPartitioner().getTokenValidator())); + } +} diff --git a/cql3/Tuples.java b/cql3/Tuples.java new file mode 100644 index 0000000000..a66c534988 --- /dev/null +++ b/cql3/Tuples.java @@ -0,0 +1,413 @@ +/* + * 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.nio.ByteBuffer; +import java.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.Term.MultiColumnRaw; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.MarshalException; + +/** + * Static helper methods and classes for tuples. + */ +public class Tuples +{ + private static final Logger logger = LoggerFactory.getLogger(Tuples.class); + + private Tuples() {} + + public static ColumnSpecification componentSpecOf(ColumnSpecification column, int component) + { + return new ColumnSpecification(column.ksName, + column.cfName, + new ColumnIdentifier(String.format("%s[%d]", column.name, component), true), + ((TupleType)column.type).type(component)); + } + + /** + * A raw, literal tuple. When prepared, this will become a Tuples.Value or Tuples.DelayedValue, depending + * on whether the tuple holds NonTerminals. + */ + public static class Literal implements Term.MultiColumnRaw + { + private final List elements; + + public Literal(List elements) + { + this.elements = elements; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + validateAssignableTo(keyspace, receiver); + + List values = new ArrayList<>(elements.size()); + boolean allTerminal = true; + for (int i = 0; i < elements.size(); i++) + { + Term value = elements.get(i).prepare(keyspace, componentSpecOf(receiver, i)); + if (value instanceof Term.NonTerminal) + allTerminal = false; + + values.add(value); + } + DelayedValue value = new DelayedValue((TupleType)receiver.type, values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + public Term prepare(String keyspace, List receivers) throws InvalidRequestException + { + if (elements.size() != receivers.size()) + throw new InvalidRequestException(String.format("Expected %d elements in value tuple, but got %d: %s", receivers.size(), elements.size(), this)); + + List values = new ArrayList<>(elements.size()); + List> types = new ArrayList<>(elements.size()); + boolean allTerminal = true; + for (int i = 0; i < elements.size(); i++) + { + Term t = elements.get(i).prepare(keyspace, receivers.get(i)); + if (t instanceof Term.NonTerminal) + allTerminal = false; + + values.add(t); + types.add(receivers.get(i).type); + } + DelayedValue value = new DelayedValue(new TupleType(types), values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof TupleType)) + throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type())); + + TupleType tt = (TupleType)receiver.type; + for (int i = 0; i < elements.size(); i++) + { + if (i >= tt.size()) + throw new InvalidRequestException(String.format("Invalid tuple literal for %s: too many elements. Type %s expects %d but got %d", + receiver.name, tt.asCQL3Type(), tt.size(), elements.size())); + + Term.Raw value = elements.get(i); + ColumnSpecification spec = componentSpecOf(receiver, i); + if (!value.testAssignment(keyspace, spec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid tuple literal for %s: component %d is not of type %s", receiver.name, i, spec.type.asCQL3Type())); + } + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + try + { + validateAssignableTo(keyspace, receiver); + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + catch (InvalidRequestException e) + { + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + } + + @Override + public String toString() + { + return tupleToString(elements); + } + } + + /** + * A tuple of terminal values (e.g (123, 'abc')). + */ + public static class Value extends Term.MultiItemTerminal + { + public final ByteBuffer[] elements; + + public Value(ByteBuffer[] elements) + { + this.elements = elements; + } + + public static Value fromSerialized(ByteBuffer bytes, TupleType type) + { + return new Value(type.split(bytes)); + } + + public ByteBuffer get(QueryOptions options) + { + return TupleType.buildValue(elements); + } + + public List getElements() + { + return Arrays.asList(elements); + } + } + + /** + * Similar to Value, but contains at least one NonTerminal, such as a non-pure functions or bind marker. + */ + public static class DelayedValue extends Term.NonTerminal + { + public final TupleType type; + public final List elements; + + public DelayedValue(TupleType type, List elements) + { + this.type = type; + this.elements = elements; + } + + public boolean containsBindMarker() + { + for (Term term : elements) + if (term.containsBindMarker()) + return true; + + return false; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + for (Term term : elements) + term.collectMarkerSpecification(boundNames); + } + + private ByteBuffer[] bindInternal(QueryOptions options) throws InvalidRequestException + { + int version = options.getProtocolVersion(); + + ByteBuffer[] buffers = new ByteBuffer[elements.size()]; + for (int i = 0; i < elements.size(); i++) + { + buffers[i] = elements.get(i).bindAndGet(options); + // Inside tuples, we must force the serialization of collections to v3 whatever protocol + // version is in use since we're going to store directly that serialized value. + if (version < 3 && type.type(i).isCollection()) + buffers[i] = ((CollectionType)type.type(i)).getSerializer().reserializeToV3(buffers[i]); + } + return buffers; + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + return new Value(bindInternal(options)); + } + + @Override + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + // We don't "need" that override but it saves us the allocation of a Value object if used + return TupleType.buildValue(bindInternal(options)); + } + + @Override + public String toString() + { + return tupleToString(elements); + } + } + + /** + * A terminal value for a list of IN values that are tuples. For example: "SELECT ... WHERE (a, b, c) IN ?" + * This is similar to Lists.Value, but allows us to keep components of the tuples in the list separate. + */ + public static class InValue extends Term.Terminal + { + List> elements; + + public InValue(List> items) + { + this.elements = items; + } + + public static InValue fromSerialized(ByteBuffer value, ListType type, QueryOptions options) throws InvalidRequestException + { + try + { + // Collections have this small hack that validate cannot be called on a serialized object, + // but the deserialization does the validation (so we're fine). + List l = (List)type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion()); + + assert type.getElementsType() instanceof TupleType; + TupleType tupleType = (TupleType) type.getElementsType(); + + // type.split(bytes) + List> elements = new ArrayList<>(l.size()); + for (Object element : l) + elements.add(Arrays.asList(tupleType.split(type.getElementsType().decompose(element)))); + return new InValue(elements); + } + catch (MarshalException e) + { + throw new InvalidRequestException(e.getMessage()); + } + } + + public ByteBuffer get(QueryOptions options) + { + throw new UnsupportedOperationException(); + } + + public List> getSplitValues() + { + return elements; + } + } + + /** + * A raw placeholder for a tuple of values for different multiple columns, each of which may have a different type. + * For example, "SELECT ... WHERE (col1, col2) > ?". + */ + public static class Raw extends AbstractMarker.Raw implements Term.MultiColumnRaw + { + public Raw(int bindIndex) + { + super(bindIndex); + } + + private static ColumnSpecification makeReceiver(List receivers) + { + List> types = new ArrayList<>(receivers.size()); + StringBuilder inName = new StringBuilder("("); + for (int i = 0; i < receivers.size(); i++) + { + ColumnSpecification receiver = receivers.get(i); + inName.append(receiver.name); + if (i < receivers.size() - 1) + inName.append(","); + types.add(receiver.type); + } + inName.append(')'); + + ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true); + TupleType type = new TupleType(types); + return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, type); + } + + public AbstractMarker prepare(String keyspace, List receivers) throws InvalidRequestException + { + return new Tuples.Marker(bindIndex, makeReceiver(receivers)); + } + + @Override + public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) + { + throw new AssertionError("Tuples.Raw.prepare() requires a list of receivers"); + } + } + + /** + * A raw marker for an IN list of tuples, like "SELECT ... WHERE (a, b, c) IN ?" + */ + public static class INRaw extends AbstractMarker.Raw implements MultiColumnRaw + { + public INRaw(int bindIndex) + { + super(bindIndex); + } + + private static ColumnSpecification makeInReceiver(List receivers) throws InvalidRequestException + { + List> types = new ArrayList<>(receivers.size()); + StringBuilder inName = new StringBuilder("in("); + for (int i = 0; i < receivers.size(); i++) + { + ColumnSpecification receiver = receivers.get(i); + inName.append(receiver.name); + if (i < receivers.size() - 1) + inName.append(","); + + if (receiver.type.isCollection() && receiver.type.isMultiCell()) + throw new InvalidRequestException("Non-frozen collection columns do not support IN relations"); + + types.add(receiver.type); + } + inName.append(')'); + + ColumnIdentifier identifier = new ColumnIdentifier(inName.toString(), true); + TupleType type = new TupleType(types); + return new ColumnSpecification(receivers.get(0).ksName, receivers.get(0).cfName, identifier, ListType.getInstance(type, false)); + } + + public AbstractMarker prepare(String keyspace, List receivers) throws InvalidRequestException + { + return new InMarker(bindIndex, makeInReceiver(receivers)); + } + + @Override + public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) + { + throw new AssertionError("Tuples.INRaw.prepare() requires a list of receivers"); + } + } + + /** + * Represents a marker for a single tuple, like "SELECT ... WHERE (a, b, c) > ?" + */ + public static class Marker extends AbstractMarker + { + public Marker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + } + + public Value bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer value = options.getValues().get(bindIndex); + return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type); + } + } + + /** + * Represents a marker for a set of IN values that are tuples, like "SELECT ... WHERE (a, b, c) IN ?" + */ + public static class InMarker extends AbstractMarker + { + protected InMarker(int bindIndex, ColumnSpecification receiver) + { + super(bindIndex, receiver); + assert receiver.type instanceof ListType; + } + + public InValue bind(QueryOptions options) throws InvalidRequestException + { + ByteBuffer value = options.getValues().get(bindIndex); + return value == null ? null : InValue.fromSerialized(value, (ListType)receiver.type, options); + } + } + + public static String tupleToString(List items) + { + + StringBuilder sb = new StringBuilder("("); + for (int i = 0; i < items.size(); i++) + { + sb.append(items.get(i)); + if (i < items.size() - 1) + sb.append(", "); + } + sb.append(')'); + return sb.toString(); + } +} diff --git a/cql3/TypeCast.java b/cql3/TypeCast.java new file mode 100644 index 0000000000..10b040e586 --- /dev/null +++ b/cql3/TypeCast.java @@ -0,0 +1,73 @@ +/* + * 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 org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +public class TypeCast implements Term.Raw +{ + private final CQL3Type.Raw type; + private final Term.Raw term; + + public TypeCast(CQL3Type.Raw type, Term.Raw term) + { + this.type = type; + this.term = term; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!term.testAssignment(keyspace, castedSpecOf(keyspace, receiver)).isAssignable()) + throw new InvalidRequestException(String.format("Cannot cast value %s to type %s", term, type)); + + if (!testAssignment(keyspace, receiver).isAssignable()) + throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver.name, receiver.type.asCQL3Type())); + + return term.prepare(keyspace, receiver); + } + + private ColumnSpecification castedSpecOf(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + return new ColumnSpecification(receiver.ksName, receiver.cfName, new ColumnIdentifier(toString(), true), type.prepare(keyspace).getType()); + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + try + { + AbstractType castedType = type.prepare(keyspace).getType(); + if (receiver.type.equals(castedType)) + return AssignmentTestable.TestResult.EXACT_MATCH; + else if (receiver.type.isValueCompatibleWith(castedType)) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + else + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + catch (InvalidRequestException e) + { + throw new AssertionError(); + } + } + + @Override + public String toString() + { + return "(" + type + ")" + term; + } +} diff --git a/cql3/UTName.java b/cql3/UTName.java new file mode 100644 index 0000000000..c8567977bd --- /dev/null +++ b/cql3/UTName.java @@ -0,0 +1,63 @@ +/* + * 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.nio.ByteBuffer; + +public class UTName +{ + private String ksName; + private final ColumnIdentifier utName; + + public UTName(ColumnIdentifier ksName, ColumnIdentifier utName) + { + this.ksName = ksName == null ? null : ksName.toString(); + this.utName = utName; + } + + public boolean hasKeyspace() + { + return ksName != null; + } + + public void setKeyspace(String keyspace) + { + this.ksName = keyspace; + } + + public String getKeyspace() + { + return ksName; + } + + public ByteBuffer getUserTypeName() + { + return utName.bytes; + } + + public String getStringTypeName() + { + return utName.toString(); + } + + @Override + public String toString() + { + return (hasKeyspace() ? (ksName + ".") : "") + utName; + } +} diff --git a/cql3/UntypedResultSet.java b/cql3/UntypedResultSet.java new file mode 100644 index 0000000000..9d5a5f19f5 --- /dev/null +++ b/cql3/UntypedResultSet.java @@ -0,0 +1,301 @@ +/* + * 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.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.*; + +import com.google.common.collect.AbstractIterator; + +import org.apache.cassandra.cql3.statements.SelectStatement; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.pager.QueryPager; + +/** a utility for doing internal cql-based queries */ +public abstract class UntypedResultSet implements Iterable +{ + public static UntypedResultSet create(ResultSet rs) + { + return new FromResultSet(rs); + } + + public static UntypedResultSet create(List> results) + { + return new FromResultList(results); + } + + public static UntypedResultSet create(SelectStatement select, QueryPager pager, int pageSize) + { + return new FromPager(select, pager, pageSize); + } + + public boolean isEmpty() + { + return size() == 0; + } + + public abstract int size(); + public abstract Row one(); + + // No implemented by all subclasses, but we use it when we know it's there (for tests) + public abstract List metadata(); + + private static class FromResultSet extends UntypedResultSet + { + private final ResultSet cqlRows; + + private FromResultSet(ResultSet cqlRows) + { + this.cqlRows = cqlRows; + } + + public int size() + { + return cqlRows.size(); + } + + public Row one() + { + if (cqlRows.rows.size() != 1) + throw new IllegalStateException("One row required, " + cqlRows.rows.size() + " found"); + return new Row(cqlRows.metadata.names, cqlRows.rows.get(0)); + } + + public Iterator iterator() + { + return new AbstractIterator() + { + Iterator> iter = cqlRows.rows.iterator(); + + protected Row computeNext() + { + if (!iter.hasNext()) + return endOfData(); + return new Row(cqlRows.metadata.names, iter.next()); + } + }; + } + + public List metadata() + { + return cqlRows.metadata.names; + } + } + + private static class FromResultList extends UntypedResultSet + { + private final List> cqlRows; + + private FromResultList(List> cqlRows) + { + this.cqlRows = cqlRows; + } + + public int size() + { + return cqlRows.size(); + } + + public Row one() + { + if (cqlRows.size() != 1) + throw new IllegalStateException("One row required, " + cqlRows.size() + " found"); + return new Row(cqlRows.get(0)); + } + + public Iterator iterator() + { + return new AbstractIterator() + { + Iterator> iter = cqlRows.iterator(); + + protected Row computeNext() + { + if (!iter.hasNext()) + return endOfData(); + return new Row(iter.next()); + } + }; + } + + public List metadata() + { + throw new UnsupportedOperationException(); + } + } + + private static class FromPager extends UntypedResultSet + { + private final SelectStatement select; + private final QueryPager pager; + private final int pageSize; + private final List metadata; + + private FromPager(SelectStatement select, QueryPager pager, int pageSize) + { + this.select = select; + this.pager = pager; + this.pageSize = pageSize; + this.metadata = select.getResultMetadata().names; + } + + public int size() + { + throw new UnsupportedOperationException(); + } + + public Row one() + { + throw new UnsupportedOperationException(); + } + + public Iterator iterator() + { + return new AbstractIterator() + { + private Iterator> currentPage; + + protected Row computeNext() + { + try { + while (currentPage == null || !currentPage.hasNext()) + { + if (pager.isExhausted()) + return endOfData(); + currentPage = select.process(pager.fetchPage(pageSize)).rows.iterator(); + } + return new Row(metadata, currentPage.next()); + } catch (RequestValidationException | RequestExecutionException e) { + throw new RuntimeException(e); + } + } + }; + } + + public List metadata() + { + return metadata; + } + } + + public static class Row + { + private final Map data = new HashMap<>(); + private final List columns = new ArrayList<>(); + + public Row(Map data) + { + this.data.putAll(data); + } + + public Row(List names, List columns) + { + this.columns.addAll(names); + for (int i = 0; i < names.size(); i++) + data.put(names.get(i).name.toString(), columns.get(i)); + } + + public boolean has(String column) + { + // Note that containsKey won't work because we may have null values + return data.get(column) != null; + } + + public ByteBuffer getBlob(String column) + { + return data.get(column); + } + + public String getString(String column) + { + return UTF8Type.instance.compose(data.get(column)); + } + + public boolean getBoolean(String column) + { + return BooleanType.instance.compose(data.get(column)); + } + + public int getInt(String column) + { + return Int32Type.instance.compose(data.get(column)); + } + + public double getDouble(String column) + { + return DoubleType.instance.compose(data.get(column)); + } + + public ByteBuffer getBytes(String column) + { + return data.get(column); + } + + public InetAddress getInetAddress(String column) + { + return InetAddressType.instance.compose(data.get(column)); + } + + public UUID getUUID(String column) + { + return UUIDType.instance.compose(data.get(column)); + } + + public Date getTimestamp(String column) + { + return TimestampType.instance.compose(data.get(column)); + } + + public long getLong(String column) + { + return LongType.instance.compose(data.get(column)); + } + + public Set getSet(String column, AbstractType type) + { + ByteBuffer raw = data.get(column); + return raw == null ? null : SetType.getInstance(type, true).compose(raw); + } + + public List getList(String column, AbstractType type) + { + ByteBuffer raw = data.get(column); + return raw == null ? null : ListType.getInstance(type, true).compose(raw); + } + + public Map getMap(String column, AbstractType keyType, AbstractType valueType) + { + ByteBuffer raw = data.get(column); + return raw == null ? null : MapType.getInstance(keyType, valueType, true).compose(raw); + } + + public List getColumns() + { + return columns; + } + + @Override + public String toString() + { + return data.toString(); + } + } +} diff --git a/cql3/UpdateParameters.java b/cql3/UpdateParameters.java new file mode 100644 index 0000000000..74c3214bda --- /dev/null +++ b/cql3/UpdateParameters.java @@ -0,0 +1,102 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.utils.FBUtilities; + +/** + * A simple container that simplify passing parameters for collections methods. + */ +public class UpdateParameters +{ + public final CFMetaData metadata; + public final QueryOptions options; + public final long timestamp; + private final int ttl; + public final int localDeletionTime; + + // For lists operation that require a read-before-write. Will be null otherwise. + private final Map prefetchedLists; + + public UpdateParameters(CFMetaData metadata, QueryOptions options, long timestamp, int ttl, Map prefetchedLists) + throws InvalidRequestException + { + this.metadata = metadata; + this.options = options; + this.timestamp = timestamp; + this.ttl = ttl; + this.localDeletionTime = (int)(System.currentTimeMillis() / 1000); + this.prefetchedLists = prefetchedLists; + + // We use MIN_VALUE internally to mean the absence of of timestamp (in Selection, in sstable stats, ...), so exclude + // it to avoid potential confusion. + if (timestamp == Long.MIN_VALUE) + throw new InvalidRequestException(String.format("Out of bound timestamp, must be in [%d, %d]", Long.MIN_VALUE + 1, Long.MAX_VALUE)); + } + + public Cell makeColumn(CellName name, ByteBuffer value) throws InvalidRequestException + { + QueryProcessor.validateCellName(name, metadata.comparator); + return AbstractCell.create(name, value, timestamp, ttl, metadata); + } + + public Cell makeCounter(CellName name, long delta) throws InvalidRequestException + { + QueryProcessor.validateCellName(name, metadata.comparator); + return new BufferCounterUpdateCell(name, delta, FBUtilities.timestampMicros()); + } + + public Cell makeTombstone(CellName name) throws InvalidRequestException + { + QueryProcessor.validateCellName(name, metadata.comparator); + return new BufferDeletedCell(name, localDeletionTime, timestamp); + } + + public RangeTombstone makeRangeTombstone(ColumnSlice slice) throws InvalidRequestException + { + QueryProcessor.validateComposite(slice.start, metadata.comparator); + QueryProcessor.validateComposite(slice.finish, metadata.comparator); + return new RangeTombstone(slice.start, slice.finish, timestamp, localDeletionTime); + } + + public RangeTombstone makeTombstoneForOverwrite(ColumnSlice slice) throws InvalidRequestException + { + QueryProcessor.validateComposite(slice.start, metadata.comparator); + QueryProcessor.validateComposite(slice.finish, metadata.comparator); + return new RangeTombstone(slice.start, slice.finish, timestamp - 1, localDeletionTime); + } + + public List getPrefetchedList(ByteBuffer rowKey, ColumnIdentifier cql3ColumnName) + { + if (prefetchedLists == null) + return Collections.emptyList(); + + CQL3Row row = prefetchedLists.get(rowKey); + return row == null ? Collections.emptyList() : row.getMultiCellColumn(cql3ColumnName); + } +} diff --git a/cql3/UserOptions.java b/cql3/UserOptions.java new file mode 100644 index 0000000000..701debd5fc --- /dev/null +++ b/cql3/UserOptions.java @@ -0,0 +1,62 @@ +/* + * 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.auth.IAuthenticator; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.utils.FBUtilities; + +public class UserOptions +{ + private final Map options = new HashMap(); + + public void put(String name, Object value) + { + options.put(IAuthenticator.Option.valueOf(name.toUpperCase()), value); + } + + public boolean isEmpty() + { + return options.isEmpty(); + } + + public Map getOptions() + { + return options; + } + + public void validate() throws InvalidRequestException + { + for (IAuthenticator.Option option : options.keySet()) + { + if (!DatabaseDescriptor.getAuthenticator().supportedOptions().contains(option)) + throw new InvalidRequestException(String.format("%s doesn't support %s option", + DatabaseDescriptor.getAuthenticator().getClass().getName(), + option)); + } + } + + public String toString() + { + return FBUtilities.toString(options); + } +} diff --git a/cql3/UserTypes.java b/cql3/UserTypes.java new file mode 100644 index 0000000000..934344c19d --- /dev/null +++ b/cql3/UserTypes.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; + +import java.nio.ByteBuffer; +import java.util.*; + +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.transport.Server; + +/** + * Static helper methods and classes for user types. + */ +public abstract class UserTypes +{ + private UserTypes() {} + + public static ColumnSpecification fieldSpecOf(ColumnSpecification column, int field) + { + UserType ut = (UserType)column.type; + return new ColumnSpecification(column.ksName, + column.cfName, + new ColumnIdentifier(column.name + "." + UTF8Type.instance.compose(ut.fieldName(field)), true), + ut.fieldType(field)); + } + + public static class Literal implements Term.Raw + { + public final Map entries; + + public Literal(Map entries) + { + this.entries = entries; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + validateAssignableTo(keyspace, receiver); + + UserType ut = (UserType)receiver.type; + boolean allTerminal = true; + List values = new ArrayList<>(entries.size()); + int foundValues = 0; + for (int i = 0; i < ut.size(); i++) + { + ColumnIdentifier field = new ColumnIdentifier(ut.fieldName(i), UTF8Type.instance); + Term.Raw raw = entries.get(field); + if (raw == null) + raw = Constants.NULL_LITERAL; + else + ++foundValues; + Term value = raw.prepare(keyspace, fieldSpecOf(receiver, i)); + + if (value instanceof Term.NonTerminal) + allTerminal = false; + + values.add(value); + } + if (foundValues != entries.size()) + { + // We had some field that are not part of the type + for (ColumnIdentifier id : entries.keySet()) + if (!ut.fieldNames().contains(id.bytes)) + throw new InvalidRequestException(String.format("Unknown field '%s' in value of user defined type %s", id, ut.getNameAsString())); + } + + DelayedValue value = new DelayedValue(((UserType)receiver.type), values); + return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; + } + + private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + if (!(receiver.type instanceof UserType)) + throw new InvalidRequestException(String.format("Invalid user type literal for %s of type %s", receiver, receiver.type.asCQL3Type())); + + UserType ut = (UserType)receiver.type; + for (int i = 0; i < ut.size(); i++) + { + ColumnIdentifier field = new ColumnIdentifier(ut.fieldName(i), UTF8Type.instance); + Term.Raw value = entries.get(field); + if (value == null) + continue; + + ColumnSpecification fieldSpec = fieldSpecOf(receiver, i); + if (!value.testAssignment(keyspace, fieldSpec).isAssignable()) + throw new InvalidRequestException(String.format("Invalid user type literal for %s: field %s is not of type %s", receiver, field, fieldSpec.type.asCQL3Type())); + } + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + try + { + validateAssignableTo(keyspace, receiver); + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + catch (InvalidRequestException e) + { + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + Iterator> iter = entries.entrySet().iterator(); + while (iter.hasNext()) + { + Map.Entry entry = iter.next(); + sb.append(entry.getKey()).append(":").append(entry.getValue()); + if (iter.hasNext()) + sb.append(", "); + } + sb.append("}"); + return sb.toString(); + } + } + + // Same purpose than Lists.DelayedValue, except we do handle bind marker in that case + public static class DelayedValue extends Term.NonTerminal + { + private final UserType type; + private final List values; + + public DelayedValue(UserType type, List values) + { + this.type = type; + this.values = values; + } + + public boolean usesFunction(String ksName, String functionName) + { + if (values != null) + for (Term value : values) + if (value != null && value.usesFunction(ksName, functionName)) + return true; + return false; + } + + public boolean containsBindMarker() + { + for (Term t : values) + if (t.containsBindMarker()) + return true; + return false; + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + for (int i = 0; i < type.size(); i++) + values.get(i).collectMarkerSpecification(boundNames); + } + + private ByteBuffer[] bindInternal(QueryOptions options) throws InvalidRequestException + { + int version = options.getProtocolVersion(); + + ByteBuffer[] buffers = new ByteBuffer[values.size()]; + for (int i = 0; i < type.size(); i++) + { + buffers[i] = values.get(i).bindAndGet(options); + // Inside UDT values, we must force the serialization of collections to v3 whatever protocol + // version is in use since we're going to store directly that serialized value. + if (version < Server.VERSION_3 && type.fieldType(i).isCollection() && buffers[i] != null) + buffers[i] = ((CollectionType)type.fieldType(i)).getSerializer().reserializeToV3(buffers[i]); + } + return buffers; + } + + public Constants.Value bind(QueryOptions options) throws InvalidRequestException + { + return new Constants.Value(bindAndGet(options)); + } + + @Override + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + return UserType.buildValue(bindInternal(options)); + } + } +} diff --git a/cql3/VariableSpecifications.java b/cql3/VariableSpecifications.java new file mode 100644 index 0000000000..0a55ced82f --- /dev/null +++ b/cql3/VariableSpecifications.java @@ -0,0 +1,68 @@ +/* + * 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.Arrays; +import java.util.Collections; +import java.util.List; + +public class VariableSpecifications +{ + private final List variableNames; + private final ColumnSpecification[] specs; + + public VariableSpecifications(List variableNames) + { + this.variableNames = variableNames; + this.specs = new ColumnSpecification[variableNames.size()]; + } + + /** + * Returns an empty instance of VariableSpecifications. + * @return an empty instance of VariableSpecifications + */ + public static VariableSpecifications empty() + { + return new VariableSpecifications(Collections. emptyList()); + } + + public int size() + { + return variableNames.size(); + } + + public List getSpecifications() + { + return Arrays.asList(specs); + } + + public void add(int bindIndex, ColumnSpecification spec) + { + ColumnIdentifier name = variableNames.get(bindIndex); + // Use the user name, if there is one + if (name != null) + spec = new ColumnSpecification(spec.ksName, spec.cfName, name, spec.type); + specs[bindIndex] = spec; + } + + @Override + public String toString() + { + return Arrays.toString(specs); + } +} diff --git a/cql3/functions/AbstractFunction.java b/cql3/functions/AbstractFunction.java new file mode 100644 index 0000000000..e2d69b89f2 --- /dev/null +++ b/cql3/functions/AbstractFunction.java @@ -0,0 +1,99 @@ +/* + * 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.functions; + +import java.util.List; + +import com.google.common.base.Objects; + +import org.apache.cassandra.db.marshal.AbstractType; + +/** + * Base class for our native/hardcoded functions. + */ +public abstract class AbstractFunction implements Function +{ + protected final FunctionName name; + protected final List> argTypes; + protected final AbstractType returnType; + + protected AbstractFunction(FunctionName name, List> argTypes, AbstractType returnType) + { + this.name = name; + this.argTypes = argTypes; + this.returnType = returnType; + } + + public FunctionName name() + { + return name; + } + + public List> argTypes() + { + return argTypes; + } + + public AbstractType returnType() + { + return returnType; + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof AbstractFunction)) + return false; + + AbstractFunction that = (AbstractFunction)o; + return Objects.equal(this.name, that.name) + && Objects.equal(this.argTypes, that.argTypes) + && Objects.equal(this.returnType, that.returnType); + } + + public boolean usesFunction(String ksName, String functionName) + { + return name.keyspace.equals(ksName) && name.name.equals(functionName); + } + + public boolean hasReferenceTo(Function function) + { + return false; + } + + @Override + public int hashCode() + { + return Objects.hashCode(name, argTypes, returnType); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append(name).append(" : ("); + for (int i = 0; i < argTypes.size(); i++) + { + if (i > 0) + sb.append(", "); + sb.append(argTypes.get(i).asCQL3Type()); + } + sb.append(") -> ").append(returnType.asCQL3Type()); + return sb.toString(); + } +} diff --git a/cql3/functions/AggregateFcts.java b/cql3/functions/AggregateFcts.java new file mode 100644 index 0000000000..865dfbf267 --- /dev/null +++ b/cql3/functions/AggregateFcts.java @@ -0,0 +1,661 @@ +/* + * 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.functions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; + +/** + * Factory methods for aggregate functions. + */ +public abstract class AggregateFcts +{ + /** + * The function used to count the number of rows of a result set. This function is called when COUNT(*) or COUNT(1) + * is specified. + */ + public static final AggregateFunction countRowsFunction = + new NativeAggregateFunction("countRows", LongType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private long count; + + public void reset() + { + count = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((LongType) returnType()).decompose(Long.valueOf(count)); + } + + public void addInput(int protocolVersion, List values) + { + count++; + } + }; + } + }; + + /** + * The SUM function for decimal values. + */ + public static final AggregateFunction sumFunctionForDecimal = + new NativeAggregateFunction("sum", DecimalType.instance, DecimalType.instance) + { + @Override + public Aggregate newAggregate() + { + return new Aggregate() + { + private BigDecimal sum = BigDecimal.ZERO; + + public void reset() + { + sum = BigDecimal.ZERO; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((DecimalType) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + BigDecimal number = ((BigDecimal) argTypes().get(0).compose(value)); + sum = sum.add(number); + } + }; + } + }; + + /** + * The AVG function for decimal values. + */ + public static final AggregateFunction avgFunctionForDecimal = + new NativeAggregateFunction("avg", DecimalType.instance, DecimalType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private BigDecimal sum = BigDecimal.ZERO; + + private int count; + + public void reset() + { + count = 0; + sum = BigDecimal.ZERO; + } + + public ByteBuffer compute(int protocolVersion) + { + if (count == 0) + return ((DecimalType) returnType()).decompose(BigDecimal.ZERO); + + return ((DecimalType) returnType()).decompose(sum.divide(BigDecimal.valueOf(count))); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + BigDecimal number = ((BigDecimal) argTypes().get(0).compose(value)); + sum = sum.add(number); + } + }; + } + }; + + /** + * The SUM function for varint values. + */ + public static final AggregateFunction sumFunctionForVarint = + new NativeAggregateFunction("sum", IntegerType.instance, IntegerType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private BigInteger sum = BigInteger.ZERO; + + public void reset() + { + sum = BigInteger.ZERO; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((IntegerType) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + BigInteger number = ((BigInteger) argTypes().get(0).compose(value)); + sum = sum.add(number); + } + }; + } + }; + + /** + * The AVG function for varint values. + */ + public static final AggregateFunction avgFunctionForVarint = + new NativeAggregateFunction("avg", IntegerType.instance, IntegerType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private BigInteger sum = BigInteger.ZERO; + + private int count; + + public void reset() + { + count = 0; + sum = BigInteger.ZERO; + } + + public ByteBuffer compute(int protocolVersion) + { + if (count == 0) + return ((IntegerType) returnType()).decompose(BigInteger.ZERO); + + return ((IntegerType) returnType()).decompose(sum.divide(BigInteger.valueOf(count))); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + BigInteger number = ((BigInteger) argTypes().get(0).compose(value)); + sum = sum.add(number); + } + }; + } + }; + + /** + * The SUM function for int32 values. + */ + public static final AggregateFunction sumFunctionForInt32 = + new NativeAggregateFunction("sum", Int32Type.instance, Int32Type.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private int sum; + + public void reset() + { + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((Int32Type) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.intValue(); + } + }; + } + }; + + /** + * AVG function for int32 values. + */ + public static final AggregateFunction avgFunctionForInt32 = + new NativeAggregateFunction("avg", Int32Type.instance, Int32Type.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private int sum; + + private int count; + + public void reset() + { + count = 0; + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + int avg = count == 0 ? 0 : sum / count; + + return ((Int32Type) returnType()).decompose(avg); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.intValue(); + } + }; + } + }; + + /** + * The SUM function for long values. + */ + public static final AggregateFunction sumFunctionForLong = + new NativeAggregateFunction("sum", LongType.instance, LongType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private long sum; + + public void reset() + { + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((LongType) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.longValue(); + } + }; + } + }; + + /** + * AVG function for long values. + */ + public static final AggregateFunction avgFunctionForLong = + new NativeAggregateFunction("avg", LongType.instance, LongType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private long sum; + + private int count; + + public void reset() + { + count = 0; + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + long avg = count == 0 ? 0 : sum / count; + + return ((LongType) returnType()).decompose(avg); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.longValue(); + } + }; + } + }; + + /** + * The SUM function for float values. + */ + public static final AggregateFunction sumFunctionForFloat = + new NativeAggregateFunction("sum", FloatType.instance, FloatType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private float sum; + + public void reset() + { + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((FloatType) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.floatValue(); + } + }; + } + }; + + /** + * AVG function for float values. + */ + public static final AggregateFunction avgFunctionForFloat = + new NativeAggregateFunction("avg", FloatType.instance, FloatType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private float sum; + + private int count; + + public void reset() + { + count = 0; + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + float avg = count == 0 ? 0 : sum / count; + + return ((FloatType) returnType()).decompose(avg); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.floatValue(); + } + }; + } + }; + + /** + * The SUM function for double values. + */ + public static final AggregateFunction sumFunctionForDouble = + new NativeAggregateFunction("sum", DoubleType.instance, DoubleType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private double sum; + + public void reset() + { + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((DoubleType) returnType()).decompose(sum); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.doubleValue(); + } + }; + } + }; + + /** + * AVG function for double values. + */ + public static final AggregateFunction avgFunctionForDouble = + new NativeAggregateFunction("avg", DoubleType.instance, DoubleType.instance) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private double sum; + + private int count; + + public void reset() + { + count = 0; + sum = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + double avg = count == 0 ? 0 : sum / count; + + return ((DoubleType) returnType()).decompose(avg); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + Number number = ((Number) argTypes().get(0).compose(value)); + sum += number.doubleValue(); + } + }; + } + }; + + /** + * Creates a MAX function for the specified type. + * + * @param inputType the function input and output type + * @return a MAX function for the specified type. + */ + public static AggregateFunction makeMaxFunction(final AbstractType inputType) + { + return new NativeAggregateFunction("max", inputType, inputType) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private ByteBuffer max; + + public void reset() + { + max = null; + } + + public ByteBuffer compute(int protocolVersion) + { + return max; + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + if (max == null || returnType().compare(max, value) < 0) + max = value; + } + }; + } + }; + } + + /** + * Creates a MIN function for the specified type. + * + * @param inputType the function input and output type + * @return a MIN function for the specified type. + */ + public static AggregateFunction makeMinFunction(final AbstractType inputType) + { + return new NativeAggregateFunction("min", inputType, inputType) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private ByteBuffer min; + + public void reset() + { + min = null; + } + + public ByteBuffer compute(int protocolVersion) + { + return min; + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + if (min == null || returnType().compare(min, value) > 0) + min = value; + } + }; + } + }; + } + + /** + * Creates a COUNT function for the specified type. + * + * @param inputType the function input type + * @return a COUNT function for the specified type. + */ + public static AggregateFunction makeCountFunction(AbstractType inputType) + { + return new NativeAggregateFunction("count", LongType.instance, inputType) + { + public Aggregate newAggregate() + { + return new Aggregate() + { + private long count; + + public void reset() + { + count = 0; + } + + public ByteBuffer compute(int protocolVersion) + { + return ((LongType) returnType()).decompose(count); + } + + public void addInput(int protocolVersion, List values) + { + ByteBuffer value = values.get(0); + + if (value == null) + return; + + count++; + } + }; + } + }; + } +} diff --git a/cql3/functions/AggregateFunction.java b/cql3/functions/AggregateFunction.java new file mode 100644 index 0000000000..ddbc9d1c35 --- /dev/null +++ b/cql3/functions/AggregateFunction.java @@ -0,0 +1,63 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Performs a calculation on a set of values and return a single value. + */ +public interface AggregateFunction extends Function +{ + /** + * Creates a new Aggregate instance. + * + * @return a new Aggregate instance. + */ + public Aggregate newAggregate() throws InvalidRequestException; + + /** + * An aggregation operation. + */ + interface Aggregate + { + /** + * Adds the specified input to this aggregate. + * + * @param protocolVersion native protocol version + * @param values the values to add to the aggregate. + */ + public void addInput(int protocolVersion, List values) throws InvalidRequestException; + + /** + * Computes and returns the aggregate current value. + * + * @param protocolVersion native protocol version + * @return the aggregate current value. + */ + public ByteBuffer compute(int protocolVersion) throws InvalidRequestException; + + /** + * Reset this aggregate. + */ + public void reset(); + } +} diff --git a/cql3/functions/BytesConversionFcts.java b/cql3/functions/BytesConversionFcts.java new file mode 100644 index 0000000000..ddb33fc093 --- /dev/null +++ b/cql3/functions/BytesConversionFcts.java @@ -0,0 +1,84 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.serializers.MarshalException; + +public abstract class BytesConversionFcts +{ + // Most of the XAsBlob and blobAsX functions are basically no-op since everything is + // bytes internally. They only "trick" the type system. + public static Function makeToBlobFunction(AbstractType fromType) + { + String name = fromType.asCQL3Type() + "asblob"; + return new NativeScalarFunction(name, BytesType.instance, fromType) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + return parameters.get(0); + } + }; + } + + public static Function makeFromBlobFunction(final AbstractType toType) + { + final String name = "blobas" + toType.asCQL3Type(); + return new NativeScalarFunction(name, toType, BytesType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) throws InvalidRequestException + { + ByteBuffer val = parameters.get(0); + try + { + if (val != null) + toType.validate(val); + return val; + } + catch (MarshalException e) + { + throw new InvalidRequestException(String.format("In call to function %s, value 0x%s is not a valid binary representation for type %s", + name, ByteBufferUtil.bytesToHex(val), toType.asCQL3Type())); + } + } + }; + } + + public static final Function VarcharAsBlobFct = new NativeScalarFunction("varcharasblob", BytesType.instance, UTF8Type.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + return parameters.get(0); + } + }; + + public static final Function BlobAsVarcharFact = new NativeScalarFunction("blobasvarchar", UTF8Type.instance, BytesType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + return parameters.get(0); + } + }; +} diff --git a/cql3/functions/Function.java b/cql3/functions/Function.java new file mode 100644 index 0000000000..4d2b99351b --- /dev/null +++ b/cql3/functions/Function.java @@ -0,0 +1,58 @@ +/* + * 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.functions; + +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; + +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.github.jamm.Unmetered; + +@Unmetered +public interface Function +{ + public FunctionName name(); + public List> argTypes(); + public AbstractType returnType(); + + /** + * Checks whether the function is a pure function (as in doesn't depend on, nor produce side effects) or not. + * + * @return true if the function is a pure function, false otherwise. + */ + public boolean isPure(); + + /** + * Checks whether the function is a native/hard coded one or not. + * + * @return true if the function is a native/hard coded one, false otherwise. + */ + public boolean isNative(); + + /** + * Checks whether the function is an aggregate function or not. + * + * @return true if the function is an aggregate function, false otherwise. + */ + public boolean isAggregate(); + + boolean usesFunction(String ksName, String functionName); + + boolean hasReferenceTo(Function function); +} diff --git a/cql3/functions/FunctionCall.java b/cql3/functions/FunctionCall.java new file mode 100644 index 0000000000..72ac63e353 --- /dev/null +++ b/cql3/functions/FunctionCall.java @@ -0,0 +1,215 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.serializers.MarshalException; + +public class FunctionCall extends Term.NonTerminal +{ + private final ScalarFunction fun; + private final List terms; + + private FunctionCall(ScalarFunction fun, List terms) + { + this.fun = fun; + this.terms = terms; + } + + public boolean usesFunction(String ksName, String functionName) + { + return fun.usesFunction(ksName, functionName); + } + + public void collectMarkerSpecification(VariableSpecifications boundNames) + { + for (Term t : terms) + t.collectMarkerSpecification(boundNames); + } + + public Term.Terminal bind(QueryOptions options) throws InvalidRequestException + { + return makeTerminal(fun, bindAndGet(options), options.getProtocolVersion()); + } + + public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException + { + List buffers = new ArrayList<>(terms.size()); + for (Term t : terms) + { + // For now, we don't allow nulls as argument as no existing function needs it and it + // simplify things. + ByteBuffer val = t.bindAndGet(options); + if (val == null) + throw new InvalidRequestException(String.format("Invalid null value for argument to %s", fun)); + buffers.add(val); + } + return executeInternal(options.getProtocolVersion(), fun, buffers); + } + + private static ByteBuffer executeInternal(int protocolVersion, ScalarFunction fun, List params) throws InvalidRequestException + { + ByteBuffer result = fun.execute(protocolVersion, params); + try + { + // Check the method didn't lied on it's declared return type + if (result != null) + fun.returnType().validate(result); + return result; + } + catch (MarshalException e) + { + throw new RuntimeException(String.format("Return of function %s (%s) is not a valid value for its declared return type %s", + fun, ByteBufferUtil.bytesToHex(result), fun.returnType().asCQL3Type())); + } + } + + public boolean containsBindMarker() + { + for (Term t : terms) + { + if (t.containsBindMarker()) + return true; + } + return false; + } + + private static Term.Terminal makeTerminal(Function fun, ByteBuffer result, int version) throws InvalidRequestException + { + if (!(fun.returnType() instanceof CollectionType)) + return new Constants.Value(result); + + switch (((CollectionType)fun.returnType()).kind) + { + case LIST: return Lists.Value.fromSerialized(result, (ListType)fun.returnType(), version); + case SET: return Sets.Value.fromSerialized(result, (SetType)fun.returnType(), version); + case MAP: return Maps.Value.fromSerialized(result, (MapType)fun.returnType(), version); + } + throw new AssertionError(); + } + + public static class Raw implements Term.Raw + { + private FunctionName name; + private final List terms; + + public Raw(FunctionName name, List terms) + { + this.name = name; + this.terms = terms; + } + + public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException + { + Function fun = Functions.get(keyspace, name, terms, receiver.ksName, receiver.cfName); + if (fun == null) + throw new InvalidRequestException(String.format("Unknown function %s called", name)); + if (fun.isAggregate()) + throw new InvalidRequestException("Aggregation function are not supported in the where clause"); + + ScalarFunction scalarFun = (ScalarFunction) fun; + + // Functions.get() will complain if no function "name" type check with the provided arguments. + // We still have to validate that the return type matches however + if (!receiver.type.isValueCompatibleWith(scalarFun.returnType())) + throw new InvalidRequestException(String.format("Type error: cannot assign result of function %s (type %s) to %s (type %s)", + scalarFun.name(), scalarFun.returnType().asCQL3Type(), + receiver.name, receiver.type.asCQL3Type())); + + if (fun.argTypes().size() != terms.size()) + throw new InvalidRequestException(String.format("Incorrect number of arguments specified for function %s (expected %d, found %d)", + fun.name(), fun.argTypes().size(), terms.size())); + + List parameters = new ArrayList<>(terms.size()); + boolean allTerminal = true; + for (int i = 0; i < terms.size(); i++) + { + Term t = terms.get(i).prepare(keyspace, Functions.makeArgSpec(receiver.ksName, receiver.cfName, scalarFun, i)); + if (t instanceof NonTerminal) + allTerminal = false; + parameters.add(t); + } + + // If all parameters are terminal and the function is pure, we can + // evaluate it now, otherwise we'd have to wait execution time + return allTerminal && scalarFun.isPure() + ? makeTerminal(scalarFun, execute(scalarFun, parameters), QueryOptions.DEFAULT.getProtocolVersion()) + : new FunctionCall(scalarFun, parameters); + } + + // All parameters must be terminal + private static ByteBuffer execute(ScalarFunction fun, List parameters) throws InvalidRequestException + { + List buffers = new ArrayList<>(parameters.size()); + for (Term t : parameters) + { + assert t instanceof Term.Terminal; + buffers.add(((Term.Terminal)t).get(QueryOptions.DEFAULT)); + } + + return executeInternal(Server.CURRENT_VERSION, fun, buffers); + } + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + // Note: Functions.get() will return null if the function doesn't exist, or throw is no function matching + // the arguments can be found. We may get one of those if an undefined/wrong function is used as argument + // of another, existing, function. In that case, we return true here because we'll throw a proper exception + // later with a more helpful error message that if we were to return false here. + try + { + Function fun = Functions.get(keyspace, name, terms, receiver.ksName, receiver.cfName); + if (fun != null && receiver.type.equals(fun.returnType())) + return AssignmentTestable.TestResult.EXACT_MATCH; + else if (fun == null || receiver.type.isValueCompatibleWith(fun.returnType())) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + else + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } + catch (InvalidRequestException e) + { + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append(name).append("("); + for (int i = 0; i < terms.size(); i++) + { + if (i > 0) + sb.append(", "); + sb.append(terms.get(i)); + } + return sb.append(")").toString(); + } + } +} diff --git a/cql3/functions/FunctionName.java b/cql3/functions/FunctionName.java new file mode 100644 index 0000000000..bb30040a3e --- /dev/null +++ b/cql3/functions/FunctionName.java @@ -0,0 +1,73 @@ +/* + * 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.functions; + +import com.google.common.base.Objects; + +import org.apache.cassandra.db.SystemKeyspace; + +public final class FunctionName +{ + public final String keyspace; + public final String name; + + public static FunctionName nativeFunction(String name) + { + return new FunctionName(SystemKeyspace.NAME, name); + } + + public FunctionName(String keyspace, String name) + { + assert name != null : "Name parameter must not be null"; + this.keyspace = keyspace != null ? keyspace : null; + this.name = name; + } + + public FunctionName asNativeFunction() + { + return FunctionName.nativeFunction(name); + } + + public boolean hasKeyspace() + { + return keyspace != null; + } + + @Override + public final int hashCode() + { + return Objects.hashCode(keyspace, name); + } + + @Override + public final boolean equals(Object o) + { + if (!(o instanceof FunctionName)) + return false; + + FunctionName that = (FunctionName)o; + return Objects.equal(this.keyspace, that.keyspace) + && Objects.equal(this.name, that.name); + } + + @Override + public String toString() + { + return keyspace == null ? name : keyspace + "." + name; + } +} diff --git a/cql3/functions/Functions.java b/cql3/functions/Functions.java new file mode 100644 index 0000000000..b55ebc5e92 --- /dev/null +++ b/cql3/functions/Functions.java @@ -0,0 +1,329 @@ +/* + * 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.functions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import com.google.common.collect.ArrayListMultimap; + +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.IMigrationListener; +import org.apache.cassandra.service.MigrationManager; + +public abstract class Functions +{ + // We special case the token function because that's the only function whose argument types actually + // depend on the table on which the function is called. Because it's the sole exception, it's easier + // to handle it as a special case. + private static final FunctionName TOKEN_FUNCTION_NAME = FunctionName.nativeFunction("token"); + + private Functions() {} + + private static final ArrayListMultimap declared = ArrayListMultimap.create(); + + static + { + declare(AggregateFcts.countRowsFunction); + declare(TimeuuidFcts.nowFct); + declare(TimeuuidFcts.minTimeuuidFct); + declare(TimeuuidFcts.maxTimeuuidFct); + declare(TimeuuidFcts.dateOfFct); + declare(TimeuuidFcts.unixTimestampOfFct); + declare(UuidFcts.uuidFct); + + for (CQL3Type type : CQL3Type.Native.values()) + { + // Note: because text and varchar ends up being synonimous, our automatic makeToBlobFunction doesn't work + // for varchar, so we special case it below. We also skip blob for obvious reasons. + if (type == CQL3Type.Native.VARCHAR || type == CQL3Type.Native.BLOB) + continue; + + declare(BytesConversionFcts.makeToBlobFunction(type.getType())); + declare(BytesConversionFcts.makeFromBlobFunction(type.getType())); + + declare(AggregateFcts.makeCountFunction(type.getType())); + declare(AggregateFcts.makeMaxFunction(type.getType())); + declare(AggregateFcts.makeMinFunction(type.getType())); + } + declare(BytesConversionFcts.VarcharAsBlobFct); + declare(BytesConversionFcts.BlobAsVarcharFact); + declare(AggregateFcts.sumFunctionForInt32); + declare(AggregateFcts.sumFunctionForLong); + declare(AggregateFcts.sumFunctionForFloat); + declare(AggregateFcts.sumFunctionForDouble); + declare(AggregateFcts.sumFunctionForDecimal); + declare(AggregateFcts.sumFunctionForVarint); + declare(AggregateFcts.avgFunctionForInt32); + declare(AggregateFcts.avgFunctionForLong); + declare(AggregateFcts.avgFunctionForFloat); + declare(AggregateFcts.avgFunctionForDouble); + declare(AggregateFcts.avgFunctionForVarint); + declare(AggregateFcts.avgFunctionForDecimal); + + MigrationManager.instance.register(new FunctionsMigrationListener()); + } + + private static void declare(Function fun) + { + declared.put(fun.name(), fun); + } + + public static ColumnSpecification makeArgSpec(String receiverKs, String receiverCf, Function fun, int i) + { + return new ColumnSpecification(receiverKs, + receiverCf, + new ColumnIdentifier("arg" + i + "(" + fun.name().toString().toLowerCase() + ")", true), + fun.argTypes().get(i)); + } + + public static int getOverloadCount(FunctionName name) + { + return declared.get(name).size(); + } + + public static Function get(String keyspace, + FunctionName name, + List providedArgs, + String receiverKs, + String receiverCf) + throws InvalidRequestException + { + if (name.hasKeyspace() + ? name.equals(TOKEN_FUNCTION_NAME) + : name.name.equals(TOKEN_FUNCTION_NAME.name)) + return new TokenFct(Schema.instance.getCFMetaData(receiverKs, receiverCf)); + + List candidates; + if (!name.hasKeyspace()) + { + // function name not fully qualified + candidates = new ArrayList<>(); + // add 'SYSTEM' (native) candidates + candidates.addAll(declared.get(name.asNativeFunction())); + // add 'current keyspace' candidates + candidates.addAll(declared.get(new FunctionName(keyspace, name.name))); + } + else + // function name is fully qualified (keyspace + name) + candidates = declared.get(name); + + if (candidates.isEmpty()) + return null; + + // Fast path if there is only one choice + if (candidates.size() == 1) + { + Function fun = candidates.get(0); + validateTypes(keyspace, fun, providedArgs, receiverKs, receiverCf); + return fun; + } + + List compatibles = null; + for (Function toTest : candidates) + { + AssignmentTestable.TestResult r = matchAguments(keyspace, toTest, providedArgs, receiverKs, receiverCf); + switch (r) + { + case EXACT_MATCH: + // We always favor exact matches + return toTest; + case WEAKLY_ASSIGNABLE: + if (compatibles == null) + compatibles = new ArrayList<>(); + compatibles.add(toTest); + break; + } + } + + if (compatibles == null || compatibles.isEmpty()) + throw new InvalidRequestException(String.format("Invalid call to function %s, none of its type signatures match (known type signatures: %s)", + name, toString(candidates))); + + if (compatibles.size() > 1) + throw new InvalidRequestException(String.format("Ambiguous call to function %s (can be matched by following signatures: %s): use type casts to disambiguate", + name, toString(compatibles))); + + return compatibles.get(0); + } + + public static List find(FunctionName name) + { + return declared.get(name); + } + + public static Function find(FunctionName name, List> argTypes) + { + assert name.hasKeyspace() : "function name not fully qualified"; + for (Function f : declared.get(name)) + { + if (typeEquals(f.argTypes(), argTypes)) + return f; + } + return null; + } + + // This method and matchArguments are somewhat duplicate, but this method allows us to provide more precise errors in the common + // case where there is no override for a given function. This is thus probably worth the minor code duplication. + private static void validateTypes(String keyspace, + Function fun, + List providedArgs, + String receiverKs, + String receiverCf) + throws InvalidRequestException + { + if (providedArgs.size() != fun.argTypes().size()) + throw new InvalidRequestException(String.format("Invalid number of arguments in call to function %s: %d required but %d provided", fun.name(), fun.argTypes().size(), providedArgs.size())); + + for (int i = 0; i < providedArgs.size(); i++) + { + AssignmentTestable provided = providedArgs.get(i); + + // If the concrete argument is a bind variables, it can have any type. + // We'll validate the actually provided value at execution time. + if (provided == null) + continue; + + ColumnSpecification expected = makeArgSpec(receiverKs, receiverCf, fun, i); + if (!provided.testAssignment(keyspace, expected).isAssignable()) + throw new InvalidRequestException(String.format("Type error: %s cannot be passed as argument %d of function %s of type %s", provided, i, fun.name(), expected.type.asCQL3Type())); + } + } + + private static AssignmentTestable.TestResult matchAguments(String keyspace, + Function fun, + List providedArgs, + String receiverKs, + String receiverCf) + { + if (providedArgs.size() != fun.argTypes().size()) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + + // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable. + AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH; + for (int i = 0; i < providedArgs.size(); i++) + { + AssignmentTestable provided = providedArgs.get(i); + if (provided == null) + { + res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + continue; + } + + ColumnSpecification expected = makeArgSpec(receiverKs, receiverCf, fun, i); + AssignmentTestable.TestResult argRes = provided.testAssignment(keyspace, expected); + if (argRes == AssignmentTestable.TestResult.NOT_ASSIGNABLE) + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + if (argRes == AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE) + res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + } + return res; + } + + private static String toString(List funs) + { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < funs.size(); i++) + { + if (i > 0) sb.append(", "); + sb.append(funs.get(i)); + } + return sb.toString(); + } + + // This is *not* thread safe but is only called in SchemaTables that is synchronized. + public static void addFunction(AbstractFunction fun) + { + // We shouldn't get there unless that function don't exist + assert find(fun.name(), fun.argTypes()) == null; + declare(fun); + } + + // Same remarks than for addFunction + public static void removeFunction(FunctionName name, List> argsTypes) + { + Function old = find(name, argsTypes); + assert old != null && !old.isNative(); + declared.remove(old.name(), old); + } + + // Same remarks than for addFunction + public static void replaceFunction(AbstractFunction fun) + { + removeFunction(fun.name(), fun.argTypes()); + addFunction(fun); + } + + public static List getReferencesTo(Function old) + { + List references = new ArrayList<>(); + for (Function function : declared.values()) + if (function.hasReferenceTo(old)) + references.add(function); + return references; + } + + public static Collection all() + { + return declared.values(); + } + + public static boolean typeEquals(AbstractType t1, AbstractType t2) + { + return t1.asCQL3Type().toString().equals(t2.asCQL3Type().toString()); + } + + public static boolean typeEquals(List> t1, List> t2) + { + if (t1.size() != t2.size()) + return false; + for (int i = 0; i < t1.size(); i ++) + if (!typeEquals(t1.get(i), t2.get(i))) + return false; + return true; + } + + private static class FunctionsMigrationListener implements IMigrationListener + { + public void onCreateKeyspace(String ksName) { } + public void onCreateColumnFamily(String ksName, String cfName) { } + public void onCreateUserType(String ksName, String typeName) { } + public void onCreateFunction(String ksName, String functionName) { } + public void onCreateAggregate(String ksName, String aggregateName) { } + + public void onUpdateKeyspace(String ksName) { } + public void onUpdateColumnFamily(String ksName, String cfName) { } + public void onUpdateUserType(String ksName, String typeName) { + for (Function function : all()) + if (function instanceof UDFunction) + ((UDFunction)function).userTypeUpdated(ksName, typeName); + } + public void onUpdateFunction(String ksName, String functionName) { } + public void onUpdateAggregate(String ksName, String aggregateName) { } + + public void onDropKeyspace(String ksName) { } + public void onDropColumnFamily(String ksName, String cfName) { } + public void onDropUserType(String ksName, String typeName) { } + public void onDropFunction(String ksName, String functionName) { } + public void onDropAggregate(String ksName, String aggregateName) { } + } +} diff --git a/cql3/functions/JavaSourceUDFFactory.java b/cql3/functions/JavaSourceUDFFactory.java new file mode 100644 index 0000000000..e4e6a559c2 --- /dev/null +++ b/cql3/functions/JavaSourceUDFFactory.java @@ -0,0 +1,258 @@ +/* + * 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.functions; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Modifier; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import javassist.CannotCompileException; +import javassist.ClassPool; +import javassist.CtClass; +import javassist.CtNewConstructor; +import javassist.CtNewMethod; +import javassist.NotFoundException; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Java source UDF code generation. + */ +public final class JavaSourceUDFFactory +{ + private static final String GENERATED_CODE_PACKAGE = "org.apache.cassandra.cql3.udf.gen."; + + protected static final Logger logger = LoggerFactory.getLogger(JavaSourceUDFFactory.class); + + private static final AtomicInteger classSequence = new AtomicInteger(); + + static UDFunction buildUDF(FunctionName name, + List argNames, + List> argTypes, + AbstractType returnType, + String body, + boolean deterministic) + throws InvalidRequestException + { + // argDataTypes is just the C* internal argTypes converted to the Java Driver DataType + DataType[] argDataTypes = UDHelper.driverTypes(argTypes); + // returnDataType is just the C* internal returnType converted to the Java Driver DataType + DataType returnDataType = UDHelper.driverType(returnType); + // javaParamTypes is just the Java representation for argTypes resp. argDataTypes + Class[] javaParamTypes = UDHelper.javaTypes(argDataTypes); + // javaReturnType is just the Java representation for returnType resp. returnDataType + Class javaReturnType = returnDataType.asJavaClass(); + + String clsName = generateClassName(name); + + String codeCtor = generateConstructor(clsName); + + // Generate 'execute' method (implements org.apache.cassandra.cql3.functions.Function.execute) + String codeExec = generateExecuteMethod(argNames, javaParamTypes); + + // Generate the 'executeInternal' method + // It is separated to allow return type and argument type checks during compile time via javassist. + String codeExecInt = generateExecuteInternalMethod(argNames, body, javaReturnType, javaParamTypes); + + logger.debug("Generating java source UDF for {} with following c'tor and functions:\n{}\n{}\n{}", + name, codeCtor, codeExecInt, codeExec); + + try + { + ClassPool classPool = ClassPool.getDefault(); + + // get super class + CtClass base = classPool.get(UDFunction.class.getName()); + + // prepare class to generate + CtClass cc = classPool.makeClass(GENERATED_CODE_PACKAGE + clsName, base); + cc.setModifiers(cc.getModifiers() | Modifier.FINAL); + + // add c'tor plus methods (order matters) + cc.addConstructor(CtNewConstructor.make(codeCtor, cc)); + cc.addMethod(CtNewMethod.make(codeExecInt, cc)); + cc.addMethod(CtNewMethod.make(codeExec, cc)); + + Constructor ctor = + cc.toClass().getDeclaredConstructor( + FunctionName.class, List.class, List.class, DataType[].class, + AbstractType.class, DataType.class, + String.class, boolean.class); + return (UDFunction) ctor.newInstance( + name, argNames, argTypes, argDataTypes, + returnType, returnDataType, + body, deterministic); + } + catch (NotFoundException | CannotCompileException | NoSuchMethodException | LinkageError | InstantiationException | IllegalAccessException e) + { + throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e)); + } + catch (InvocationTargetException e) + { + // in case of an ITE, use the cause + throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e.getCause())); + } + } + + private static String generateClassName(FunctionName name) + { + String qualifiedName = name.toString(); + + StringBuilder sb = new StringBuilder(qualifiedName.length()+10); + sb.append('C'); + for (int i = 0; i < qualifiedName.length(); i++) + { + char c = qualifiedName.charAt(i); + if (Character.isJavaIdentifierPart(c)) + sb.append(c); + } + sb.append('_'); + sb.append(classSequence.incrementAndGet()); + return sb.toString(); + } + + /** + * Generates constructor with just a call super class (UDFunction) constructor with constant 'java' as language. + */ + private static String generateConstructor(String clsName) + { + return "public " + clsName + + "(org.apache.cassandra.cql3.functions.FunctionName name, " + + "java.util.List argNames, " + + "java.util.List argTypes, " + + "com.datastax.driver.core.DataType[] argDataTypes, " + + "org.apache.cassandra.db.marshal.AbstractType returnType, " + + "com.datastax.driver.core.DataType returnDataType, " + + "String body," + + "boolean deterministic)\n{" + + " super(name, argNames, argTypes, argDataTypes, returnType, returnDataType, \"java\", body, deterministic);\n" + + "}"; + } + + /** + * Generate executeInternal method (just there to find return and argument type mismatches in UDF body). + * + * Generated looks like this: + *
+     * private  executeInternal( paramOne,  nextParam)
+     * {
+     *     
+     * }
+     * 
+ */ + private static String generateExecuteInternalMethod(List argNames, String body, Class returnType, Class[] paramTypes) + { + // initial builder size can just be a guess (prevent temp object allocations) + StringBuilder codeInt = new StringBuilder(64 + 32*paramTypes.length + body.length()); + codeInt.append("private ").append(returnType.getName()).append(" executeInternal("); + for (int i = 0; i < paramTypes.length; i++) + { + if (i > 0) + codeInt.append(", "); + codeInt.append(paramTypes[i].getName()). + append(' '). + append(argNames.get(i)); + } + codeInt.append(")\n{"). + append(body). + append('}'); + return codeInt.toString(); + } + + /** + * + * Generate public execute() method implementation. + * + * Generated looks like this: + *
+     *
+     * public java.nio.ByteBuffer execute(int protocolVersion, java.util.List params)
+     * throws org.apache.cassandra.exceptions.InvalidRequestException
+     * {
+     *     try
+     *     {
+     *         Object result = executeInternal(
+     *             ()compose(protocolVersion, 0, (java.nio.ByteBuffer)params.get(0)),
+     *             ()compose(protocolVersion, 1, (java.nio.ByteBuffer)params.get(1)),
+     *             ...
+     *         );
+     *         return decompose(protocolVersion, result);
+     *     }
+     *     catch (Throwable t)
+     *     {
+     *         logger.error("Invocation of function '{}' failed", this, t);
+     *         if (t instanceof VirtualMachineError)
+     *             throw (VirtualMachineError)t;
+     *         throw new org.apache.cassandra.exceptions.InvalidRequestException("Invocation of function '" + this + "' failed: " + t);
+     *     }
+     * }
+     * 
+ */ + private static String generateExecuteMethod(List argNames, Class[] paramTypes) + { + // usual methods are 700-800 chars long (prevent temp object allocations) + StringBuilder code = new StringBuilder(1024); + // overrides org.apache.cassandra.cql3.functions.Function.execute(java.util.List) + code.append("public java.nio.ByteBuffer execute(int protocolVersion, java.util.List params)\n" + + "throws org.apache.cassandra.exceptions.InvalidRequestException\n" + + "{\n" + + " try\n" + + " {\n" + + " Object result = executeInternal("); + for (int i = 0; i < paramTypes.length; i++) + { + if (i > 0) + code.append(','); + + if (logger.isDebugEnabled()) + code.append("\n /* ").append(argNames.get(i)).append(" */ "); + + code. + // cast to Java type + append("\n (").append(paramTypes[i].getName()).append(")"). + // generate object representation of input parameter (call UDFunction.compose) + append("compose(protocolVersion, ").append(i).append(", (java.nio.ByteBuffer)params.get(").append(i).append("))"); + } + + code.append("\n );\n" + + // generate serialized return value (returnType is a field in AbstractFunction class), (call UDFunction.decompose) + " return decompose(protocolVersion, result);\n" + + // + // error handling ... + " }\n" + + " catch (Throwable t)\n" + + " {\n" + + " logger.error(\"Invocation of function '{}' failed\", this, t);\n" + + // handle OutOfMemoryError and other fatals not here! + " if (t instanceof VirtualMachineError)\n" + + " throw (VirtualMachineError)t;\n" + + " throw new org.apache.cassandra.exceptions.InvalidRequestException(\"Invocation of function '\" + this + \"' failed: \" + t);\n" + + " }\n" + + "}"); + + return code.toString(); + } + +} diff --git a/cql3/functions/NativeAggregateFunction.java b/cql3/functions/NativeAggregateFunction.java new file mode 100644 index 0000000000..88aab4be77 --- /dev/null +++ b/cql3/functions/NativeAggregateFunction.java @@ -0,0 +1,36 @@ +/* + * 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.functions; + +import org.apache.cassandra.db.marshal.AbstractType; + +/** + * Base class for the AggregateFunction native classes. + */ +public abstract class NativeAggregateFunction extends NativeFunction implements AggregateFunction +{ + protected NativeAggregateFunction(String name, AbstractType returnType, AbstractType... argTypes) + { + super(name, returnType, argTypes); + } + + public final boolean isAggregate() + { + return true; + } +} diff --git a/cql3/functions/NativeFunction.java b/cql3/functions/NativeFunction.java new file mode 100644 index 0000000000..bff768818e --- /dev/null +++ b/cql3/functions/NativeFunction.java @@ -0,0 +1,44 @@ +/* + * 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.functions; + +import java.util.Arrays; + +import org.apache.cassandra.db.marshal.AbstractType; + +/** + * Base class for our native/hardcoded functions. + */ +public abstract class NativeFunction extends AbstractFunction +{ + protected NativeFunction(String name, AbstractType returnType, AbstractType... argTypes) + { + super(FunctionName.nativeFunction(name), Arrays.asList(argTypes), returnType); + } + + // Most of our functions are pure, the other ones should override this + public boolean isPure() + { + return true; + } + + public boolean isNative() + { + return true; + } +} diff --git a/cql3/functions/NativeScalarFunction.java b/cql3/functions/NativeScalarFunction.java new file mode 100644 index 0000000000..8f7f2212f8 --- /dev/null +++ b/cql3/functions/NativeScalarFunction.java @@ -0,0 +1,36 @@ +/* + * 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.functions; + +import org.apache.cassandra.db.marshal.AbstractType; + +/** + * Base class for the ScalarFunction native classes. + */ +public abstract class NativeScalarFunction extends NativeFunction implements ScalarFunction +{ + protected NativeScalarFunction(String name, AbstractType returnType, AbstractType... argsType) + { + super(name, returnType, argsType); + } + + public final boolean isAggregate() + { + return false; + } +} diff --git a/cql3/functions/ScalarFunction.java b/cql3/functions/ScalarFunction.java new file mode 100644 index 0000000000..f00faf79d3 --- /dev/null +++ b/cql3/functions/ScalarFunction.java @@ -0,0 +1,39 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Determines a single output value based on a single input value. + */ +public interface ScalarFunction extends Function +{ + /** + * Applies this function to the specified parameter. + * + * @param protocolVersion protocol version used for parameters and return value + * @param parameters the input parameters + * @return the result of applying this function to the parameter + * @throws InvalidRequestException if this function cannot not be applied to the parameter + */ + public ByteBuffer execute(int protocolVersion, List parameters) throws InvalidRequestException; +} diff --git a/cql3/functions/ScriptBasedUDF.java b/cql3/functions/ScriptBasedUDF.java new file mode 100644 index 0000000000..059a61298f --- /dev/null +++ b/cql3/functions/ScriptBasedUDF.java @@ -0,0 +1,145 @@ +/* + * 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.functions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.script.Bindings; +import javax.script.Compilable; +import javax.script.CompiledScript; +import javax.script.ScriptEngine; +import javax.script.ScriptEngineFactory; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; +import javax.script.SimpleBindings; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +public class ScriptBasedUDF extends UDFunction +{ + static final Map scriptEngines = new HashMap<>(); + + static { + ScriptEngineManager scriptEngineManager = new ScriptEngineManager(); + for (ScriptEngineFactory scriptEngineFactory : scriptEngineManager.getEngineFactories()) + { + ScriptEngine scriptEngine = scriptEngineFactory.getScriptEngine(); + boolean compilable = scriptEngine instanceof Compilable; + if (compilable) + { + logger.info("Found scripting engine {} {} - {} {} - language names: {}", + scriptEngineFactory.getEngineName(), scriptEngineFactory.getEngineVersion(), + scriptEngineFactory.getLanguageName(), scriptEngineFactory.getLanguageVersion(), + scriptEngineFactory.getNames()); + for (String name : scriptEngineFactory.getNames()) + scriptEngines.put(name, (Compilable) scriptEngine); + } + } + } + + private final CompiledScript script; + + ScriptBasedUDF(FunctionName name, + List argNames, + List> argTypes, + AbstractType returnType, + String language, + String body, + boolean deterministic) + throws InvalidRequestException + { + super(name, argNames, argTypes, returnType, language, body, deterministic); + + Compilable scriptEngine = scriptEngines.get(language); + if (scriptEngine == null) + throw new InvalidRequestException(String.format("Invalid language '%s' for function '%s'", language, name)); + + try + { + this.script = scriptEngine.compile(body); + } + catch (RuntimeException | ScriptException e) + { + logger.info("Failed to compile function '{}' for language {}: ", name, language, e); + throw new InvalidRequestException( + String.format("Failed to compile function '%s' for language %s: %s", name, language, e)); + } + } + + public ByteBuffer execute(int protocolVersion, List parameters) throws InvalidRequestException + { + Object[] params = new Object[argTypes.size()]; + for (int i = 0; i < params.length; i++) + params[i] = compose(protocolVersion, i, parameters.get(i)); + + try + { + Bindings bindings = new SimpleBindings(); + for (int i = 0; i < params.length; i++) + bindings.put(argNames.get(i).toString(), params[i]); + + Object result = script.eval(bindings); + if (result == null) + return null; + + Class javaReturnType = returnDataType.asJavaClass(); + Class resultType = result.getClass(); + if (!javaReturnType.isAssignableFrom(resultType)) + { + if (result instanceof Number) + { + Number rNumber = (Number) result; + if (javaReturnType == Integer.class) + result = rNumber.intValue(); + else if (javaReturnType == Long.class) + result = rNumber.longValue(); + else if (javaReturnType == Float.class) + result = rNumber.floatValue(); + else if (javaReturnType == Double.class) + result = rNumber.doubleValue(); + else if (javaReturnType == BigInteger.class) + { + if (rNumber instanceof BigDecimal) + result = ((BigDecimal)rNumber).toBigInteger(); + else if (rNumber instanceof Double || rNumber instanceof Float) + result = new BigDecimal(rNumber.toString()).toBigInteger(); + else + result = BigInteger.valueOf(rNumber.longValue()); + } + else if (javaReturnType == BigDecimal.class) + // String c'tor of BigDecimal is more accurate than valueOf(double) + result = new BigDecimal(rNumber.toString()); + } + } + + return decompose(protocolVersion, result); + } + catch (RuntimeException | ScriptException e) + { + logger.info("Execution of UDF '{}' failed", name, e); + throw new InvalidRequestException("Execution of user-defined function '" + name + "' failed: " + e); + } + } +} diff --git a/cql3/functions/TimeuuidFcts.java b/cql3/functions/TimeuuidFcts.java new file mode 100644 index 0000000000..c1c3490a20 --- /dev/null +++ b/cql3/functions/TimeuuidFcts.java @@ -0,0 +1,94 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.Date; +import java.util.List; + +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.db.marshal.TimeUUIDType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.UUIDGen; + +public abstract class TimeuuidFcts +{ + public static final Function nowFct = new NativeScalarFunction("now", TimeUUIDType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()); + } + + @Override + public boolean isPure() + { + return false; + } + }; + + public static final Function minTimeuuidFct = new NativeScalarFunction("mintimeuuid", TimeUUIDType.instance, TimestampType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + ByteBuffer bb = parameters.get(0); + if (bb == null) + return null; + + return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(TimestampType.instance.compose(bb).getTime()))); + } + }; + + public static final Function maxTimeuuidFct = new NativeScalarFunction("maxtimeuuid", TimeUUIDType.instance, TimestampType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + ByteBuffer bb = parameters.get(0); + if (bb == null) + return null; + + return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(TimestampType.instance.compose(bb).getTime()))); + } + }; + + public static final Function dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + ByteBuffer bb = parameters.get(0); + if (bb == null) + return null; + + return TimestampType.instance.decompose(new Date(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)))); + } + }; + + public static final Function unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + ByteBuffer bb = parameters.get(0); + if (bb == null) + return null; + + return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb))); + } + }; +} + diff --git a/cql3/functions/TokenFct.java b/cql3/functions/TokenFct.java new file mode 100644 index 0000000000..9d50a972f6 --- /dev/null +++ b/cql3/functions/TokenFct.java @@ -0,0 +1,65 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.composites.CBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.StorageService; + +public class TokenFct extends NativeScalarFunction +{ + // The actual token function depends on the partitioner used + private static final IPartitioner partitioner = StorageService.getPartitioner(); + + private final CFMetaData cfm; + + public TokenFct(CFMetaData cfm) + { + super("token", partitioner.getTokenValidator(), getKeyTypes(cfm)); + this.cfm = cfm; + } + + private static AbstractType[] getKeyTypes(CFMetaData cfm) + { + AbstractType[] types = new AbstractType[cfm.partitionKeyColumns().size()]; + int i = 0; + for (ColumnDefinition def : cfm.partitionKeyColumns()) + types[i++] = def.type; + return types; + } + + public ByteBuffer execute(int protocolVersion, List parameters) throws InvalidRequestException + { + CBuilder builder = cfm.getKeyValidatorAsCType().builder(); + for (int i = 0; i < parameters.size(); i++) + { + ByteBuffer bb = parameters.get(i); + if (bb == null) + return null; + builder.add(bb); + } + return partitioner.getTokenFactory().toByteArray(partitioner.getToken(builder.build().toByteBuffer())); + } +} diff --git a/cql3/functions/UDAggregate.java b/cql3/functions/UDAggregate.java new file mode 100644 index 0000000000..e9c33ba574 --- /dev/null +++ b/cql3/functions/UDAggregate.java @@ -0,0 +1,206 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.*; + +/** + * Base class for user-defined-aggregates. + */ +public class UDAggregate extends AbstractFunction implements AggregateFunction +{ + protected static final Logger logger = LoggerFactory.getLogger(UDAggregate.class); + + protected final AbstractType stateType; + protected final ByteBuffer initcond; + private final ScalarFunction stateFunction; + private final ScalarFunction finalFunction; + + public UDAggregate(FunctionName name, + List> argTypes, + AbstractType returnType, + ScalarFunction stateFunc, + ScalarFunction finalFunc, + ByteBuffer initcond) + { + super(name, argTypes, returnType); + this.stateFunction = stateFunc; + this.finalFunction = finalFunc; + this.stateType = stateFunc != null ? stateFunc.returnType() : null; + this.initcond = initcond; + } + + public static UDAggregate create(FunctionName name, + List> argTypes, + AbstractType returnType, + FunctionName stateFunc, + FunctionName finalFunc, + AbstractType stateType, + ByteBuffer initcond) + throws InvalidRequestException + { + List> stateTypes = new ArrayList<>(argTypes.size() + 1); + stateTypes.add(stateType); + stateTypes.addAll(argTypes); + List> finalTypes = Collections.>singletonList(stateType); + return new UDAggregate(name, + argTypes, + returnType, + resolveScalar(name, stateFunc, stateTypes), + finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null, + initcond); + } + + public static UDAggregate createBroken(FunctionName name, + List> argTypes, + AbstractType returnType, + ByteBuffer initcond, + final InvalidRequestException reason) + { + return new UDAggregate(name, argTypes, returnType, null, null, initcond) + { + public Aggregate newAggregate() throws InvalidRequestException + { + throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. " + + "Please see the server log for more details", + this, + reason.getMessage())); + } + }; + } + + public boolean hasReferenceTo(Function function) + { + return stateFunction == function || finalFunction == function; + } + + public boolean usesFunction(String ksName, String functionName) + { + return super.usesFunction(ksName, functionName) + || stateFunction != null && stateFunction.name().keyspace.equals(ksName) && stateFunction.name().name.equals(functionName) + || finalFunction != null && finalFunction.name().keyspace.equals(ksName) && finalFunction.name().name.equals(functionName); + } + + public boolean isAggregate() + { + return true; + } + + public boolean isPure() + { + return false; + } + + public boolean isNative() + { + return false; + } + + public ScalarFunction stateFunction() + { + return stateFunction; + } + + public ScalarFunction finalFunction() + { + return finalFunction; + } + + public ByteBuffer initialCondition() + { + return initcond; + } + + public AbstractType stateType() + { + return stateType; + } + + public Aggregate newAggregate() throws InvalidRequestException + { + return new Aggregate() + { + private ByteBuffer state; + { + reset(); + } + + public void addInput(int protocolVersion, List values) throws InvalidRequestException + { + List copy = new ArrayList<>(values.size() + 1); + copy.add(state); + copy.addAll(values); + state = stateFunction.execute(protocolVersion, copy); + } + + public ByteBuffer compute(int protocolVersion) throws InvalidRequestException + { + if (finalFunction == null) + return state; + return finalFunction.execute(protocolVersion, Collections.singletonList(state)); + } + + public void reset() + { + state = initcond != null ? initcond.duplicate() : null; + } + }; + } + + private static ScalarFunction resolveScalar(FunctionName aName, FunctionName fName, List> argTypes) throws InvalidRequestException + { + Function func = Functions.find(fName, argTypes); + if (func == null) + throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' does not exist", + fName, Arrays.toString(UDHelper.driverTypes(argTypes)), aName)); + if (!(func instanceof ScalarFunction)) + throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' is not a scalar function", + fName, Arrays.toString(UDHelper.driverTypes(argTypes)), aName)); + return (ScalarFunction) func; + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof UDAggregate)) + return false; + + UDAggregate that = (UDAggregate) o; + return Objects.equal(name, that.name) + && Functions.typeEquals(argTypes, that.argTypes) + && Functions.typeEquals(returnType, that.returnType) + && Objects.equal(stateFunction, that.stateFunction) + && Objects.equal(finalFunction, that.finalFunction) + && Objects.equal(stateType, that.stateType) + && Objects.equal(initcond, that.initcond); + } + + @Override + public int hashCode() + { + return Objects.hashCode(name, argTypes, returnType, stateFunction, finalFunction, stateType, initcond); + } +} diff --git a/cql3/functions/UDFunction.java b/cql3/functions/UDFunction.java new file mode 100644 index 0000000000..46724516e2 --- /dev/null +++ b/cql3/functions/UDFunction.java @@ -0,0 +1,244 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.ProtocolVersion; +import com.datastax.driver.core.UserType; +import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * Base class for User Defined Functions. + */ +public abstract class UDFunction extends AbstractFunction implements ScalarFunction +{ + protected static final Logger logger = LoggerFactory.getLogger(UDFunction.class); + + protected final List argNames; + + protected final String language; + protected final String body; + protected final boolean isDeterministic; + + protected final DataType[] argDataTypes; + protected final DataType returnDataType; + + protected UDFunction(FunctionName name, + List argNames, + List> argTypes, + AbstractType returnType, + String language, + String body, + boolean isDeterministic) + { + this(name, argNames, argTypes, UDHelper.driverTypes(argTypes), returnType, + UDHelper.driverType(returnType), language, body, isDeterministic); + } + + protected UDFunction(FunctionName name, + List argNames, + List> argTypes, + DataType[] argDataTypes, + AbstractType returnType, + DataType returnDataType, + String language, + String body, + boolean isDeterministic) + { + super(name, argTypes, returnType); + assert new HashSet<>(argNames).size() == argNames.size() : "duplicate argument names"; + this.argNames = argNames; + this.language = language; + this.body = body; + this.isDeterministic = isDeterministic; + this.argDataTypes = argDataTypes; + this.returnDataType = returnDataType; + } + + public static UDFunction create(FunctionName name, + List argNames, + List> argTypes, + AbstractType returnType, + String language, + String body, + boolean isDeterministic) + throws InvalidRequestException + { + switch (language) + { + case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, body, isDeterministic); + default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, language, body, isDeterministic); + } + } + + /** + * It can happen that a function has been declared (is listed in the scheam) but cannot + * be loaded (maybe only on some nodes). This is the case for instance if the class defining + * the class is not on the classpath for some of the node, or after a restart. In that case, + * we create a "fake" function so that: + * 1) the broken function can be dropped easily if that is what people want to do. + * 2) we return a meaningful error message if the function is executed (something more precise + * than saying that the function doesn't exist) + */ + public static UDFunction createBrokenFunction(FunctionName name, + List argNames, + List> argTypes, + AbstractType returnType, + String language, + String body, + final InvalidRequestException reason) + { + return new UDFunction(name, argNames, argTypes, returnType, language, body, true) + { + public ByteBuffer execute(int protocolVersion, List parameters) throws InvalidRequestException + { + throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully " + + "for the following reason: %s. Please see the server log for details", + this, + reason.getMessage())); + } + }; + } + + + public boolean isAggregate() + { + return false; + } + + public boolean isPure() + { + return isDeterministic; + } + + public boolean isNative() + { + return false; + } + + public List argNames() + { + return argNames; + } + + public boolean isDeterministic() + { + return isDeterministic; + } + + public String body() + { + return body; + } + + public String language() + { + return language; + } + + /** + * Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory} + * and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the C* + * serialized representation to the Java object representation. + * + * @param protocolVersion the native protocol version used for serialization + * @param argIndex index of the UDF input argument + */ + protected Object compose(int protocolVersion, int argIndex, ByteBuffer value) + { + return value == null ? null : argDataTypes[argIndex].deserialize(value, ProtocolVersion.fromInt(protocolVersion)); + } + + /** + * Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory} + * and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the Java + * object representation for the return value to the C* serialized representation. + * + * @param protocolVersion the native protocol version used for serialization + */ + protected ByteBuffer decompose(int protocolVersion, Object value) + { + return value == null ? null : returnDataType.serialize(value, ProtocolVersion.fromInt(protocolVersion)); + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof UDFunction)) + return false; + + UDFunction that = (UDFunction)o; + return Objects.equal(name, that.name) + && Objects.equal(argNames, that.argNames) + && Functions.typeEquals(argTypes, that.argTypes) + && Functions.typeEquals(returnType, that.returnType) + && Objects.equal(language, that.language) + && Objects.equal(body, that.body) + && Objects.equal(isDeterministic, that.isDeterministic); + } + + @Override + public int hashCode() + { + return Objects.hashCode(name, argNames, argTypes, returnType, language, body, isDeterministic); + } + + public void userTypeUpdated(String ksName, String typeName) + { + boolean updated = false; + + for (int i = 0; i < argDataTypes.length; i++) + { + DataType dataType = argDataTypes[i]; + if (dataType instanceof UserType) + { + UserType userType = (UserType) dataType; + if (userType.getKeyspace().equals(ksName) && userType.getTypeName().equals(typeName)) + { + KSMetaData ksm = Schema.instance.getKSMetaData(ksName); + assert ksm != null; + + org.apache.cassandra.db.marshal.UserType ut = ksm.userTypes.getType(ByteBufferUtil.bytes(typeName)); + + DataType newUserType = UDHelper.driverType(ut); + argDataTypes[i] = newUserType; + + argTypes.set(i, ut); + + updated = true; + } + } + } + + if (updated) + MigrationManager.announceNewFunction(this, true); + } +} diff --git a/cql3/functions/UDHelper.java b/cql3/functions/UDHelper.java new file mode 100644 index 0000000000..0738cbe68d --- /dev/null +++ b/cql3/functions/UDHelper.java @@ -0,0 +1,125 @@ +/* + * 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.functions; + +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Helper class for User Defined Functions + Aggregates. + */ +public final class UDHelper +{ + protected static final Logger logger = LoggerFactory.getLogger(UDHelper.class); + + // TODO make these c'tors and methods public in Java-Driver - see https://datastax-oss.atlassian.net/browse/JAVA-502 + static final MethodHandle methodParseOne; + static + { + try + { + Class cls = Class.forName("com.datastax.driver.core.CassandraTypeParser"); + Method m = cls.getDeclaredMethod("parseOne", String.class); + m.setAccessible(true); + methodParseOne = MethodHandles.lookup().unreflect(m); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + + /** + * Construct an array containing the Java classes for the given Java Driver {@link com.datastax.driver.core.DataType}s. + * + * @param dataTypes array with UDF argument types + * @return array of same size with UDF arguments + */ + public static Class[] javaTypes(DataType[] dataTypes) + { + Class paramTypes[] = new Class[dataTypes.length]; + for (int i = 0; i < paramTypes.length; i++) + paramTypes[i] = dataTypes[i].asJavaClass(); + return paramTypes; + } + + /** + * Construct an array containing the Java Driver {@link com.datastax.driver.core.DataType}s for the + * C* internal types. + * + * @param abstractTypes list with UDF argument types + * @return array with argument types as {@link com.datastax.driver.core.DataType} + */ + public static DataType[] driverTypes(List> abstractTypes) + { + DataType[] argDataTypes = new DataType[abstractTypes.size()]; + for (int i = 0; i < argDataTypes.length; i++) + argDataTypes[i] = driverType(abstractTypes.get(i)); + return argDataTypes; + } + + /** + * Returns the Java Driver {@link com.datastax.driver.core.DataType} for the C* internal type. + */ + public static DataType driverType(AbstractType abstractType) + { + CQL3Type cqlType = abstractType.asCQL3Type(); + try + { + return (DataType) methodParseOne.invoke(cqlType.getType().toString()); + } + catch (RuntimeException | Error e) + { + // immediately rethrow these... + throw e; + } + catch (Throwable e) + { + throw new RuntimeException("cannot parse driver type " + cqlType.getType().toString(), e); + } + } + + // We allow method overloads, so a function is not uniquely identified by its name only, but + // also by its argument types. To distinguish overloads of given function name in the schema + // we use a "signature" which is just a SHA-1 of it's argument types (we could replace that by + // using a "signature" UDT that would be comprised of the function name and argument types, + // which we could then use as clustering column. But as we haven't yet used UDT in system tables, + // We'll leave that decision to #6717). + public static ByteBuffer calculateSignature(AbstractFunction fun) + { + MessageDigest digest = FBUtilities.newMessageDigest("SHA-1"); + digest.update(UTF8Type.instance.decompose(fun.name().name)); + for (AbstractType type : fun.argTypes()) + digest.update(UTF8Type.instance.decompose(type.asCQL3Type().toString())); + return ByteBuffer.wrap(digest.digest()); + } +} diff --git a/cql3/functions/UuidFcts.java b/cql3/functions/UuidFcts.java new file mode 100644 index 0000000000..afb5aaebf4 --- /dev/null +++ b/cql3/functions/UuidFcts.java @@ -0,0 +1,42 @@ +/* + * 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.functions; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; + +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.serializers.UUIDSerializer; + +public abstract class UuidFcts +{ + public static final Function uuidFct = new NativeScalarFunction("uuid", UUIDType.instance) + { + public ByteBuffer execute(int protocolVersion, List parameters) + { + return UUIDSerializer.instance.serialize(UUID.randomUUID()); + } + + @Override + public boolean isPure() + { + return false; + } + }; +} diff --git a/cql3/restrictions/AbstractPrimaryKeyRestrictions.java b/cql3/restrictions/AbstractPrimaryKeyRestrictions.java new file mode 100644 index 0000000000..f137a776f5 --- /dev/null +++ b/cql3/restrictions/AbstractPrimaryKeyRestrictions.java @@ -0,0 +1,36 @@ +/* + * 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.restrictions; + +/** + * Base class for PrimaryKeyRestrictions. + */ +abstract class AbstractPrimaryKeyRestrictions extends AbstractRestriction implements PrimaryKeyRestrictions +{ + @Override + public final boolean isEmpty() + { + return getColumnDefs().isEmpty(); + } + + @Override + public final int size() + { + return getColumnDefs().size(); + } +} diff --git a/cql3/restrictions/AbstractRestriction.java b/cql3/restrictions/AbstractRestriction.java new file mode 100644 index 0000000000..0ae7b22311 --- /dev/null +++ b/cql3/restrictions/AbstractRestriction.java @@ -0,0 +1,129 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull; + +/** + * Base class for Restrictions + */ +abstract class AbstractRestriction implements Restriction +{ + @Override + public boolean isOnToken() + { + return false; + } + + @Override + public boolean isMultiColumn() + { + return false; + } + + @Override + public boolean isSlice() + { + return false; + } + + @Override + public boolean isEQ() + { + return false; + } + + @Override + public boolean isIN() + { + return false; + } + + @Override + public boolean isContains() + { + return false; + } + + @Override + public boolean hasBound(Bound b) + { + return true; + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return values(options); + } + + @Override + public boolean isInclusive(Bound b) + { + return true; + } + + protected static ByteBuffer validateIndexedValue(ColumnSpecification columnSpec, + ByteBuffer value) + throws InvalidRequestException + { + checkNotNull(value, "Unsupported null value for indexed column %s", columnSpec.name); + checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K"); + return value; + } + + /** + * Checks if the specified term is using the specified function. + * + * @param term the term to check + * @param ksName the function keyspace name + * @param functionName the function name + * @return true if the specified term is using the specified function, false otherwise. + */ + protected static final boolean usesFunction(Term term, String ksName, String functionName) + { + return term != null && term.usesFunction(ksName, functionName); + } + + /** + * Checks if one of the specified term is using the specified function. + * + * @param terms the terms to check + * @param ksName the function keyspace name + * @param functionName the function name + * @return true if onee of the specified term is using the specified function, false otherwise. + */ + protected static final boolean usesFunction(List terms, String ksName, String functionName) + { + if (terms != null) + for (Term value : terms) + if (usesFunction(value, ksName, functionName)) + return true; + return false; + } +} \ No newline at end of file diff --git a/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java b/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java new file mode 100644 index 0000000000..8a57292151 --- /dev/null +++ b/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java @@ -0,0 +1,159 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.List; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A PrimaryKeyRestrictions which forwards all its method calls to another + * PrimaryKeyRestrictions. Subclasses should override one or more methods to modify the behavior + * of the backing PrimaryKeyRestrictions as desired per the decorator pattern. + */ +abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestrictions +{ + /** + * Returns the backing delegate instance that methods are forwarded to. + * @return the backing delegate instance that methods are forwarded to. + */ + protected abstract PrimaryKeyRestrictions getDelegate(); + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return getDelegate().usesFunction(ksName, functionName); + } + + @Override + public Collection getColumnDefs() + { + return getDelegate().getColumnDefs(); + } + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException + { + return getDelegate().mergeWith(restriction); + } + + @Override + public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager) + { + return getDelegate().hasSupportingIndex(secondaryIndexManager); + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return getDelegate().values(options); + } + + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + return getDelegate().valuesAsComposites(options); + } + + @Override + public List bounds(Bound bound, QueryOptions options) throws InvalidRequestException + { + return getDelegate().bounds(bound, options); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + return getDelegate().boundsAsComposites(bound, options); + } + + @Override + public boolean isInclusive(Bound bound) + { + return getDelegate().isInclusive(bound.reverse()); + } + + @Override + public boolean isEmpty() + { + return getDelegate().isEmpty(); + } + + @Override + public int size() + { + return getDelegate().size(); + } + + @Override + public boolean isOnToken() + { + return getDelegate().isOnToken(); + } + + @Override + public boolean isSlice() + { + return getDelegate().isSlice(); + } + + @Override + public boolean isEQ() + { + return getDelegate().isEQ(); + } + + @Override + public boolean isIN() + { + return getDelegate().isIN(); + } + + @Override + public boolean isContains() + { + return getDelegate().isContains(); + } + + @Override + public boolean isMultiColumn() + { + return getDelegate().isMultiColumn(); + } + + @Override + public boolean hasBound(Bound b) + { + return getDelegate().hasBound(b); + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + getDelegate().addIndexExpressionTo(expressions, options); + } +} diff --git a/cql3/restrictions/MultiColumnRestriction.java b/cql3/restrictions/MultiColumnRestriction.java new file mode 100644 index 0000000000..e3b3c4cfda --- /dev/null +++ b/cql3/restrictions/MultiColumnRestriction.java @@ -0,0 +1,520 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.*; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.AbstractMarker; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.Tuples; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.composites.CBuilder; +import org.apache.cassandra.db.composites.CType; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.composites.Composites; +import org.apache.cassandra.db.index.SecondaryIndex; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +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.cql3.statements.RequestValidations.invalidRequest; + +public abstract class MultiColumnRestriction extends AbstractPrimaryKeyRestrictions +{ + protected final CType ctype; + + /** + * The columns to which the restriction apply. + */ + protected final List columnDefs; + + public MultiColumnRestriction(CType ctype, List columnDefs) + { + this.ctype = ctype; + this.columnDefs = columnDefs; + } + + @Override + public boolean isMultiColumn() + { + return true; + } + + @Override + public Collection getColumnDefs() + { + return columnDefs; + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return Composites.toByteBuffers(valuesAsComposites(options)); + } + + @Override + public final PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException + { + checkTrue(otherRestriction.isMultiColumn(), + "Mixing single column relations and multi column relations on clustering columns is not allowed"); + return doMergeWith((PrimaryKeyRestrictions) otherRestriction); + } + + protected abstract PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException; + + /** + * Returns the names of the columns that are specified within this Restrictions and the other one + * as a comma separated String. + * + * @param otherRestrictions the other restrictions + * @return the names of the columns that are specified within this Restrictions and the other one + * as a comma separated String. + */ + protected final String getColumnsInCommons(Restrictions otherRestrictions) + { + Set commons = new HashSet<>(getColumnDefs()); + commons.retainAll(otherRestrictions.getColumnDefs()); + StringBuilder builder = new StringBuilder(); + for (ColumnDefinition columnDefinition : commons) + { + if (builder.length() != 0) + builder.append(" ,"); + builder.append(columnDefinition.name); + } + return builder.toString(); + } + + @Override + public final boolean hasSupportingIndex(SecondaryIndexManager indexManager) + { + for (ColumnDefinition columnDef : columnDefs) + { + SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes); + if (index != null && isSupportedBy(index)) + return true; + } + return false; + } + + /** + * Check if this type of restriction is supported for the specified column by the specified index. + * @param index the Secondary index + * + * @return true this type of restriction is supported by the specified index, + * false otherwise. + */ + protected abstract boolean isSupportedBy(SecondaryIndex index); + + public static class EQ extends MultiColumnRestriction + { + protected final Term value; + + public EQ(CType ctype, List columnDefs, Term value) + { + super(ctype, columnDefs); + this.value = value; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(value, ksName, functionName); + } + + @Override + public String toString() + { + return String.format("EQ(%s)", value); + } + + @Override + public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException + { + throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", + getColumnsInCommons(otherRestriction)); + } + + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + return Collections.singletonList(compositeValue(options)); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + Composite prefix = compositeValue(options); + return Collections.singletonList(ctype.size() > prefix.size() && bound.isEnd() + ? prefix.end() + : prefix); + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + return index.supportsOperator(Operator.EQ); + } + + private Composite compositeValue(QueryOptions options) throws InvalidRequestException + { + CBuilder builder = ctype.builder(); + Tuples.Value t = ((Tuples.Value) value.bind(options)); + List values = t.getElements(); + for (int i = 0; i < values.size(); i++) + { + ByteBuffer component = checkNotNull(values.get(i), + "Invalid null value in condition for column %s", + columnDefs.get(i).name); + builder.add(component); + } + + return builder.build(); + } + + @Override + public final void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + Tuples.Value t = ((Tuples.Value) value.bind(options)); + List values = t.getElements(); + for (int i = 0; i < values.size(); i++) + { + ColumnDefinition columnDef = columnDefs.get(i); + ByteBuffer component = validateIndexedValue(columnDef, values.get(i)); + expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component)); + } + } + } + + public abstract static class IN extends MultiColumnRestriction + { + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + CBuilder builder = ctype.builder(); + List> splitInValues = splitValues(options); + // The IN query might not have listed the values in comparator order, so we need to re-sort + // the bounds lists to make sure the slices works correctly (also, to avoid duplicates). + TreeSet inValues = new TreeSet<>(ctype); + for (List components : splitInValues) + { + for (int i = 0; i < components.size(); i++) + checkNotNull(components.get(i), "Invalid null value in condition for column " + columnDefs.get(i).name); + + inValues.add(builder.buildWith(components)); + } + return new ArrayList<>(inValues); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + CBuilder builder = ctype.builder(); + List> splitInValues = splitValues(options); + // The IN query might not have listed the values in comparator order, so we need to re-sort + // the bounds lists to make sure the slices works correctly (also, to avoid duplicates). + TreeSet inValues = new TreeSet<>(ctype); + for (List components : splitInValues) + { + for (int i = 0; i < components.size(); i++) + checkNotNull(components.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name); + + Composite prefix = builder.buildWith(components); + inValues.add(bound.isEnd() && builder.remainingCount() - components.size() > 0 + ? prefix.end() + : prefix); + } + return new ArrayList<>(inValues); + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + List> splitInValues = splitValues(options); + checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns"); + + List values = splitInValues.get(0); + checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns"); + + ColumnDefinition columnDef = columnDefs.get(0); + ByteBuffer component = validateIndexedValue(columnDef, values.get(0)); + expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component)); + } + + public IN(CType ctype, List columnDefs) + { + super(ctype, columnDefs); + } + + @Override + public boolean isIN() + { + return true; + } + + @Override + public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestrictions) throws InvalidRequestException + { + throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", + getColumnsInCommons(otherRestrictions)); + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + return index.supportsOperator(Operator.IN); + } + + protected abstract List> splitValues(QueryOptions options) throws InvalidRequestException; + } + + /** + * An IN restriction that has a set of terms for in values. + * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)" + */ + public static class InWithValues extends MultiColumnRestriction.IN + { + protected final List values; + + public InWithValues(CType ctype, List columnDefs, List values) + { + super(ctype, columnDefs); + this.values = values; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(values, ksName, functionName); + } + + @Override + public String toString() + { + return String.format("IN(%s)", values); + } + + @Override + protected List> splitValues(QueryOptions options) throws InvalidRequestException + { + List> buffers = new ArrayList<>(values.size()); + for (Term value : values) + { + Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options); + buffers.add(term.getElements()); + } + return buffers; + } + } + + /** + * An IN restriction that uses a single marker for a set of IN values that are tuples. + * For example: "SELECT ... WHERE (a, b, c) IN ?" + */ + public static class InWithMarker extends MultiColumnRestriction.IN + { + protected final AbstractMarker marker; + + public InWithMarker(CType ctype, List columnDefs, AbstractMarker marker) + { + super(ctype, columnDefs); + this.marker = marker; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + @Override + public String toString() + { + return "IN ?"; + } + + @Override + protected List> splitValues(QueryOptions options) throws InvalidRequestException + { + Tuples.InMarker inMarker = (Tuples.InMarker) marker; + Tuples.InValue inValue = inMarker.bind(options); + checkNotNull(inValue, "Invalid null value for IN restriction"); + return inValue.getSplitValues(); + } + } + + public static class Slice extends MultiColumnRestriction + { + private final TermSlice slice; + + public Slice(CType ctype, List columnDefs, Bound bound, boolean inclusive, Term term) + { + this(ctype, columnDefs, TermSlice.newInstance(bound, inclusive, term)); + } + + private Slice(CType ctype, List columnDefs, TermSlice slice) + { + super(ctype, columnDefs); + this.slice = slice; + } + + @Override + public boolean isSlice() + { + return true; + } + + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return Composites.toByteBuffers(boundsAsComposites(b, options)); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + CBuilder builder = ctype.builder(); + Iterator iter = columnDefs.iterator(); + ColumnDefinition firstName = iter.next(); + // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING + // and DESCENDING orders. This stores the bound for the first component; we will re-use it for all following + // components, even if they don't match the first component's reversal/non-reversal. Note that this does *not* + // guarantee correct query results, it just preserves the previous behavior. + Bound firstComponentBound = !firstName.isReversedType() ? bound : bound.reverse(); + + if (!hasBound(firstComponentBound)) + { + Composite prefix = builder.build(); + return Collections.singletonList(builder.remainingCount() > 0 && bound.isEnd() + ? prefix.end() + : prefix); + } + + List vals = componentBounds(firstComponentBound, options); + + ByteBuffer v = checkNotNull(vals.get(firstName.position()), "Invalid null value in condition for column %s", firstName.name); + builder.add(v); + + while (iter.hasNext()) + { + ColumnDefinition def = iter.next(); + if (def.position() >= vals.size()) + break; + + v = checkNotNull(vals.get(def.position()), "Invalid null value in condition for column %s", def.name); + builder.add(v); + } + Composite.EOC eoc = eocFor(this, bound, firstComponentBound); + return Collections.singletonList(builder.build().withEOC(eoc)); + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + throw invalidRequest("Slice restrictions are not supported on indexed columns which are part of a multi column relation"); + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + return slice.isSupportedBy(index); + } + + private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound) + { + if (eocBound.isStart()) + return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END; + + return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START; + } + + @Override + public boolean hasBound(Bound b) + { + return slice.hasBound(b); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName)) + || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName)); + } + + @Override + public boolean isInclusive(Bound b) + { + return slice.isInclusive(b); + } + + @Override + public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException + { + checkTrue(otherRestriction.isSlice(), + "Column \"%s\" cannot be restricted by both an equality and an inequality relation", + getColumnsInCommons(otherRestriction)); + + Slice otherSlice = (Slice) otherRestriction; + + checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START), + "More than one restriction was found for the start bound on %s", + getColumnsInCommons(otherRestriction)); + checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END), + "More than one restriction was found for the end bound on %s", + getColumnsInCommons(otherRestriction)); + + List newColumnDefs = size() >= otherSlice.size() ? columnDefs : otherSlice.columnDefs; + return new Slice(ctype, newColumnDefs, slice.merge(otherSlice.slice)); + } + + @Override + public String toString() + { + return "SLICE" + slice; + } + + /** + * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single + * ByteBuffer to represent the entire bound. + * @param b the bound type + * @param options the query options + * @return one ByteBuffer per-component in the bound + * @throws InvalidRequestException if the components cannot be retrieved + */ + private List componentBounds(Bound b, QueryOptions options) throws InvalidRequestException + { + Tuples.Value value = (Tuples.Value) slice.bound(b).bind(options); + return value.getElements(); + } + } +} diff --git a/cql3/restrictions/PrimaryKeyRestrictions.java b/cql3/restrictions/PrimaryKeyRestrictions.java new file mode 100644 index 0000000000..5f977b750d --- /dev/null +++ b/cql3/restrictions/PrimaryKeyRestrictions.java @@ -0,0 +1,40 @@ +/* + * 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.restrictions; + +import java.util.List; + +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A set of restrictions on a primary key part (partition key or clustering key). + * + */ +interface PrimaryKeyRestrictions extends Restriction, Restrictions +{ + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException; + + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException; + + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException; +} diff --git a/cql3/restrictions/Restriction.java b/cql3/restrictions/Restriction.java new file mode 100644 index 0000000000..d0ed1932e9 --- /dev/null +++ b/cql3/restrictions/Restriction.java @@ -0,0 +1,97 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A restriction/clause on a column. + * The goal of this class being to group all conditions for a column in a SELECT. + */ +public interface Restriction +{ + public boolean isOnToken(); + public boolean isSlice(); + public boolean isEQ(); + public boolean isIN(); + public boolean isContains(); + public boolean isMultiColumn(); + + public List values(QueryOptions options) throws InvalidRequestException; + + /** + * Returns true if one of the restrictions use the specified function. + * + * @param ksName the keyspace name + * @param functionName the function name + * @return true if one of the restrictions use the specified function, false otherwise. + */ + public boolean usesFunction(String ksName, String functionName); + + /** + * Checks if the specified bound is set or not. + * @param b the bound type + * @return true if the specified bound is set, false otherwise + */ + public boolean hasBound(Bound b); + + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException; + + /** + * Checks if the specified bound is inclusive or not. + * @param b the bound type + * @return true if the specified bound is inclusive, false otherwise + */ + public boolean isInclusive(Bound b); + + /** + * Merges this restriction with the specified one. + * + * @param otherRestriction the restriction to merge into this one + * @return the restriction resulting of the merge + * @throws InvalidRequestException if the restrictions cannot be merged + */ + public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException; + + /** + * Check if the restriction is on indexed columns. + * + * @param indexManager the index manager + * @return true if the restriction is on indexed columns, false + */ + public boolean hasSupportingIndex(SecondaryIndexManager indexManager); + + /** + * Adds to the specified list the IndexExpressions corresponding to this Restriction. + * + * @param expressions the list to add the IndexExpressions to + * @param options the query options + * @throws InvalidRequestException if this Restriction cannot be converted into + * IndexExpressions + */ + public void addIndexExpressionTo(List expressions, + QueryOptions options) + throws InvalidRequestException; +} diff --git a/cql3/restrictions/Restrictions.java b/cql3/restrictions/Restrictions.java new file mode 100644 index 0000000000..cf2555ee30 --- /dev/null +++ b/cql3/restrictions/Restrictions.java @@ -0,0 +1,82 @@ +/* + * 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.restrictions; + +import java.util.Collection; +import java.util.List; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Sets of restrictions + */ +interface Restrictions +{ + /** + * Returns the column definitions in position order. + * @return the column definitions in position order. + */ + public Collection getColumnDefs(); + + /** + * Returns true if one of the restrictions use the specified function. + * + * @param ksName the keyspace name + * @param functionName the function name + * @return true if one of the restrictions use the specified function, false otherwise. + */ + public boolean usesFunction(String ksName, String functionName); + + /** + * Check if the restriction is on indexed columns. + * + * @param indexManager the index manager + * @return true if the restriction is on indexed columns, false + */ + public boolean hasSupportingIndex(SecondaryIndexManager indexManager); + + /** + * Adds to the specified list the IndexExpressions corresponding to this Restriction. + * + * @param expressions the list to add the IndexExpressions to + * @param options the query options + * @throws InvalidRequestException if this Restriction cannot be converted into + * IndexExpressions + */ + public void addIndexExpressionTo(List expressions, + QueryOptions options) + throws InvalidRequestException; + + /** + * Checks if this SingleColumnPrimaryKeyRestrictions is empty or not. + * + * @return true if this SingleColumnPrimaryKeyRestrictions is empty, false otherwise. + */ + boolean isEmpty(); + + /** + * Returns the number of columns that have a restriction. + * + * @return the number of columns that have a restriction. + */ + public int size(); +} \ No newline at end of file diff --git a/cql3/restrictions/ReversedPrimaryKeyRestrictions.java b/cql3/restrictions/ReversedPrimaryKeyRestrictions.java new file mode 100644 index 0000000000..9b3316117d --- /dev/null +++ b/cql3/restrictions/ReversedPrimaryKeyRestrictions.java @@ -0,0 +1,77 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; + +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * PrimaryKeyRestrictions decorator that reverse the slices. + */ +final class ReversedPrimaryKeyRestrictions extends ForwardingPrimaryKeyRestrictions +{ + /** + * The decorated restrictions. + */ + private PrimaryKeyRestrictions restrictions; + + public ReversedPrimaryKeyRestrictions(PrimaryKeyRestrictions restrictions) + { + this.restrictions = restrictions; + } + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException + { + return new ReversedPrimaryKeyRestrictions(this.restrictions.mergeWith(restriction)); + } + + @Override + public List bounds(Bound bound, QueryOptions options) throws InvalidRequestException + { + List buffers = restrictions.bounds(bound.reverse(), options); + Collections.reverse(buffers); + return buffers; + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + List composites = restrictions.boundsAsComposites(bound.reverse(), options); + Collections.reverse(composites); + return composites; + } + + @Override + public boolean isInclusive(Bound bound) + { + return this.restrictions.isInclusive(bound.reverse()); + } + + @Override + protected PrimaryKeyRestrictions getDelegate() + { + return this.restrictions; + } +} diff --git a/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java b/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java new file mode 100644 index 0000000000..5c8386e444 --- /dev/null +++ b/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java @@ -0,0 +1,312 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.*; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.composites.CBuilder; +import org.apache.cassandra.db.composites.CType; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.composites.Composite.EOC; +import org.apache.cassandra.db.composites.Composites; +import org.apache.cassandra.db.composites.CompositesBuilder; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +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; + +/** + * A set of single column restrictions on a primary key part (partition key or clustering key). + */ +final class SingleColumnPrimaryKeyRestrictions extends AbstractPrimaryKeyRestrictions +{ + /** + * The composite type. + */ + private final CType ctype; + + /** + * The restrictions. + */ + private final SingleColumnRestrictions restrictions; + + /** + * true if the restrictions are corresponding to an EQ, false otherwise. + */ + private boolean eq; + + /** + * true if the restrictions are corresponding to an IN, false otherwise. + */ + private boolean in; + + /** + * true if the restrictions are corresponding to a Slice, false otherwise. + */ + private boolean slice; + + /** + * true if the restrictions are corresponding to a Contains, false otherwise. + */ + private boolean contains; + + public SingleColumnPrimaryKeyRestrictions(CType ctype) + { + this.ctype = ctype; + this.restrictions = new SingleColumnRestrictions(); + this.eq = true; + } + + private SingleColumnPrimaryKeyRestrictions(SingleColumnPrimaryKeyRestrictions primaryKeyRestrictions, + SingleColumnRestriction restriction) throws InvalidRequestException + { + this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction); + this.ctype = primaryKeyRestrictions.ctype; + + if (!primaryKeyRestrictions.isEmpty()) + { + ColumnDefinition lastColumn = primaryKeyRestrictions.restrictions.lastColumn(); + ColumnDefinition newColumn = restriction.getColumnDef(); + + checkFalse(primaryKeyRestrictions.isSlice() && newColumn.position() > lastColumn.position(), + "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", + newColumn.name, + lastColumn.name); + + if (newColumn.position() < lastColumn.position()) + checkFalse(restriction.isSlice(), + "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", + restrictions.nextColumn(newColumn).name, + newColumn.name); + } + + if (restriction.isSlice() || primaryKeyRestrictions.isSlice()) + this.slice = true; + else if (restriction.isContains() || primaryKeyRestrictions.isContains()) + this.contains = true; + else if (restriction.isIN()) + this.in = true; + else + this.eq = true; + } + + @Override + public boolean isSlice() + { + return slice; + } + + @Override + public boolean isEQ() + { + return eq; + } + + @Override + public boolean isIN() + { + return in; + } + + @Override + public boolean isOnToken() + { + return false; + } + + @Override + public boolean isContains() + { + return contains; + } + + @Override + public boolean isMultiColumn() + { + return false; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return restrictions.usesFunction(ksName, functionName); + } + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException + { + if (restriction.isMultiColumn()) + { + checkTrue(isEmpty(), + "Mixing single column relations and multi column relations on clustering columns is not allowed"); + return (PrimaryKeyRestrictions) restriction; + } + + if (restriction.isOnToken()) + { + checkTrue(isEmpty(), "Columns \"%s\" cannot be restricted by both a normal relation and a token relation", + ((TokenRestriction) restriction).getColumnNamesAsString()); + return (PrimaryKeyRestrictions) restriction; + } + + return new SingleColumnPrimaryKeyRestrictions(this, (SingleColumnRestriction) restriction); + } + + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + CompositesBuilder builder = new CompositesBuilder(ctype.builder(), ctype); + for (ColumnDefinition def : restrictions.getColumnDefs()) + { + Restriction r = restrictions.getRestriction(def); + assert !r.isSlice(); + + List values = r.values(options); + + if (values.isEmpty()) + return Collections.emptyList(); + + builder.addEachElementToAll(values); + checkFalse(builder.containsNull(), "Invalid null value for column %s", def.name); + } + + return builder.build(); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + CBuilder builder = ctype.builder(); + List defs = new ArrayList<>(restrictions.getColumnDefs()); + + CompositesBuilder compositeBuilder = new CompositesBuilder(builder, ctype); + // The end-of-component of composite doesn't depend on whether the + // component type is reversed or not (i.e. the ReversedType is applied + // to the component comparator but not to the end-of-component itself), + // it only depends on whether the slice is reversed + int keyPosition = 0; + for (ColumnDefinition def : defs) + { + // In a restriction, we always have Bound.START < Bound.END for the "base" comparator. + // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter. + // But if the actual comparator itself is reversed, we must inversed the bounds too. + Bound b = !def.isReversedType() ? bound : bound.reverse(); + Restriction r = restrictions.getRestriction(def); + if (keyPosition != def.position() || r.isContains()) + { + EOC eoc = !compositeBuilder.isEmpty() && bound.isEnd() ? EOC.END : EOC.NONE; + return compositeBuilder.buildWithEOC(eoc); + } + if (r.isSlice()) + { + if (!r.hasBound(b)) + { + // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix. + // For composites, if there was preceding component and we're computing the end, we must change the last component + // End-Of-Component, otherwise we would be selecting only one record. + EOC eoc = !compositeBuilder.isEmpty() && bound.isEnd() ? EOC.END : EOC.NONE; + return compositeBuilder.buildWithEOC(eoc); + } + + ByteBuffer value = checkNotNull(r.bounds(b, options).get(0), "Invalid null clustering key part %s", r); + compositeBuilder.addElementToAll(value); + Composite.EOC eoc = eocFor(r, bound, b); + return compositeBuilder.buildWithEOC(eoc); + } + + List values = r.values(options); + + if (values.isEmpty()) + return Collections.emptyList(); + + compositeBuilder.addEachElementToAll(values); + + checkFalse(compositeBuilder.containsNull(), "Invalid null clustering key part %s", def.name); + keyPosition++; + } + // Means no relation at all or everything was an equal + // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection, + // it would be harmless to do it. However, we use this method got the partition key too. And when a query + // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that + // case using the eoc would be bad, since for the random partitioner we have no guarantee that + // prefix.end() will sort after prefix (see #5240). + EOC eoc = bound.isEnd() && compositeBuilder.hasRemaining() ? EOC.END : EOC.NONE; + return compositeBuilder.buildWithEOC(eoc); + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return Composites.toByteBuffers(valuesAsComposites(options)); + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return Composites.toByteBuffers(boundsAsComposites(b, options)); + } + + private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound) + { + if (eocBound.isStart()) + return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END; + + return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START; + } + + @Override + public boolean hasBound(Bound b) + { + if (isEmpty()) + return false; + return restrictions.lastRestriction().hasBound(b); + } + + @Override + public boolean isInclusive(Bound b) + { + if (isEmpty()) + return false; + return restrictions.lastRestriction().isInclusive(b); + } + + @Override + public boolean hasSupportingIndex(SecondaryIndexManager indexManager) + { + return restrictions.hasSupportingIndex(indexManager); + } + + @Override + public void addIndexExpressionTo(List expressions, QueryOptions options) throws InvalidRequestException + { + restrictions.addIndexExpressionTo(expressions, options); + } + + @Override + public Collection getColumnDefs() + { + return restrictions.getColumnDefs(); + } +} \ No newline at end of file diff --git a/cql3/restrictions/SingleColumnRestriction.java b/cql3/restrictions/SingleColumnRestriction.java new file mode 100644 index 0000000000..44a9e68475 --- /dev/null +++ b/cql3/restrictions/SingleColumnRestriction.java @@ -0,0 +1,518 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.AbstractMarker; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.index.SecondaryIndex; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue; +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +public abstract class SingleColumnRestriction extends AbstractRestriction +{ + /** + * The definition of the column to which apply the restriction. + */ + protected final ColumnDefinition columnDef; + + public SingleColumnRestriction(ColumnDefinition columnDef) + { + this.columnDef = columnDef; + } + + /** + * Returns the definition of the column to which is associated this restriction. + * @return the definition of the column to which is associated this restriction + */ + public ColumnDefinition getColumnDef() + { + return columnDef; + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + List values = values(options); + checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns"); + + ByteBuffer value = validateIndexedValue(columnDef, values.get(0)); + expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value)); + } + + @Override + public boolean hasSupportingIndex(SecondaryIndexManager indexManager) + { + SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes); + return index != null && isSupportedBy(index); + } + + /** + * Check if this type of restriction is supported by the specified index. + * + * @param index the Secondary index + * @return true this type of restriction is supported by the specified index, + * false otherwise. + */ + protected abstract boolean isSupportedBy(SecondaryIndex index); + + public static final class EQ extends SingleColumnRestriction + { + private final Term value; + + public EQ(ColumnDefinition columnDef, Term value) + { + super(columnDef); + this.value = value; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(value, ksName, functionName); + } + + public boolean isEQ() + { + return true; + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return Collections.singletonList(value.bindAndGet(options)); + } + + @Override + public String toString() + { + return String.format("EQ(%s)", value); + } + + @Override + public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException + { + throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name); + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + return index.supportsOperator(Operator.EQ); + } + } + + public static abstract class IN extends SingleColumnRestriction + { + public IN(ColumnDefinition columnDef) + { + super(columnDef); + } + + @Override + public final boolean isIN() + { + return true; + } + + @Override + public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException + { + throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name); + } + + @Override + protected final boolean isSupportedBy(SecondaryIndex index) + { + return index.supportsOperator(Operator.IN); + } + } + + public static class InWithValues extends IN + { + protected final List values; + + public InWithValues(ColumnDefinition columnDef, List values) + { + super(columnDef); + this.values = values; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(values, ksName, functionName); + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + List buffers = new ArrayList<>(values.size()); + for (Term value : values) + buffers.add(value.bindAndGet(options)); + return buffers; + } + + @Override + public String toString() + { + return String.format("IN(%s)", values); + } + } + + public static class InWithMarker extends IN + { + protected final AbstractMarker marker; + + public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker) + { + super(columnDef); + this.marker = marker; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + public List values(QueryOptions options) throws InvalidRequestException + { + Term.MultiItemTerminal lval = (Term.MultiItemTerminal) marker.bind(options); + if (lval == null) + throw new InvalidRequestException("Invalid null value for IN restriction"); + return lval.getElements(); + } + + @Override + public String toString() + { + return "IN ?"; + } + } + + public static class Slice extends SingleColumnRestriction + { + private final TermSlice slice; + + public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term) + { + super(columnDef); + slice = TermSlice.newInstance(bound, inclusive, term); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName)) + || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName)); + } + + public boolean isSlice() + { + return true; + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasBound(Bound b) + { + return slice.hasBound(b); + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return Collections.singletonList(slice.bound(b).bindAndGet(options)); + } + + @Override + public boolean isInclusive(Bound b) + { + return slice.isInclusive(b); + } + + @Override + public Restriction mergeWith(Restriction otherRestriction) + throws InvalidRequestException + { + checkTrue(otherRestriction.isSlice(), + "Column \"%s\" cannot be restricted by both an equality and an inequality relation", + columnDef.name); + + SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction; + + checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START), + "More than one restriction was found for the start bound on %s", columnDef.name); + + checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END), + "More than one restriction was found for the end bound on %s", columnDef.name); + + return new Slice(columnDef, slice.merge(otherSlice.slice)); + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + for (Bound b : Bound.values()) + { + if (hasBound(b)) + { + ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options)); + Operator op = slice.getIndexOperator(b); + // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation + // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does + // use the underlying comparator as is. + op = columnDef.isReversedType() ? op.reverse() : op; + expressions.add(new IndexExpression(columnDef.name.bytes, op, value)); + } + } + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + return slice.isSupportedBy(index); + } + + @Override + public String toString() + { + return String.format("SLICE%s", slice); + } + + private Slice(ColumnDefinition columnDef, TermSlice slice) + { + super(columnDef); + this.slice = slice; + } + } + + // This holds CONTAINS, CONTAINS_KEY, and map[key] = value restrictions because we might want to have any combination of them. + public static final class Contains extends SingleColumnRestriction + { + private List values = new ArrayList<>(); // for CONTAINS + private List keys = new ArrayList<>(); // for CONTAINS_KEY + private List entryKeys = new ArrayList<>(); // for map[key] = value + private List entryValues = new ArrayList<>(); // for map[key] = value + + public Contains(ColumnDefinition columnDef, Term t, boolean isKey) + { + super(columnDef); + if (isKey) + keys.add(t); + else + values.add(t); + } + + public Contains(ColumnDefinition columnDef, Term mapKey, Term mapValue) + { + super(columnDef); + entryKeys.add(mapKey); + entryValues.add(mapValue); + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return bindAndGet(values, options); + } + + @Override + public boolean isContains() + { + return true; + } + + @Override + public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException + { + checkTrue(otherRestriction.isContains(), + "Collection column %s can only be restricted by CONTAINS, CONTAINS KEY, or map-entry equality", + getColumnDef().name); + + SingleColumnRestriction.Contains newContains = new Contains(getColumnDef()); + + copyKeysAndValues(this, newContains); + copyKeysAndValues((Contains) otherRestriction, newContains); + + return newContains; + } + + @Override + public void addIndexExpressionTo(List expressions, + QueryOptions options) + throws InvalidRequestException + { + addExpressionsFor(expressions, values(options), Operator.CONTAINS); + addExpressionsFor(expressions, keys(options), Operator.CONTAINS_KEY); + addExpressionsFor(expressions, entries(options), Operator.EQ); + } + + private void addExpressionsFor(List target, List values, + Operator op) throws InvalidRequestException + { + for (ByteBuffer value : values) + { + validateIndexedValue(columnDef, value); + target.add(new IndexExpression(columnDef.name.bytes, op, value)); + } + } + + @Override + protected boolean isSupportedBy(SecondaryIndex index) + { + boolean supported = false; + + if (numberOfValues() > 0) + supported |= index.supportsOperator(Operator.CONTAINS); + + if (numberOfKeys() > 0) + supported |= index.supportsOperator(Operator.CONTAINS_KEY); + + if (numberOfEntries() > 0) + supported |= index.supportsOperator(Operator.EQ); + + return supported; + } + + public int numberOfValues() + { + return values.size(); + } + + public int numberOfKeys() + { + return keys.size(); + } + + public int numberOfEntries() + { + return entryKeys.size(); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(values, ksName, functionName) || usesFunction(keys, ksName, functionName) || + usesFunction(entryKeys, ksName, functionName) || usesFunction(entryValues, ksName, functionName); + } + + @Override + public String toString() + { + return String.format("CONTAINS(values=%s, keys=%s, entryKeys=%s, entryValues=%s)", values, keys, entryKeys, entryValues); + } + + @Override + public boolean hasBound(Bound b) + { + throw new UnsupportedOperationException(); + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isInclusive(Bound b) + { + throw new UnsupportedOperationException(); + } + + private List keys(QueryOptions options) throws InvalidRequestException + { + return bindAndGet(keys, options); + } + + private List entries(QueryOptions options) throws InvalidRequestException + { + List entryBuffers = new ArrayList<>(entryKeys.size()); + List keyBuffers = bindAndGet(entryKeys, options); + List valueBuffers = bindAndGet(entryValues, options); + for (int i = 0; i < entryKeys.size(); i++) + { + if (valueBuffers.get(i) == null) + throw new InvalidRequestException("Unsupported null value for map-entry equality"); + entryBuffers.add(CompositeType.build(keyBuffers.get(i), valueBuffers.get(i))); + } + return entryBuffers; + } + + /** + * Binds the query options to the specified terms and returns the resulting values. + * + * @param terms the terms + * @param options the query options + * @return the value resulting from binding the query options to the specified terms + * @throws InvalidRequestException if a problem occurs while binding the query options + */ + private static List bindAndGet(List terms, QueryOptions options) throws InvalidRequestException + { + List buffers = new ArrayList<>(terms.size()); + for (Term value : terms) + buffers.add(value.bindAndGet(options)); + return buffers; + } + + /** + * Copies the keys and value from the first Contains to the second one. + * + * @param from the Contains to copy from + * @param to the Contains to copy to + */ + private static void copyKeysAndValues(Contains from, Contains to) + { + to.values.addAll(from.values); + to.keys.addAll(from.keys); + to.entryKeys.addAll(from.entryKeys); + to.entryValues.addAll(from.entryValues); + } + + private Contains(ColumnDefinition columnDef) + { + super(columnDef); + } + } +} diff --git a/cql3/restrictions/SingleColumnRestrictions.java b/cql3/restrictions/SingleColumnRestrictions.java new file mode 100644 index 0000000000..b9ffc68fd4 --- /dev/null +++ b/cql3/restrictions/SingleColumnRestrictions.java @@ -0,0 +1,209 @@ +/* + * 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.restrictions; + +import java.util.*; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.Contains; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Sets of single column restrictions. + */ +final class SingleColumnRestrictions implements Restrictions +{ + /** + * The comparator used to sort the Restrictions. + */ + private static final Comparator COLUMN_DEFINITION_COMPARATOR = new Comparator() + { + @Override + public int compare(ColumnDefinition column, ColumnDefinition otherColumn) + { + int value = Integer.compare(column.position(), otherColumn.position()); + return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes); + } + }; + + /** + * The restrictions per column. + */ + protected final TreeMap restrictions; + + public SingleColumnRestrictions() + { + this(new TreeMap(COLUMN_DEFINITION_COMPARATOR)); + } + + protected SingleColumnRestrictions(TreeMap restrictions) + { + this.restrictions = restrictions; + } + + @Override + public final void addIndexExpressionTo(List expressions, + QueryOptions options) throws InvalidRequestException + { + for (Restriction restriction : restrictions.values()) + restriction.addIndexExpressionTo(expressions, options); + } + + @Override + public final Set getColumnDefs() + { + return restrictions.keySet(); + } + + /** + * Returns the restriction associated to the specified column. + * + * @param columnDef the column definition + * @return the restriction associated to the specified column + */ + public Restriction getRestriction(ColumnDefinition columnDef) + { + return restrictions.get(columnDef); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + for (Restriction restriction : restrictions.values()) + if (restriction.usesFunction(ksName, functionName)) + return true; + + return false; + } + + @Override + public final boolean isEmpty() + { + return getColumnDefs().isEmpty(); + } + + @Override + public final int size() + { + return getColumnDefs().size(); + } + + /** + * Adds the specified restriction to this set of restrictions. + * + * @param restriction the restriction to add + * @return the new set of restrictions + * @throws InvalidRequestException if the new restriction cannot be added + */ + public SingleColumnRestrictions addRestriction(SingleColumnRestriction restriction) throws InvalidRequestException + { + TreeMap newRestrictions = new TreeMap<>(this.restrictions); + return new SingleColumnRestrictions(mergeRestrictions(newRestrictions, restriction)); + } + + private static TreeMap mergeRestrictions(TreeMap restrictions, + Restriction restriction) + throws InvalidRequestException + { + ColumnDefinition def = ((SingleColumnRestriction) restriction).getColumnDef(); + Restriction existing = restrictions.get(def); + Restriction newRestriction = mergeRestrictions(existing, restriction); + restrictions.put(def, newRestriction); + return restrictions; + } + + @Override + public final boolean hasSupportingIndex(SecondaryIndexManager indexManager) + { + for (Restriction restriction : restrictions.values()) + { + if (restriction.hasSupportingIndex(indexManager)) + return true; + } + return false; + } + + /** + * Returns the column after the specified one. + * + * @param columnDef the column for which the next one need to be found + * @return the column after the specified one. + */ + ColumnDefinition nextColumn(ColumnDefinition columnDef) + { + return restrictions.tailMap(columnDef, false).firstKey(); + } + + /** + * Returns the definition of the last column. + * + * @return the definition of the last column. + */ + ColumnDefinition lastColumn() + { + return isEmpty() ? null : this.restrictions.lastKey(); + } + + /** + * Returns the last restriction. + * + * @return the last restriction. + */ + Restriction lastRestriction() + { + return isEmpty() ? null : this.restrictions.lastEntry().getValue(); + } + + /** + * Merges the two specified restrictions. + * + * @param restriction the first restriction + * @param otherRestriction the second restriction + * @return the merged restriction + * @throws InvalidRequestException if the two restrictions cannot be merged + */ + private static Restriction mergeRestrictions(Restriction restriction, + Restriction otherRestriction) throws InvalidRequestException + { + return restriction == null ? otherRestriction + : restriction.mergeWith(otherRestriction); + } + + /** + * Checks if the restrictions contains multiple contains, contains key, or map[key] = value. + * + * @return true if the restrictions contains multiple contains, contains key, or , + * map[key] = value; false otherwise + */ + public final boolean hasMultipleContains() + { + int numberOfContains = 0; + for (Restriction restriction : restrictions.values()) + { + if (restriction.isContains()) + { + Contains contains = (Contains) restriction; + numberOfContains += (contains.numberOfValues() + contains.numberOfKeys() + contains.numberOfEntries()); + } + } + return numberOfContains > 1; + } +} \ No newline at end of file diff --git a/cql3/restrictions/StatementRestrictions.java b/cql3/restrictions/StatementRestrictions.java new file mode 100644 index 0000000000..60c7465b54 --- /dev/null +++ b/cql3/restrictions/StatementRestrictions.java @@ -0,0 +1,600 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.*; + +import com.google.common.base.Joiner; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Relation; +import org.apache.cassandra.cql3.VariableSpecifications; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowPosition; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.dht.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.ByteBufferUtil; + +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.cql3.statements.RequestValidations.invalidRequest; + +/** + * The restrictions corresponding to the relations specified on the where-clause of CQL query. + */ +public final class StatementRestrictions +{ + /** + * The Column Family meta data + */ + public final CFMetaData cfm; + + /** + * Restrictions on partitioning columns + */ + private PrimaryKeyRestrictions partitionKeyRestrictions; + + /** + * Restrictions on clustering columns + */ + private PrimaryKeyRestrictions clusteringColumnsRestrictions; + + /** + * Restriction on non-primary key columns (i.e. secondary index restrictions) + */ + private SingleColumnRestrictions nonPrimaryKeyRestrictions; + + /** + * The restrictions used to build the index expressions + */ + private final List indexRestrictions = new ArrayList<>(); + + /** + * true if the secondary index need to be queried, false otherwise + */ + private boolean usesSecondaryIndexing; + + /** + * Specify if the query will return a range of partition keys. + */ + private boolean isKeyRange; + + /** + * Creates a new empty StatementRestrictions. + * + * @param cfm the column family meta data + * @return a new empty StatementRestrictions. + */ + public static StatementRestrictions empty(CFMetaData cfm) + { + return new StatementRestrictions(cfm); + } + + private StatementRestrictions(CFMetaData cfm) + { + this.cfm = cfm; + this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType()); + this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator); + this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions(); + } + + public StatementRestrictions(CFMetaData cfm, + List whereClause, + VariableSpecifications boundNames, + boolean selectsOnlyStaticColumns, + boolean selectACollection) throws InvalidRequestException + { + this.cfm = cfm; + this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType()); + this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator); + this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions(); + + /* + * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ) + * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys + * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem + * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value + * in CQL so far) + */ + for (Relation relation : whereClause) + addRestriction(relation.toRestriction(cfm, boundNames)); + + ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName); + SecondaryIndexManager secondaryIndexManager = cfs.indexManager; + + boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager); + boolean hasQueriableIndex = hasQueriableClusteringColumnIndex + || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager) + || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager); + + // At this point, the select statement if fully constructed, but we still have a few things to validate + processPartitionKeyRestrictions(hasQueriableIndex); + + // Some but not all of the partition key columns have been specified; + // hence we need turn these restrictions into index expressions. + if (usesSecondaryIndexing) + indexRestrictions.add(partitionKeyRestrictions); + + checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(), + "Cannot restrict clustering columns when selecting only static columns"); + + processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection); + + // Covers indexes on the first clustering column (among others). + if (isKeyRange && hasQueriableClusteringColumnIndex) + usesSecondaryIndexing = true; + + if (usesSecondaryIndexing) + { + indexRestrictions.add(clusteringColumnsRestrictions); + } + else if (clusteringColumnsRestrictions.isContains()) + { + indexRestrictions.add(new ForwardingPrimaryKeyRestrictions() { + + @Override + protected PrimaryKeyRestrictions getDelegate() + { + return clusteringColumnsRestrictions; + } + + @Override + public void addIndexExpressionTo(List expressions, QueryOptions options) throws InvalidRequestException + { + List list = new ArrayList<>(); + super.addIndexExpressionTo(list, options); + + for (IndexExpression expression : list) + { + if (expression.isContains() || expression.isContainsKey()) + expressions.add(expression); + } + } + }); + usesSecondaryIndexing = true; + } + // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if + // there is restrictions not covered by the PK. + if (!nonPrimaryKeyRestrictions.isEmpty()) + { + usesSecondaryIndexing = true; + indexRestrictions.add(nonPrimaryKeyRestrictions); + } + + if (usesSecondaryIndexing) + validateSecondaryIndexSelections(selectsOnlyStaticColumns); + } + + private void addRestriction(Restriction restriction) throws InvalidRequestException + { + if (restriction.isMultiColumn()) + clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction); + else if (restriction.isOnToken()) + partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction); + else + addSingleColumnRestriction((SingleColumnRestriction) restriction); + } + + public boolean usesFunction(String ksName, String functionName) + { + return partitionKeyRestrictions.usesFunction(ksName, functionName) + || clusteringColumnsRestrictions.usesFunction(ksName, functionName) + || nonPrimaryKeyRestrictions.usesFunction(ksName, functionName); + } + + private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException + { + ColumnDefinition def = restriction.getColumnDef(); + if (def.isPartitionKey()) + partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction); + else if (def.isClusteringColumn()) + clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction); + else + nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction); + } + + /** + * Checks if the restrictions on the partition key is an IN restriction. + * + * @return true the restrictions on the partition key is an IN restriction, false + * otherwise. + */ + public boolean keyIsInRelation() + { + return partitionKeyRestrictions.isIN(); + } + + /** + * Checks if the query request a range of partition keys. + * + * @return true if the query request a range of partition keys, false otherwise. + */ + public boolean isKeyRange() + { + return this.isKeyRange; + } + + /** + * Checks if the secondary index need to be queried. + * + * @return true if the secondary index need to be queried, false otherwise. + */ + public boolean usesSecondaryIndexing() + { + return this.usesSecondaryIndexing; + } + + private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException + { + // If there is a queriable index, no special condition are required on the other restrictions. + // But we still need to know 2 things: + // - If we don't have a queriable index, is the query ok + // - Is it queriable without 2ndary index, which is always more efficient + // If a component of the partition key is restricted by a relation, all preceding + // components must have a EQ. Only the last partition key component can be in IN relation. + if (partitionKeyRestrictions.isOnToken()) + { + isKeyRange = true; + } + else if (hasPartitionKeyUnrestrictedComponents()) + { + if (!partitionKeyRestrictions.isEmpty()) + { + if (!hasQueriableIndex) + throw invalidRequest("Partition key parts: %s must be restricted as other parts are", + Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents())); + } + + isKeyRange = true; + usesSecondaryIndexing = hasQueriableIndex; + } + } + + /** + * Checks if the partition key has some unrestricted components. + * @return true if the partition key has some unrestricted components, false otherwise. + */ + private boolean hasPartitionKeyUnrestrictedComponents() + { + return partitionKeyRestrictions.size() < cfm.partitionKeyColumns().size(); + } + + /** + * Returns the partition key components that are not restricted. + * @return the partition key components that are not restricted. + */ + private List getPartitionKeyUnrestrictedComponents() + { + List list = new ArrayList<>(cfm.partitionKeyColumns()); + list.removeAll(partitionKeyRestrictions.getColumnDefs()); + return ColumnDefinition.toIdentifiers(list); + } + + /** + * Processes the clustering column restrictions. + * + * @param hasQueriableIndex true if some of the queried data are indexed, false otherwise + * @param selectACollection true if the query should return a collection column + * @throws InvalidRequestException if the request is invalid + */ + private void processClusteringColumnsRestrictions(boolean hasQueriableIndex, + boolean selectACollection) throws InvalidRequestException + { + checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection, + "Cannot restrict clustering columns by IN relations when a collection is selected by the query"); + checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex, + "Cannot restrict clustering columns by a CONTAINS relation without a secondary index"); + + if (hasClusteringColumnsRestriction()) + { + List clusteringColumns = cfm.clusteringColumns(); + List restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs()); + + for (int i = 0, m = restrictedColumns.size(); i < m; i++) + { + ColumnDefinition clusteringColumn = clusteringColumns.get(i); + ColumnDefinition restrictedColumn = restrictedColumns.get(i); + + if (!clusteringColumn.equals(restrictedColumn)) + { + checkTrue(hasQueriableIndex, + "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted", + restrictedColumn.name, + clusteringColumn.name); + + usesSecondaryIndexing = true; // handle gaps and non-keyrange cases. + break; + } + } + } + + if (clusteringColumnsRestrictions.isContains()) + usesSecondaryIndexing = true; + } + + public List getIndexExpressions(QueryOptions options) throws InvalidRequestException + { + if (!usesSecondaryIndexing || indexRestrictions.isEmpty()) + return Collections.emptyList(); + + List expressions = new ArrayList<>(); + for (Restrictions restrictions : indexRestrictions) + restrictions.addIndexExpressionTo(expressions, options); + + return expressions; + } + + /** + * Returns the partition keys for which the data is requested. + * + * @param options the query options + * @return the partition keys for which the data is requested. + * @throws InvalidRequestException if the partition keys cannot be retrieved + */ + public Collection getPartitionKeys(final QueryOptions options) throws InvalidRequestException + { + return partitionKeyRestrictions.values(options); + } + + /** + * Returns the specified bound of the partition key. + * + * @param b the boundary type + * @param options the query options + * @return the specified bound of the partition key + * @throws InvalidRequestException if the boundary cannot be retrieved + */ + private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException + { + // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the + // first + // component of a composite partition key). + if (hasPartitionKeyUnrestrictedComponents()) + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + + // We deal with IN queries for keys in other places, so we know buildBound will return only one result + return partitionKeyRestrictions.bounds(b, options).get(0); + } + + /** + * Returns the partition key bounds. + * + * @param options the query options + * @return the partition key bounds + * @throws InvalidRequestException if the query is invalid + */ + public AbstractBounds getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException + { + IPartitioner p = StorageService.getPartitioner(); + + if (partitionKeyRestrictions.isOnToken()) + { + return getPartitionKeyBoundsForTokenRestrictions(p, options); + } + + return getPartitionKeyBounds(p, options); + } + + private AbstractBounds getPartitionKeyBounds(IPartitioner p, + QueryOptions options) throws InvalidRequestException + { + ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options); + ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options); + + RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p); + RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p); + + if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum()) + return null; + + if (partitionKeyRestrictions.isInclusive(Bound.START)) + { + return partitionKeyRestrictions.isInclusive(Bound.END) + ? new Bounds<>(startKey, finishKey) + : new IncludingExcludingBounds<>(startKey, finishKey); + } + + return partitionKeyRestrictions.isInclusive(Bound.END) + ? new Range<>(startKey, finishKey) + : new ExcludingBounds<>(startKey, finishKey); + } + + private AbstractBounds getPartitionKeyBoundsForTokenRestrictions(IPartitioner p, + QueryOptions options) + throws InvalidRequestException + { + Token startToken = getTokenBound(Bound.START, options, p); + Token endToken = getTokenBound(Bound.END, options, p); + + boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START); + boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END); + + /* + * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring. + * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that + * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty. + * + * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but + * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)). + * Note though that in the case where startToken or endToken is the minimum token, then this special case + * rule should not apply. + */ + int cmp = startToken.compareTo(endToken); + if (!startToken.isMinimum() && !endToken.isMinimum() + && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd)))) + return null; + + RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound(); + RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound(); + + return new Range<>(start, end); + } + + private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException + { + if (!partitionKeyRestrictions.hasBound(b)) + return p.getMinimumToken(); + + ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0); + checkNotNull(value, "Invalid null token value"); + return p.getTokenFactory().fromByteArray(value); + } + + /** + * Checks if the query does not contains any restriction on the clustering columns. + * + * @return true if the query does not contains any restriction on the clustering columns, + * false otherwise. + */ + public boolean hasNoClusteringColumnsRestriction() + { + return clusteringColumnsRestrictions.isEmpty(); + } + + // For non-composite slices, we don't support internally the difference between exclusive and + // inclusive bounds, so we deal with it manually. + public boolean isNonCompositeSliceWithExclusiveBounds() + { + return !cfm.comparator.isCompound() + && clusteringColumnsRestrictions.isSlice() + && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END)); + } + + /** + * Returns the requested clustering columns as Composites. + * + * @param options the query options + * @return the requested clustering columns as Composites + * @throws InvalidRequestException if the query is not valid + */ + public List getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException + { + return clusteringColumnsRestrictions.valuesAsComposites(options); + } + + /** + * Returns the bounds (start or end) of the clustering columns as Composites. + * + * @param b the bound type + * @param options the query options + * @return the bounds (start or end) of the clustering columns as Composites + * @throws InvalidRequestException if the request is not valid + */ + public List getClusteringColumnsBoundsAsComposites(Bound b, + QueryOptions options) throws InvalidRequestException + { + return clusteringColumnsRestrictions.boundsAsComposites(b, options); + } + + /** + * Returns the bounds (start or end) of the clustering columns. + * + * @param b the bound type + * @param options the query options + * @return the bounds (start or end) of the clustering columns + * @throws InvalidRequestException if the request is not valid + */ + public List getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return clusteringColumnsRestrictions.bounds(b, options); + } + + /** + * Checks if the bounds (start or end) of the clustering columns are inclusive. + * + * @param bound the bound type + * @return true if the bounds (start or end) of the clustering columns are inclusive, + * false otherwise + */ + public boolean areRequestedBoundsInclusive(Bound bound) + { + return clusteringColumnsRestrictions.isInclusive(bound); + } + + /** + * Checks if the query returns a range of columns. + * + * @return true if the query returns a range of columns, false otherwise. + */ + public boolean isColumnRange() + { + // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite). + // Static CF (non dense but non composite) never entails a column slice however + if (!cfm.comparator.isDense()) + return cfm.comparator.isCompound(); + + // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about + // CASSANDRA-5762), + // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ. + return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice(); + } + + /** + * Checks if the query need to use filtering. + * @return true if the query need to use filtering, false otherwise. + */ + public boolean needFiltering() + { + int numberOfRestrictedColumns = 0; + for (Restrictions restrictions : indexRestrictions) + numberOfRestrictedColumns += restrictions.size(); + + return numberOfRestrictedColumns > 1 + || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty()) + || (numberOfRestrictedColumns != 0 + && nonPrimaryKeyRestrictions.hasMultipleContains()); + } + + private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException + { + checkFalse(keyIsInRelation(), + "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported"); + // When the user only select static columns, the intent is that we don't query the whole partition but just + // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on + // static columns + // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical. + checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns"); + } + + /** + * Checks if the query has some restrictions on the clustering columns. + * + * @return true if the query has some restrictions on the clustering columns, + * false otherwise. + */ + private boolean hasClusteringColumnsRestriction() + { + return !clusteringColumnsRestrictions.isEmpty(); + } + + public void reverse() + { + clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions); + } +} diff --git a/cql3/restrictions/TermSlice.java b/cql3/restrictions/TermSlice.java new file mode 100644 index 0000000000..3622220453 --- /dev/null +++ b/cql3/restrictions/TermSlice.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.restrictions; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.index.SecondaryIndex; + +final class TermSlice +{ + /** + * The slice boundaries. + */ + private final Term[] bounds; + + /** + * Specifies if a slice boundary is inclusive or not. + */ + private final boolean[] boundInclusive; + + /** + * Creates a new TermSlice with the specified boundaries. + * + * @param start the lower boundary + * @param includeStart true if the lower boundary is inclusive + * @param end the upper boundary + * @param includeEnd true if the upper boundary is inclusive + */ + private TermSlice(Term start, boolean includeStart, Term end, boolean includeEnd) + { + bounds = new Term[]{start, end}; + boundInclusive = new boolean[]{includeStart, includeEnd}; + } + + /** + * Creates a new TermSlice with the specified boundary. + * + * @param bound the boundary type + * @param include true if the boundary is inclusive + * @param term the value + * @return a new TermSlice instance + */ + public static TermSlice newInstance(Bound bound, boolean include, Term term) + { + return bound.isStart() ? new TermSlice(term, include, null, false) + : new TermSlice(null, false, term, include); + } + + /** + * Returns the boundary value. + * + * @param bound the boundary type + * @return the boundary value + */ + public Term bound(Bound bound) + { + return bounds[bound.idx]; + } + + /** + * Checks if this slice has a boundary for the specified type. + * + * @param b the boundary type + * @return true if this slice has a boundary for the specified type, false otherwise. + */ + public boolean hasBound(Bound b) + { + return bounds[b.idx] != null; + } + + /** + * Checks if this slice boundary is inclusive for the specified type. + * + * @param b the boundary type + * @return true if this slice boundary is inclusive for the specified type, + * false otherwise. + */ + public boolean isInclusive(Bound b) + { + return bounds[b.idx] == null || boundInclusive[b.idx]; + } + + /** + * Merges this slice with the specified one. + * + * @param otherSlice the slice to merge to + * @return the new slice resulting from the merge + */ + public TermSlice merge(TermSlice otherSlice) + { + if (hasBound(Bound.START)) + { + assert !otherSlice.hasBound(Bound.START); + + return new TermSlice(bound(Bound.START), + isInclusive(Bound.START), + otherSlice.bound(Bound.END), + otherSlice.isInclusive(Bound.END)); + } + assert !otherSlice.hasBound(Bound.END); + + return new TermSlice(otherSlice.bound(Bound.START), + otherSlice.isInclusive(Bound.START), + bound(Bound.END), + isInclusive(Bound.END)); + } + + @Override + public String toString() + { + return String.format("(%s %s, %s %s)", boundInclusive[0] ? ">=" : ">", + bounds[0], + boundInclusive[1] ? "<=" : "<", + bounds[1]); + } + + /** + * Returns the index operator corresponding to the specified boundary. + * + * @param b the boundary type + * @return the index operator corresponding to the specified boundary + */ + public Operator getIndexOperator(Bound b) + { + if (b.isStart()) + return boundInclusive[b.idx] ? Operator.GTE : Operator.GT; + + return boundInclusive[b.idx] ? Operator.LTE : Operator.LT; + } + + /** + * Check if this TermSlice is supported by the specified index. + * + * @param index the Secondary index + * @return true this type of TermSlice is supported by the specified index, + * false otherwise. + */ + public boolean isSupportedBy(SecondaryIndex index) + { + boolean supported = false; + + if (hasBound(Bound.START)) + supported |= isInclusive(Bound.START) ? index.supportsOperator(Operator.GTE) + : index.supportsOperator(Operator.GT); + if (hasBound(Bound.END)) + supported |= isInclusive(Bound.END) ? index.supportsOperator(Operator.LTE) + : index.supportsOperator(Operator.LT); + + return supported; + } +} diff --git a/cql3/restrictions/TokenRestriction.java b/cql3/restrictions/TokenRestriction.java new file mode 100644 index 0000000000..85d614eded --- /dev/null +++ b/cql3/restrictions/TokenRestriction.java @@ -0,0 +1,224 @@ +/* + * 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.restrictions; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import com.google.common.base.Joiner; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.db.IndexExpression; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +/** + * Restriction using the token function. + */ +public abstract class TokenRestriction extends AbstractPrimaryKeyRestrictions +{ + /** + * The definition of the columns to which apply the token restriction. + */ + protected final List columnDefs; + + /** + * Creates a new TokenRestriction that apply to the specified columns. + * + * @param columnDefs the definition of the columns to which apply the token restriction + */ + public TokenRestriction(List columnDefs) + { + this.columnDefs = columnDefs; + } + + @Override + public boolean isOnToken() + { + return true; + } + + @Override + public Collection getColumnDefs() + { + return columnDefs; + } + + @Override + public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager) + { + return false; + } + + @Override + public void addIndexExpressionTo(List expressions, QueryOptions options) + { + throw new UnsupportedOperationException("Index expression cannot be created for token restriction"); + } + + @Override + public List valuesAsComposites(QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + @Override + public List boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + /** + * Returns the column names as a comma separated String. + * + * @return the column names as a comma separated String. + */ + protected final String getColumnNamesAsString() + { + return Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs)); + } + + public static final class EQ extends TokenRestriction + { + private final Term value; + + public EQ(List columnDefs, Term value) + { + super(columnDefs); + this.value = value; + } + + @Override + public boolean isEQ() + { + return true; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return usesFunction(value, ksName, functionName); + } + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException + { + throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", + Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs))); + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + return Collections.singletonList(value.bindAndGet(options)); + } + } + + public static class Slice extends TokenRestriction + { + private final TermSlice slice; + + public Slice(List columnDefs, Bound bound, boolean inclusive, Term term) + { + super(columnDefs); + slice = TermSlice.newInstance(bound, inclusive, term); + } + + @Override + public boolean isSlice() + { + return true; + } + + @Override + public List values(QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasBound(Bound b) + { + return slice.hasBound(b); + } + + @Override + public List bounds(Bound b, QueryOptions options) throws InvalidRequestException + { + return Collections.singletonList(slice.bound(b).bindAndGet(options)); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName)) + || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName)); + } + + @Override + public boolean isInclusive(Bound b) + { + return slice.isInclusive(b); + } + + @Override + public PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) + throws InvalidRequestException + { + if (!otherRestriction.isOnToken()) + throw invalidRequest("Columns \"%s\" cannot be restricted by both a normal relation and a token relation", + getColumnNamesAsString()); + + if (!otherRestriction.isSlice()) + throw invalidRequest("Columns \"%s\" cannot be restricted by both an equality and an inequality relation", + getColumnNamesAsString()); + + TokenRestriction.Slice otherSlice = (TokenRestriction.Slice) otherRestriction; + + if (hasBound(Bound.START) && otherSlice.hasBound(Bound.START)) + throw invalidRequest("More than one restriction was found for the start bound on %s", + getColumnNamesAsString()); + + if (hasBound(Bound.END) && otherSlice.hasBound(Bound.END)) + throw invalidRequest("More than one restriction was found for the end bound on %s", + getColumnNamesAsString()); + + return new Slice(columnDefs, slice.merge(otherSlice.slice)); + } + + @Override + public String toString() + { + return String.format("SLICE%s", slice); + } + + private Slice(List columnDefs, TermSlice slice) + { + super(columnDefs); + this.slice = slice; + } + } +} diff --git a/cql3/selection/AbstractFunctionSelector.java b/cql3/selection/AbstractFunctionSelector.java new file mode 100644 index 0000000000..2bf169d379 --- /dev/null +++ b/cql3/selection/AbstractFunctionSelector.java @@ -0,0 +1,119 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; + +import org.apache.commons.lang3.text.StrBuilder; + +import org.apache.cassandra.cql3.functions.Function; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +abstract class AbstractFunctionSelector extends Selector +{ + protected final T fun; + + /** + * The list used to pass the function arguments is recycled to avoid the cost of instantiating a new list + * with each function call. + */ + protected final List args; + protected final List argSelectors; + + public static Factory newFactory(final Function fun, final SelectorFactories factories) throws InvalidRequestException + { + if (fun.isAggregate()) + { + if (factories.doesAggregation()) + throw new InvalidRequestException("aggregate functions cannot be used as arguments of aggregate functions"); + } + else + { + if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions()) + throw new InvalidRequestException(String.format("the %s function arguments must be either all aggregates or all none aggregates", + fun.name())); + } + + return new Factory() + { + protected String getColumnName() + { + return new StrBuilder(fun.name().toString()).append('(') + .appendWithSeparators(factories.getColumnNames(), ", ") + .append(')') + .toString(); + } + + protected AbstractType getReturnType() + { + return fun.returnType(); + } + + public boolean usesFunction(String ksName, String functionName) + { + return fun.usesFunction(ksName, functionName); + } + + public Selector newInstance() throws InvalidRequestException + { + return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances()) + : new ScalarFunctionSelector(fun, factories.newInstances()); + } + + public boolean isWritetimeSelectorFactory() + { + return factories.containsWritetimeSelectorFactory(); + } + + public boolean isTTLSelectorFactory() + { + return factories.containsTTLSelectorFactory(); + } + + public boolean isAggregateSelectorFactory() + { + return fun.isAggregate() || factories.containsOnlyAggregateFunctions(); + } + }; + } + + protected AbstractFunctionSelector(T fun, List argSelectors) + { + this.fun = fun; + this.argSelectors = argSelectors; + this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]); + } + + public AbstractType getType() + { + return fun.returnType(); + } + + @Override + public String toString() + { + return new StrBuilder().append(fun.name()) + .append("(") + .appendWithSeparators(argSelectors, ", ") + .append(")") + .toString(); + } +} diff --git a/cql3/selection/AggregateFunctionSelector.java b/cql3/selection/AggregateFunctionSelector.java new file mode 100644 index 0000000000..27a8294bfd --- /dev/null +++ b/cql3/selection/AggregateFunctionSelector.java @@ -0,0 +1,66 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.cql3.functions.AggregateFunction; +import org.apache.cassandra.cql3.functions.Function; +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.exceptions.InvalidRequestException; + +final class AggregateFunctionSelector extends AbstractFunctionSelector +{ + private final AggregateFunction.Aggregate aggregate; + + public boolean isAggregate() + { + return true; + } + + public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + // Aggregation of aggregation is not supported + for (int i = 0, m = argSelectors.size(); i < m; i++) + { + Selector s = argSelectors.get(i); + s.addInput(protocolVersion, rs); + args.set(i, s.getOutput(protocolVersion)); + s.reset(); + } + this.aggregate.addInput(protocolVersion, args); + } + + public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException + { + return aggregate.compute(protocolVersion); + } + + public void reset() + { + aggregate.reset(); + } + + AggregateFunctionSelector(Function fun, List argSelectors) throws InvalidRequestException + { + super((AggregateFunction) fun, argSelectors); + + this.aggregate = this.fun.newAggregate(); + } +} diff --git a/cql3/selection/FieldSelector.java b/cql3/selection/FieldSelector.java new file mode 100644 index 0000000000..76dbb22f58 --- /dev/null +++ b/cql3/selection/FieldSelector.java @@ -0,0 +1,103 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +final class FieldSelector extends Selector +{ + private final UserType type; + private final int field; + private final Selector selected; + + public static Factory newFactory(final UserType type, final int field, final Selector.Factory factory) + { + return new Factory() + { + protected String getColumnName() + { + return String.format("%s.%s", + factory.getColumnName(), + UTF8Type.instance.getString(type.fieldName(field))); + } + + protected AbstractType getReturnType() + { + return type.fieldType(field); + } + + public Selector newInstance() throws InvalidRequestException + { + return new FieldSelector(type, field, factory.newInstance()); + } + + public boolean isAggregateSelectorFactory() + { + return factory.isAggregateSelectorFactory(); + } + }; + } + + public boolean isAggregate() + { + return false; + } + + public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + selected.addInput(protocolVersion, rs); + } + + public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException + { + ByteBuffer value = selected.getOutput(protocolVersion); + if (value == null) + return null; + ByteBuffer[] buffers = type.split(value); + return field < buffers.length ? buffers[field] : null; + } + + public AbstractType getType() + { + return type.fieldType(field); + } + + public void reset() + { + selected.reset(); + } + + @Override + public String toString() + { + return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field))); + } + + private FieldSelector(UserType type, int field, Selector selected) + { + this.type = type; + this.field = field; + this.selected = selected; + } +} \ No newline at end of file diff --git a/cql3/selection/RawSelector.java b/cql3/selection/RawSelector.java new file mode 100644 index 0000000000..7d5543f87b --- /dev/null +++ b/cql3/selection/RawSelector.java @@ -0,0 +1,61 @@ +/* + * 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.selection; + +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.cql3.ColumnIdentifier; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; + +public class RawSelector +{ + public final Selectable.Raw selectable; + public final ColumnIdentifier alias; + + public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias) + { + this.selectable = selectable; + this.alias = alias; + } + + /** + * Converts the specified list of RawSelectors into a list of Selectables. + * + * @param raws the RawSelectors to converts. + * @return a list of Selectables + */ + public static List toSelectables(List raws, final CFMetaData cfm) + { + return Lists.transform(raws, new Function() + { + public Selectable apply(RawSelector raw) + { + return raw.selectable.prepare(cfm); + } + }); + } + + public boolean processesSelection() + { + return selectable.processesSelection(); + } +} diff --git a/cql3/selection/ScalarFunctionSelector.java b/cql3/selection/ScalarFunctionSelector.java new file mode 100644 index 0000000000..bb56bb8dcc --- /dev/null +++ b/cql3/selection/ScalarFunctionSelector.java @@ -0,0 +1,67 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.cql3.functions.Function; +import org.apache.cassandra.cql3.functions.ScalarFunction; +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.exceptions.InvalidRequestException; + +final class ScalarFunctionSelector extends AbstractFunctionSelector +{ + public boolean isAggregate() + { + // We cannot just return true as it is possible to have a scalar function wrapping an aggregation function + if (argSelectors.isEmpty()) + return false; + + return argSelectors.get(0).isAggregate(); + } + + public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + for (int i = 0, m = argSelectors.size(); i < m; i++) + { + Selector s = argSelectors.get(i); + s.addInput(protocolVersion, rs); + } + } + + public void reset() + { + } + + public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException + { + for (int i = 0, m = argSelectors.size(); i < m; i++) + { + Selector s = argSelectors.get(i); + args.set(i, s.getOutput(protocolVersion)); + s.reset(); + } + return fun.execute(protocolVersion, args); + } + + ScalarFunctionSelector(Function fun, List argSelectors) + { + super((ScalarFunction) fun, argSelectors); + } +} \ No newline at end of file diff --git a/cql3/selection/Selectable.java b/cql3/selection/Selectable.java new file mode 100644 index 0000000000..c5ef857202 --- /dev/null +++ b/cql3/selection/Selectable.java @@ -0,0 +1,246 @@ +/* + * 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.selection; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.functions.Function; +import org.apache.cassandra.cql3.functions.FunctionName; +import org.apache.cassandra.cql3.functions.Functions; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.commons.lang3.text.StrBuilder; + +public abstract class Selectable +{ + public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List defs) + throws InvalidRequestException; + + protected static int addAndGetIndex(ColumnDefinition def, List l) + { + int idx = l.indexOf(def); + if (idx < 0) + { + idx = l.size(); + l.add(def); + } + return idx; + } + + public static interface Raw + { + public Selectable prepare(CFMetaData cfm); + + /** + * Returns true if any processing is performed on the selected column. + **/ + public boolean processesSelection(); + } + + public static class WritetimeOrTTL extends Selectable + { + public final ColumnIdentifier id; + public final boolean isWritetime; + + public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime) + { + this.id = id; + this.isWritetime = isWritetime; + } + + @Override + public String toString() + { + return (isWritetime ? "writetime" : "ttl") + "(" + id + ")"; + } + + public Selector.Factory newSelectorFactory(CFMetaData cfm, + List defs) throws InvalidRequestException + { + ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) + throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id)); + if (def.isPrimaryKeyColumn()) + throw new InvalidRequestException( + String.format("Cannot use selection function %s on PRIMARY KEY part %s", + isWritetime ? "writeTime" : "ttl", + def.name)); + if (def.type.isCollection()) + throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", + isWritetime ? "writeTime" : "ttl")); + + return WritetimeOrTTLSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs), isWritetime); + } + + public static class Raw implements Selectable.Raw + { + private final ColumnIdentifier.Raw id; + private final boolean isWritetime; + + public Raw(ColumnIdentifier.Raw id, boolean isWritetime) + { + this.id = id; + this.isWritetime = isWritetime; + } + + public WritetimeOrTTL prepare(CFMetaData cfm) + { + return new WritetimeOrTTL(id.prepare(cfm), isWritetime); + } + + public boolean processesSelection() + { + return true; + } + } + } + + public static class WithFunction extends Selectable + { + public final FunctionName functionName; + public final List args; + + public WithFunction(FunctionName functionName, List args) + { + this.functionName = functionName; + this.args = args; + } + + @Override + public String toString() + { + return new StrBuilder().append(functionName) + .append("(") + .appendWithSeparators(args, ", ") + .append(")") + .toString(); + } + + public Selector.Factory newSelectorFactory(CFMetaData cfm, + List defs) throws InvalidRequestException + { + SelectorFactories factories = + SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs); + + // resolve built-in functions before user defined functions + Function fun = Functions.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName); + if (fun == null) + throw new InvalidRequestException(String.format("Unknown function '%s'", functionName)); + if (fun.returnType() == null) + throw new InvalidRequestException(String.format("Unknown function %s called in selection clause", + functionName)); + + return AbstractFunctionSelector.newFactory(fun, factories); + } + + public static class Raw implements Selectable.Raw + { + private final FunctionName functionName; + private final List args; + + public Raw(FunctionName functionName, List args) + { + this.functionName = functionName; + this.args = args; + } + + public WithFunction prepare(CFMetaData cfm) + { + List preparedArgs = new ArrayList<>(args.size()); + for (Selectable.Raw arg : args) + preparedArgs.add(arg.prepare(cfm)); + return new WithFunction(functionName, preparedArgs); + } + + public boolean processesSelection() + { + return true; + } + } + } + + public static class WithFieldSelection extends Selectable + { + public final Selectable selected; + public final ColumnIdentifier field; + + public WithFieldSelection(Selectable selected, ColumnIdentifier field) + { + this.selected = selected; + this.field = field; + } + + @Override + public String toString() + { + return String.format("%s.%s", selected, field); + } + + public Selector.Factory newSelectorFactory(CFMetaData cfm, + List defs) throws InvalidRequestException + { + Selector.Factory factory = selected.newSelectorFactory(cfm, defs); + AbstractType type = factory.newInstance().getType(); + if (!(type instanceof UserType)) + throw new InvalidRequestException( + String.format("Invalid field selection: %s of type %s is not a user type", + selected, + type.asCQL3Type())); + + UserType ut = (UserType) type; + for (int i = 0; i < ut.size(); i++) + { + if (!ut.fieldName(i).equals(field.bytes)) + continue; + return FieldSelector.newFactory(ut, i, factory); + } + throw new InvalidRequestException(String.format("%s of type %s has no field %s", + selected, + type.asCQL3Type(), + field)); + } + + public static class Raw implements Selectable.Raw + { + private final Selectable.Raw selected; + private final ColumnIdentifier.Raw field; + + public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field) + { + this.selected = selected; + this.field = field; + } + + public WithFieldSelection prepare(CFMetaData cfm) + { + return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm)); + } + + public boolean processesSelection() + { + return true; + } + } + } +} diff --git a/cql3/selection/Selection.java b/cql3/selection/Selection.java new file mode 100644 index 0000000000..58e994a82a --- /dev/null +++ b/cql3/selection/Selection.java @@ -0,0 +1,512 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.ResultSet; +import org.apache.cassandra.db.Cell; +import org.apache.cassandra.db.CounterCell; +import org.apache.cassandra.db.ExpiringCell; +import org.apache.cassandra.db.context.CounterContext; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.utils.ByteBufferUtil; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; + +public abstract class Selection +{ + /** + * A predicate that returns true for static columns. + */ + private static final Predicate STATIC_COLUMN_FILTER = new Predicate() + { + public boolean apply(ColumnDefinition def) + { + return def.isStatic(); + } + }; + + private final CFMetaData cfm; + private final Collection columns; + private final ResultSet.Metadata metadata; + private final boolean collectTimestamps; + private final boolean collectTTLs; + + protected Selection(CFMetaData cfm, + Collection columns, + List metadata, + boolean collectTimestamps, + boolean collectTTLs) + { + this.cfm = cfm; + this.columns = columns; + this.metadata = new ResultSet.Metadata(metadata); + this.collectTimestamps = collectTimestamps; + this.collectTTLs = collectTTLs; + } + + // Overriden by SimpleSelection when appropriate. + public boolean isWildcard() + { + return false; + } + + /** + * Checks if this selection contains static columns. + * @return true if this selection contains static columns, false otherwise; + */ + public boolean containsStaticColumns() + { + if (!cfm.hasStaticColumns()) + return false; + + if (isWildcard()) + return true; + + return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER)); + } + + /** + * Checks if this selection contains only static columns. + * @return true if this selection contains only static columns, false otherwise; + */ + public boolean containsOnlyStaticColumns() + { + if (!containsStaticColumns()) + return false; + + if (isWildcard()) + return false; + + for (ColumnDefinition def : getColumns()) + { + if (!def.isPartitionKey() && !def.isStatic()) + return false; + } + + return true; + } + + /** + * Checks if this selection contains a collection. + * + * @return true if this selection contains a collection, false otherwise. + */ + public boolean containsACollection() + { + if (!cfm.comparator.hasCollections()) + return false; + + for (ColumnDefinition def : getColumns()) + if (def.type.isCollection() && def.type.isMultiCell()) + return true; + + return false; + } + + /** + * Returns the index of the specified column. + * + * @param def the column definition + * @return the index of the specified column + */ + public int indexOf(final ColumnDefinition def) + { + return Iterators.indexOf(getColumns().iterator(), new Predicate() + { + public boolean apply(ColumnDefinition n) + { + return def.name.equals(n.name); + } + }); + } + + public ResultSet.Metadata getResultMetadata() + { + return metadata; + } + + public static Selection wildcard(CFMetaData cfm) + { + List all = new ArrayList(cfm.allColumns().size()); + Iterators.addAll(all, cfm.allColumnsInSelectOrder()); + return new SimpleSelection(cfm, all, true); + } + + public static Selection forColumns(CFMetaData cfm, Collection columns) + { + return new SimpleSelection(cfm, columns, false); + } + + public int addColumnForOrdering(ColumnDefinition c) + { + columns.add(c); + metadata.addNonSerializedColumn(c); + return columns.size() - 1; + } + + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + private static boolean processesSelection(List rawSelectors) + { + for (RawSelector rawSelector : rawSelectors) + { + if (rawSelector.processesSelection()) + return true; + } + return false; + } + + public static Selection fromSelectors(CFMetaData cfm, List rawSelectors) throws InvalidRequestException + { + List defs = new ArrayList(); + + SelectorFactories factories = + SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs); + List metadata = collectMetadata(cfm, rawSelectors, factories); + + return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, metadata, factories) + : new SimpleSelection(cfm, defs, metadata, false); + } + + private static List collectMetadata(CFMetaData cfm, + List rawSelectors, + SelectorFactories factories) + { + List metadata = new ArrayList(rawSelectors.size()); + Iterator iter = rawSelectors.iterator(); + for (Selector.Factory factory : factories) + { + ColumnSpecification colSpec = factory.getColumnSpecification(cfm); + ColumnIdentifier alias = iter.next().alias; + metadata.add(alias == null ? colSpec : colSpec.withAlias(alias)); + } + return metadata; + } + + protected abstract Selectors newSelectors() throws InvalidRequestException; + + /** + * @return the list of CQL3 columns value this SelectionClause needs. + */ + public Collection getColumns() + { + return columns; + } + + public ResultSetBuilder resultSetBuilder(long now) throws InvalidRequestException + { + return new ResultSetBuilder(now); + } + + public abstract boolean isAggregate(); + + /** + * Checks that selectors are either all aggregates or that none of them is. + * + * @param selectors the selectors to test. + * @param messageTemplate the error message template + * @param messageArgs the error message arguments + * @throws InvalidRequestException if some of the selectors are aggregate but not all of them + */ + static void validateSelectors(List selectors, String messageTemplate, Object... messageArgs) + throws InvalidRequestException + { + int aggregates = 0; + for (Selector s : selectors) + if (s.isAggregate()) + ++aggregates; + + if (aggregates != 0 && aggregates != selectors.size()) + throw new InvalidRequestException(String.format(messageTemplate, messageArgs)); + } + + public class ResultSetBuilder + { + private final ResultSet resultSet; + + /** + * As multiple thread can access a Selection instance each ResultSetBuilder will use + * its own Selectors instance. + */ + private final Selectors selectors; + + /* + * We'll build CQL3 row one by one. + * The currentRow is the values for the (CQL3) columns we've fetched. + * We also collect timestamps and ttls for the case where the writetime and + * ttl functions are used. Note that we might collect timestamp and/or ttls + * we don't care about, but since the array below are allocated just once, + * it doesn't matter performance wise. + */ + List current; + final long[] timestamps; + final int[] ttls; + final long now; + + private ResultSetBuilder(long now) throws InvalidRequestException + { + this.resultSet = new ResultSet(getResultMetadata().copy(), new ArrayList>()); + this.selectors = newSelectors(); + this.timestamps = collectTimestamps ? new long[columns.size()] : null; + this.ttls = collectTTLs ? new int[columns.size()] : null; + this.now = now; + } + + public void add(ByteBuffer v) + { + current.add(v); + } + + public void add(Cell c) + { + current.add(isDead(c) ? null : value(c)); + if (timestamps != null) + { + timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp(); + } + if (ttls != null) + { + int ttl = -1; + if (!isDead(c) && c instanceof ExpiringCell) + ttl = c.getLocalDeletionTime() - (int) (now / 1000); + ttls[current.size() - 1] = ttl; + } + } + + private boolean isDead(Cell c) + { + return c == null || !c.isLive(now); + } + + public void newRow(int protocolVersion) throws InvalidRequestException + { + if (current != null) + { + selectors.addInputRow(protocolVersion, this); + if (!selectors.isAggregate()) + { + resultSet.addRow(selectors.getOutputRow(protocolVersion)); + selectors.reset(); + } + } + current = new ArrayList(columns.size()); + } + + public ResultSet build(int protocolVersion) throws InvalidRequestException + { + if (current != null) + { + selectors.addInputRow(protocolVersion, this); + resultSet.addRow(selectors.getOutputRow(protocolVersion)); + selectors.reset(); + current = null; + } + + if (resultSet.isEmpty() && selectors.isAggregate()) + { + resultSet.addRow(selectors.getOutputRow(protocolVersion)); + } + return resultSet; + } + + private ByteBuffer value(Cell c) + { + return (c instanceof CounterCell) + ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value())) + : c.value(); + } + } + + private static interface Selectors + { + public boolean isAggregate(); + + /** + * Adds the current row of the specified ResultSetBuilder. + * + * @param rs the ResultSetBuilder + * @throws InvalidRequestException + */ + public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException; + + public List getOutputRow(int protocolVersion) throws InvalidRequestException; + + public void reset(); + } + + // Special cased selection for when no function is used (this save some allocations). + private static class SimpleSelection extends Selection + { + private final boolean isWildcard; + + public SimpleSelection(CFMetaData cfm, Collection columns, boolean isWildcard) + { + this(cfm, columns, new ArrayList(columns), isWildcard); + } + + public SimpleSelection(CFMetaData cfm, + Collection columns, + List metadata, + boolean isWildcard) + { + /* + * In theory, even a simple selection could have multiple time the same column, so we + * could filter those duplicate out of columns. But since we're very unlikely to + * get much duplicate in practice, it's more efficient not to bother. + */ + super(cfm, columns, metadata, false, false); + this.isWildcard = isWildcard; + } + + @Override + public boolean isWildcard() + { + return isWildcard; + } + + public boolean isAggregate() + { + return false; + } + + protected Selectors newSelectors() + { + return new Selectors() + { + private List current; + + public void reset() + { + current = null; + } + + public List getOutputRow(int protocolVersion) + { + return current; + } + + public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + current = rs.current; + } + + public boolean isAggregate() + { + return false; + } + }; + } + } + + private static class SelectionWithProcessing extends Selection + { + private final SelectorFactories factories; + + public SelectionWithProcessing(CFMetaData cfm, + Collection columns, + List metadata, + SelectorFactories factories) throws InvalidRequestException + { + super(cfm, + columns, + metadata, + factories.containsWritetimeSelectorFactory(), + factories.containsTTLSelectorFactory()); + + this.factories = factories; + + if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions()) + throw new InvalidRequestException("the select clause must either contains only aggregates or none"); + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return factories.usesFunction(ksName, functionName); + } + + @Override + public int addColumnForOrdering(ColumnDefinition c) + { + int index = super.addColumnForOrdering(c); + factories.addSelectorForOrdering(c, index); + return index; + } + + public boolean isAggregate() + { + return factories.containsOnlyAggregateFunctions(); + } + + protected Selectors newSelectors() throws InvalidRequestException + { + return new Selectors() + { + private final List selectors = factories.newInstances(); + + public void reset() + { + for (int i = 0, m = selectors.size(); i < m; i++) + { + selectors.get(i).reset(); + } + } + + public boolean isAggregate() + { + return factories.containsOnlyAggregateFunctions(); + } + + public List getOutputRow(int protocolVersion) throws InvalidRequestException + { + List outputRow = new ArrayList<>(selectors.size()); + + for (int i = 0, m = selectors.size(); i < m; i++) + { + outputRow.add(selectors.get(i).getOutput(protocolVersion)); + } + return outputRow; + } + + public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + for (int i = 0, m = selectors.size(); i < m; i++) + { + selectors.get(i).addInput(protocolVersion, rs); + } + } + }; + } + + } +} diff --git a/cql3/selection/Selector.java b/cql3/selection/Selector.java new file mode 100644 index 0000000000..3ed773bd49 --- /dev/null +++ b/cql3/selection/Selector.java @@ -0,0 +1,170 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.cql3.AssignmentTestable; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A Selector is used to convert the data returned by the storage engine into the data requested by the + * user. They correspond to the <selector> elements from the select clause. + *

Since the introduction of aggregation, Selectors cannot be called anymore by multiple threads + * as they have an internal state.

+ */ +public abstract class Selector implements AssignmentTestable +{ + /** + * A factory for Selector instances. + */ + public static abstract class Factory + { + public boolean usesFunction(String ksName, String functionName) + { + return false; + } + + /** + * Returns the column specification corresponding to the output value of the selector instances created by + * this factory. + * + * @param cfm the column family meta data + * @return a column specification + */ + public final ColumnSpecification getColumnSpecification(CFMetaData cfm) + { + return new ColumnSpecification(cfm.ksName, + cfm.cfName, + new ColumnIdentifier(getColumnName(), true), + getReturnType()); + } + + /** + * Creates a new Selector instance. + * + * @return a new Selector instance + */ + public abstract Selector newInstance() throws InvalidRequestException; + + /** + * Checks if this factory creates selectors instances that creates aggregates. + * + * @return true if this factory creates selectors instances that creates aggregates, + * false otherwise + */ + public boolean isAggregateSelectorFactory() + { + return false; + } + + /** + * Checks if this factory creates writetime selectors instances. + * + * @return true if this factory creates writetime selectors instances, + * false otherwise + */ + public boolean isWritetimeSelectorFactory() + { + return false; + } + + /** + * Checks if this factory creates TTL selectors instances. + * + * @return true if this factory creates TTL selectors instances, + * false otherwise + */ + public boolean isTTLSelectorFactory() + { + return false; + } + + /** + * Returns the name of the column corresponding to the output value of the selector instances created by + * this factory. + * + * @return a column name + */ + protected abstract String getColumnName(); + + /** + * Returns the type of the values returned by the selector instances created by this factory. + * + * @return the selector output type + */ + protected abstract AbstractType getReturnType(); + + } + + /** + * Add the current value from the specified ResultSetBuilder. + * + * @param protocolVersion protocol version used for serialization + * @param rs the ResultSetBuilder + * @throws InvalidRequestException if a problem occurs while add the input value + */ + public abstract void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException; + + /** + * Returns the selector output. + * + * @param protocolVersion protocol version used for serialization + * @return the selector output + * @throws InvalidRequestException if a problem occurs while computing the output value + */ + public abstract ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException; + + /** + * Returns the Selector output type. + * + * @return the Selector output type. + */ + public abstract AbstractType getType(); + + /** + * Checks if this Selector is creating aggregates. + * + * @return true if this Selector is creating aggregates false + * otherwise. + */ + public boolean isAggregate() + { + return false; + } + + /** + * Reset the internal state of this Selector. + */ + public abstract void reset(); + + public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver) + { + if (receiver.type.equals(getType())) + return AssignmentTestable.TestResult.EXACT_MATCH; + else if (receiver.type.isValueCompatibleWith(getType())) + return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE; + else + return AssignmentTestable.TestResult.NOT_ASSIGNABLE; + } +} \ No newline at end of file diff --git a/cql3/selection/SelectorFactories.java b/cql3/selection/SelectorFactories.java new file mode 100644 index 0000000000..3afd1ec4b8 --- /dev/null +++ b/cql3/selection/SelectorFactories.java @@ -0,0 +1,189 @@ +/* + * 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.selection; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.selection.Selector.Factory; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * A set of Selector factories. + */ +final class SelectorFactories implements Iterable +{ + /** + * The Selector factories. + */ + private final List factories; + + /** + * true if one of the factory creates writetime selectors. + */ + private boolean containsWritetimeFactory; + + /** + * true if one of the factory creates TTL selectors. + */ + private boolean containsTTLFactory; + + /** + * The number of factories creating aggregates. + */ + private int numberOfAggregateFactories; + + /** + * Creates a new SelectorFactories instance and collect the column definitions. + * + * @param selectables the Selectables for which the factories must be created + * @param cfm the Column Family Definition + * @param defs the collector parameter for the column definitions + * @return a new SelectorFactories instance + * @throws InvalidRequestException if a problem occurs while creating the factories + */ + public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List selectables, + CFMetaData cfm, + List defs) + throws InvalidRequestException + { + return new SelectorFactories(selectables, cfm, defs); + } + + private SelectorFactories(List selectables, + CFMetaData cfm, + List defs) + throws InvalidRequestException + { + factories = new ArrayList<>(selectables.size()); + + for (Selectable selectable : selectables) + { + Factory factory = selectable.newSelectorFactory(cfm, defs); + containsWritetimeFactory |= factory.isWritetimeSelectorFactory(); + containsTTLFactory |= factory.isTTLSelectorFactory(); + if (factory.isAggregateSelectorFactory()) + ++numberOfAggregateFactories; + factories.add(factory); + } + } + + public boolean usesFunction(String ksName, String functionName) + { + for (Factory factory : factories) + if (factory != null && factory.usesFunction(ksName, functionName)) + return true; + return false; + } + + /** + * Adds a new Selector.Factory for a column that is needed only for ORDER BY purposes. + * @param def the column that is needed for ordering + * @param index the index of the column definition in the Selection's list of columns + */ + public void addSelectorForOrdering(ColumnDefinition def, int index) + { + factories.add(SimpleSelector.newFactory(def.name.toString(), index, def.type)); + } + + /** + * Checks if this SelectorFactories contains only factories for aggregates. + * + * @return true if this SelectorFactories contains only factories for aggregates, + * false otherwise. + */ + public boolean containsOnlyAggregateFunctions() + { + int size = factories.size(); + return size != 0 && numberOfAggregateFactories == size; + } + + /** + * Whether the selector built by this factory does aggregation or not (either directly or in a sub-selector). + * + * @return true if the selector built by this factor does aggregation, false otherwise. + */ + public boolean doesAggregation() + { + return numberOfAggregateFactories > 0; + } + + /** + * Checks if this SelectorFactories contains at least one factory for writetime selectors. + * + * @return true if this SelectorFactories contains at least one factory for writetime + * selectors, false otherwise. + */ + public boolean containsWritetimeSelectorFactory() + { + return containsWritetimeFactory; + } + + /** + * Checks if this SelectorFactories contains at least one factory for TTL selectors. + * + * @return true if this SelectorFactories contains at least one factory for TTL + * selectors, false otherwise. + */ + public boolean containsTTLSelectorFactory() + { + return containsTTLFactory; + } + + /** + * Creates a list of new Selector instances. + * @return a list of new Selector instances. + */ + public List newInstances() throws InvalidRequestException + { + List selectors = new ArrayList<>(factories.size()); + for (Selector.Factory factory : factories) + { + selectors.add(factory.newInstance()); + } + return selectors; + } + + public Iterator iterator() + { + return factories.iterator(); + } + + /** + * Returns the names of the columns corresponding to the output values of the selector instances created by + * these factories. + * + * @return a list of column names + */ + public List getColumnNames() + { + return Lists.transform(factories, new Function() + { + public String apply(Selector.Factory factory) + { + return factory.getColumnName(); + } + }); + } +} diff --git a/cql3/selection/SimpleSelector.java b/cql3/selection/SimpleSelector.java new file mode 100644 index 0000000000..c2edaeddc3 --- /dev/null +++ b/cql3/selection/SimpleSelector.java @@ -0,0 +1,93 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +public final class SimpleSelector extends Selector +{ + private final String columnName; + private final int idx; + private final AbstractType type; + private ByteBuffer current; + + public static Factory newFactory(final String columnName, final int idx, final AbstractType type) + { + return new Factory() + { + @Override + protected String getColumnName() + { + return columnName; + } + + @Override + protected AbstractType getReturnType() + { + return type; + } + + @Override + public Selector newInstance() + { + return new SimpleSelector(columnName, idx, type); + } + }; + } + + @Override + public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException + { + current = rs.current.get(idx); + } + + @Override + public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException + { + return current; + } + + @Override + public void reset() + { + current = null; + } + + @Override + public AbstractType getType() + { + return type; + } + + @Override + public String toString() + { + return columnName; + } + + private SimpleSelector(String columnName, int idx, AbstractType type) + { + this.columnName = columnName; + this.idx = idx; + this.type = type; + } +} \ No newline at end of file diff --git a/cql3/selection/WritetimeOrTTLSelector.java b/cql3/selection/WritetimeOrTTLSelector.java new file mode 100644 index 0000000000..a1ecd3d4d8 --- /dev/null +++ b/cql3/selection/WritetimeOrTTLSelector.java @@ -0,0 +1,108 @@ +/* + * 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.selection; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.utils.ByteBufferUtil; + +final class WritetimeOrTTLSelector extends Selector +{ + private final String columnName; + private final int idx; + private final boolean isWritetime; + private ByteBuffer current; + + public static Factory newFactory(final String columnName, final int idx, final boolean isWritetime) + { + return new Factory() + { + protected String getColumnName() + { + return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", columnName); + } + + protected AbstractType getReturnType() + { + return isWritetime ? LongType.instance : Int32Type.instance; + } + + public Selector newInstance() + { + return new WritetimeOrTTLSelector(columnName, idx, isWritetime); + } + + public boolean isWritetimeSelectorFactory() + { + return isWritetime; + } + + public boolean isTTLSelectorFactory() + { + return !isWritetime; + } + }; + } + + public void addInput(int protocolVersion, ResultSetBuilder rs) + { + if (isWritetime) + { + long ts = rs.timestamps[idx]; + current = ts != Long.MIN_VALUE ? ByteBufferUtil.bytes(ts) : null; + } + else + { + int ttl = rs.ttls[idx]; + current = ttl > 0 ? ByteBufferUtil.bytes(ttl) : null; + } + } + + public ByteBuffer getOutput(int protocolVersion) + { + return current; + } + + public void reset() + { + current = null; + } + + public AbstractType getType() + { + return isWritetime ? LongType.instance : Int32Type.instance; + } + + @Override + public String toString() + { + return columnName; + } + + private WritetimeOrTTLSelector(String columnName, int idx, boolean isWritetime) + { + this.columnName = columnName; + this.idx = idx; + this.isWritetime = isWritetime; + } + +} diff --git a/cql3/statements/AlterKeyspaceStatement.java b/cql3/statements/AlterKeyspaceStatement.java new file mode 100644 index 0000000000..50c3f00c78 --- /dev/null +++ b/cql3/statements/AlterKeyspaceStatement.java @@ -0,0 +1,97 @@ +/* + * 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 org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.transport.Event; + +public class AlterKeyspaceStatement extends SchemaAlteringStatement +{ + private final String name; + private final KSPropDefs attrs; + + public AlterKeyspaceStatement(String name, KSPropDefs attrs) + { + super(); + this.name = name; + this.attrs = attrs; + } + + @Override + public String keyspace() + { + return name; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(name, Permission.ALTER); + } + + public void validate(ClientState state) throws RequestValidationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name); + if (ksm == null) + throw new InvalidRequestException("Unknown keyspace " + name); + if (ksm.name.equalsIgnoreCase(SystemKeyspace.NAME)) + throw new InvalidRequestException("Cannot alter system keyspace"); + + attrs.validate(); + + if (attrs.getReplicationStrategyClass() == null && !attrs.getReplicationOptions().isEmpty()) + { + throw new ConfigurationException("Missing replication strategy class"); + } + else if (attrs.getReplicationStrategyClass() != null) + { + // The strategy is validated through KSMetaData.validate() in announceKeyspaceUpdate below. + // However, for backward compatibility with thrift, this doesn't validate unexpected options yet, + // so doing proper validation here. + AbstractReplicationStrategy.validateReplicationStrategy(name, + AbstractReplicationStrategy.getClass(attrs.getReplicationStrategyClass()), + StorageService.instance.getTokenMetadata(), + DatabaseDescriptor.getEndpointSnitch(), + attrs.getReplicationOptions()); + } + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name); + // In the (very) unlikely case the keyspace was dropped since validate() + if (ksm == null) + throw new InvalidRequestException("Unknown keyspace " + name); + + MigrationManager.announceKeyspaceUpdate(attrs.asKSMetadataUpdate(ksm), isLocalOnly); + return true; + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, keyspace()); + } +} diff --git a/cql3/statements/AlterTableStatement.java b/cql3/statements/AlterTableStatement.java new file mode 100644 index 0000000000..b34052472d --- /dev/null +++ b/cql3/statements/AlterTableStatement.java @@ -0,0 +1,292 @@ +/* + * 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.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; + +import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily; + +public class AlterTableStatement extends SchemaAlteringStatement +{ + public static enum Type + { + ADD, ALTER, DROP, OPTS, RENAME + } + + public final Type oType; + public final CQL3Type.Raw validator; + public final ColumnIdentifier.Raw rawColumnName; + private final CFPropDefs cfProps; + private final Map renames; + private final boolean isStatic; // Only for ALTER ADD + + public AlterTableStatement(CFName name, + Type type, + ColumnIdentifier.Raw columnName, + CQL3Type.Raw validator, + CFPropDefs cfProps, + Map renames, + boolean isStatic) + { + super(name); + this.oType = type; + this.rawColumnName = columnName; + this.validator = validator; // used only for ADD/ALTER commands + this.cfProps = cfProps; + this.renames = renames; + this.isStatic = isStatic; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.ALTER); + } + + public void validate(ClientState state) + { + // validated in announceMigration() + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + CFMetaData meta = validateColumnFamily(keyspace(), columnFamily()); + CFMetaData cfm = meta.copy(); + + CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace()); + ColumnIdentifier columnName = null; + ColumnDefinition def = null; + if (rawColumnName != null) + { + columnName = rawColumnName.prepare(cfm); + def = cfm.getColumnDefinition(columnName); + } + + switch (oType) + { + case ADD: + assert columnName != null; + if (cfm.comparator.isDense()) + throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table"); + + if (isStatic) + { + if (!cfm.comparator.isCompound()) + throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables"); + if (cfm.clusteringColumns().isEmpty()) + throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column"); + } + + if (def != null) + { + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName)); + default: + throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName)); + } + } + + // Cannot re-add a dropped counter column. See #7831. + if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName)) + throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName)); + + AbstractType type = validator.getType(); + if (type.isCollection() && type.isMultiCell()) + { + if (!cfm.comparator.supportCollections()) + throw new InvalidRequestException("Cannot use non-frozen collections with a non-composite PRIMARY KEY"); + if (cfm.isSuper()) + throw new InvalidRequestException("Cannot use non-frozen collections with super column families"); + + // If there used to be a collection column with the same name (that has been dropped), it will + // still be appear in the ColumnToCollectionType because or reasons explained on #6276. The same + // reason mean that we can't allow adding a new collection with that name (see the ticket for details). + if (cfm.comparator.hasCollections()) + { + CollectionType previous = cfm.comparator.collectionType() == null ? null : cfm.comparator.collectionType().defined.get(columnName.bytes); + if (previous != null && !type.isCompatibleWith(previous)) + throw new InvalidRequestException(String.format("Cannot add a collection with the name %s " + + "because a collection with the same name and a different type has already been used in the past", columnName)); + } + + cfm.comparator = cfm.comparator.addOrUpdateCollection(columnName, (CollectionType)type); + } + + Integer componentIndex = cfm.comparator.isCompound() ? cfm.comparator.clusteringPrefixSize() : null; + cfm.addColumnDefinition(isStatic + ? ColumnDefinition.staticDef(cfm, columnName.bytes, type, componentIndex) + : ColumnDefinition.regularDef(cfm, columnName.bytes, type, componentIndex)); + break; + + case ALTER: + assert columnName != null; + if (def == null) + throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily())); + + AbstractType validatorType = validator.getType(); + switch (def.kind) + { + case PARTITION_KEY: + if (validatorType instanceof CounterColumnType) + throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", columnName)); + if (cfm.getKeyValidator() instanceof CompositeType) + { + List> oldTypes = ((CompositeType) cfm.getKeyValidator()).types; + if (!validatorType.isValueCompatibleWith(oldTypes.get(def.position()))) + throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.", + columnName, + oldTypes.get(def.position()).asCQL3Type(), + validator)); + + List> newTypes = new ArrayList>(oldTypes); + newTypes.set(def.position(), validatorType); + cfm.keyValidator(CompositeType.getInstance(newTypes)); + } + else + { + if (!validatorType.isValueCompatibleWith(cfm.getKeyValidator())) + throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.", + columnName, + cfm.getKeyValidator().asCQL3Type(), + validator)); + cfm.keyValidator(validatorType); + } + break; + case CLUSTERING_COLUMN: + AbstractType oldType = cfm.comparator.subtype(def.position()); + // Note that CFMetaData.validateCompatibility already validate the change we're about to do. However, the error message it + // sends is a bit cryptic for a CQL3 user, so validating here for a sake of returning a better error message + // Do note that we need isCompatibleWith here, not just isValueCompatibleWith. + if (!validatorType.isCompatibleWith(oldType)) + throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are not order-compatible.", + columnName, + oldType.asCQL3Type(), + validator)); + + cfm.comparator = cfm.comparator.setSubtype(def.position(), validatorType); + break; + case COMPACT_VALUE: + // See below + if (!validatorType.isValueCompatibleWith(cfm.getDefaultValidator())) + throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.", + columnName, + cfm.getDefaultValidator().asCQL3Type(), + validator)); + cfm.defaultValidator(validatorType); + break; + case REGULAR: + case STATIC: + // Thrift allows to change a column validator so CFMetaData.validateCompatibility will let it slide + // if we change to an incompatible type (contrarily to the comparator case). But we don't want to + // allow it for CQL3 (see #5882) so validating it explicitly here. We only care about value compatibility + // though since we won't compare values (except when there is an index, but that is validated by + // ColumnDefinition already). + if (!validatorType.isValueCompatibleWith(def.type)) + throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.", + columnName, + def.type.asCQL3Type(), + validator)); + + // For collections, if we alter the type, we need to update the comparator too since it includes + // the type too (note that isValueCompatibleWith above has validated that the new type doesn't + // change the underlying sorting order, but we still don't want to have a discrepancy between the type + // in the comparator and the one in the ColumnDefinition as that would be dodgy). + if (validatorType.isCollection() && validatorType.isMultiCell()) + cfm.comparator = cfm.comparator.addOrUpdateCollection(def.name, (CollectionType)validatorType); + + break; + } + // In any case, we update the column definition + cfm.addOrReplaceColumnDefinition(def.withNewType(validatorType)); + break; + + case DROP: + assert columnName != null; + if (!cfm.isCQL3Table()) + throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table"); + if (def == null) + throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily())); + + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName)); + case REGULAR: + case STATIC: + ColumnDefinition toDelete = null; + for (ColumnDefinition columnDef : cfm.regularAndStaticColumns()) + { + if (columnDef.name.equals(columnName)) + toDelete = columnDef; + } + assert toDelete != null; + cfm.removeColumnDefinition(toDelete); + cfm.recordColumnDrop(toDelete); + break; + } + break; + case OPTS: + if (cfProps == null) + throw new InvalidRequestException(String.format("ALTER TABLE WITH invoked, but no parameters found")); + + cfProps.validate(); + cfProps.applyToCFMetadata(cfm); + break; + case RENAME: + for (Map.Entry entry : renames.entrySet()) + { + ColumnIdentifier from = entry.getKey().prepare(cfm); + ColumnIdentifier to = entry.getValue().prepare(cfm); + cfm.renameColumn(from, to); + } + break; + } + + MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly); + return true; + } + + public String toString() + { + return String.format("AlterTableStatement(name=%s, type=%s, column=%s, validator=%s)", + cfName, + oType, + rawColumnName, + validator); + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } +} diff --git a/cql3/statements/AlterTypeStatement.java b/cql3/statements/AlterTypeStatement.java new file mode 100644 index 0000000000..576011fd50 --- /dev/null +++ b/cql3/statements/AlterTypeStatement.java @@ -0,0 +1,348 @@ +/* + * 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.*; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.*; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.composites.CellNames; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; + +public abstract class AlterTypeStatement extends SchemaAlteringStatement +{ + protected final UTName name; + + protected AlterTypeStatement(UTName name) + { + super(); + this.name = name; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!name.hasKeyspace()) + name.setKeyspace(state.getKeyspace()); + + if (name.getKeyspace() == null) + throw new InvalidRequestException("You need to be logged in a keyspace or use a fully qualified user type name"); + } + + protected abstract UserType makeUpdatedType(UserType toUpdate) throws InvalidRequestException; + + public static AlterTypeStatement addition(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type) + { + return new AddOrAlter(name, true, fieldName, type); + } + + public static AlterTypeStatement alter(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type) + { + return new AddOrAlter(name, false, fieldName, type); + } + + public static AlterTypeStatement renames(UTName name, Map renames) + { + return new Renames(name, renames); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(keyspace(), Permission.ALTER); + } + + public void validate(ClientState state) throws RequestValidationException + { + // Validation is left to announceMigration as it's easier to do it while constructing the updated type. + // It doesn't really change anything anyway. + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName()); + } + + @Override + public String keyspace() + { + return name.getKeyspace(); + } + + public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + if (ksm == null) + throw new InvalidRequestException(String.format("Cannot alter type in unknown keyspace %s", name.getKeyspace())); + + UserType toUpdate = ksm.userTypes.getType(name.getUserTypeName()); + // Shouldn't happen, unless we race with a drop + if (toUpdate == null) + throw new InvalidRequestException(String.format("No user type named %s exists.", name)); + + UserType updated = makeUpdatedType(toUpdate); + + // Now, we need to announce the type update to basically change it for new tables using this type, + // but we also need to find all existing user types and CF using it and change them. + MigrationManager.announceTypeUpdate(updated, isLocalOnly); + + for (KSMetaData ksm2 : Schema.instance.getKeyspaceDefinitions()) + { + for (CFMetaData cfm : ksm2.cfMetaData().values()) + { + CFMetaData copy = cfm.copy(); + boolean modified = false; + for (ColumnDefinition def : copy.allColumns()) + modified |= updateDefinition(copy, def, toUpdate.keyspace, toUpdate.name, updated); + if (modified) + MigrationManager.announceColumnFamilyUpdate(copy, false, isLocalOnly); + } + + // Other user types potentially using the updated type + for (UserType ut : ksm2.userTypes.getAllTypes().values()) + { + // Re-updating the type we've just updated would be harmless but useless so we avoid it. + // Besides, we use the occasion to drop the old version of the type if it's a type rename + if (ut.keyspace.equals(toUpdate.keyspace) && ut.name.equals(toUpdate.name)) + { + if (!ut.keyspace.equals(updated.keyspace) || !ut.name.equals(updated.name)) + MigrationManager.announceTypeDrop(ut); + continue; + } + AbstractType upd = updateWith(ut, toUpdate.keyspace, toUpdate.name, updated); + if (upd != null) + MigrationManager.announceTypeUpdate((UserType)upd, isLocalOnly); + } + } + return true; + } + + private static int getIdxOfField(UserType type, ColumnIdentifier field) + { + for (int i = 0; i < type.size(); i++) + if (field.bytes.equals(type.fieldName(i))) + return i; + return -1; + } + + private boolean updateDefinition(CFMetaData cfm, ColumnDefinition def, String keyspace, ByteBuffer toReplace, UserType updated) + { + AbstractType t = updateWith(def.type, keyspace, toReplace, updated); + if (t == null) + return false; + + // We need to update this validator ... + cfm.addOrReplaceColumnDefinition(def.withNewType(t)); + + // ... but if it's part of the comparator or key validator, we need to go update those too. + switch (def.kind) + { + case PARTITION_KEY: + cfm.keyValidator(updateWith(cfm.getKeyValidator(), keyspace, toReplace, updated)); + break; + case CLUSTERING_COLUMN: + cfm.comparator = CellNames.fromAbstractType(updateWith(cfm.comparator.asAbstractType(), keyspace, toReplace, updated), cfm.comparator.isDense()); + break; + default: + // If it's a collection, we still want to modify the comparator because the collection is aliased in it + if (def.type instanceof CollectionType) + cfm.comparator = CellNames.fromAbstractType(updateWith(cfm.comparator.asAbstractType(), keyspace, toReplace, updated), cfm.comparator.isDense()); + } + return true; + } + + // Update the provided type were all instance of a given userType is replaced by a new version + // Note that this methods reaches inside other UserType, CompositeType and CollectionType. + private static AbstractType updateWith(AbstractType type, String keyspace, ByteBuffer toReplace, UserType updated) + { + if (type instanceof UserType) + { + UserType ut = (UserType)type; + + // If it's directly the type we've updated, then just use the new one. + if (keyspace.equals(ut.keyspace) && toReplace.equals(ut.name)) + return updated; + + // Otherwise, check for nesting + List> updatedTypes = updateTypes(ut.fieldTypes(), keyspace, toReplace, updated); + return updatedTypes == null ? null : new UserType(ut.keyspace, ut.name, new ArrayList<>(ut.fieldNames()), updatedTypes); + } + else if (type instanceof CompositeType) + { + CompositeType ct = (CompositeType)type; + List> updatedTypes = updateTypes(ct.types, keyspace, toReplace, updated); + return updatedTypes == null ? null : CompositeType.getInstance(updatedTypes); + } + else if (type instanceof ColumnToCollectionType) + { + ColumnToCollectionType ctct = (ColumnToCollectionType)type; + Map updatedTypes = null; + for (Map.Entry entry : ctct.defined.entrySet()) + { + AbstractType t = updateWith(entry.getValue(), keyspace, toReplace, updated); + if (t == null) + continue; + + if (updatedTypes == null) + updatedTypes = new HashMap<>(ctct.defined); + + updatedTypes.put(entry.getKey(), (CollectionType)t); + } + return updatedTypes == null ? null : ColumnToCollectionType.getInstance(updatedTypes); + } + else if (type instanceof CollectionType) + { + if (type instanceof ListType) + { + AbstractType t = updateWith(((ListType)type).getElementsType(), keyspace, toReplace, updated); + if (t == null) + return null; + return ListType.getInstance(t, type.isMultiCell()); + } + else if (type instanceof SetType) + { + AbstractType t = updateWith(((SetType)type).getElementsType(), keyspace, toReplace, updated); + if (t == null) + return null; + return SetType.getInstance(t, type.isMultiCell()); + } + else + { + assert type instanceof MapType; + MapType mt = (MapType)type; + AbstractType k = updateWith(mt.getKeysType(), keyspace, toReplace, updated); + AbstractType v = updateWith(mt.getValuesType(), keyspace, toReplace, updated); + if (k == null && v == null) + return null; + return MapType.getInstance(k == null ? mt.getKeysType() : k, v == null ? mt.getValuesType() : v, type.isMultiCell()); + } + } + else + { + return null; + } + } + + private static List> updateTypes(List> toUpdate, String keyspace, ByteBuffer toReplace, UserType updated) + { + // But this can also be nested. + List> updatedTypes = null; + for (int i = 0; i < toUpdate.size(); i++) + { + AbstractType t = updateWith(toUpdate.get(i), keyspace, toReplace, updated); + if (t == null) + continue; + + if (updatedTypes == null) + updatedTypes = new ArrayList<>(toUpdate); + + updatedTypes.set(i, t); + } + return updatedTypes; + } + + private static class AddOrAlter extends AlterTypeStatement + { + private final boolean isAdd; + private final ColumnIdentifier fieldName; + private final CQL3Type.Raw type; + + public AddOrAlter(UTName name, boolean isAdd, ColumnIdentifier fieldName, CQL3Type.Raw type) + { + super(name); + this.isAdd = isAdd; + this.fieldName = fieldName; + this.type = type; + } + + private UserType doAdd(UserType toUpdate) throws InvalidRequestException + { + if (getIdxOfField(toUpdate, fieldName) >= 0) + throw new InvalidRequestException(String.format("Cannot add new field %s to type %s: a field of the same name already exists", fieldName, name)); + + List newNames = new ArrayList<>(toUpdate.size() + 1); + newNames.addAll(toUpdate.fieldNames()); + newNames.add(fieldName.bytes); + + List> newTypes = new ArrayList<>(toUpdate.size() + 1); + newTypes.addAll(toUpdate.fieldTypes()); + newTypes.add(type.prepare(keyspace()).getType()); + + return new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes); + } + + private UserType doAlter(UserType toUpdate) throws InvalidRequestException + { + int idx = getIdxOfField(toUpdate, fieldName); + if (idx < 0) + throw new InvalidRequestException(String.format("Unknown field %s in type %s", fieldName, name)); + + AbstractType previous = toUpdate.fieldType(idx); + if (!type.prepare(keyspace()).getType().isCompatibleWith(previous)) + throw new InvalidRequestException(String.format("Type %s is incompatible with previous type %s of field %s in user type %s", type, previous.asCQL3Type(), fieldName, name)); + + List newNames = new ArrayList<>(toUpdate.fieldNames()); + List> newTypes = new ArrayList<>(toUpdate.fieldTypes()); + newTypes.set(idx, type.prepare(keyspace()).getType()); + + return new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes); + } + + protected UserType makeUpdatedType(UserType toUpdate) throws InvalidRequestException + { + return isAdd ? doAdd(toUpdate) : doAlter(toUpdate); + } + } + + private static class Renames extends AlterTypeStatement + { + private final Map renames; + + public Renames(UTName name, Map renames) + { + super(name); + this.renames = renames; + } + + protected UserType makeUpdatedType(UserType toUpdate) throws InvalidRequestException + { + List newNames = new ArrayList<>(toUpdate.fieldNames()); + List> newTypes = new ArrayList<>(toUpdate.fieldTypes()); + + for (Map.Entry entry : renames.entrySet()) + { + ColumnIdentifier from = entry.getKey(); + ColumnIdentifier to = entry.getValue(); + int idx = getIdxOfField(toUpdate, from); + if (idx < 0) + throw new InvalidRequestException(String.format("Unknown field %s in type %s", from, name)); + newNames.set(idx, to.bytes); + } + + UserType updated = new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes); + CreateTypeStatement.checkForDuplicateNames(updated); + return updated; + } + + } +} diff --git a/cql3/statements/AlterUserStatement.java b/cql3/statements/AlterUserStatement.java new file mode 100644 index 0000000000..7a6e96b030 --- /dev/null +++ b/cql3/statements/AlterUserStatement.java @@ -0,0 +1,92 @@ +/* + * 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 org.apache.cassandra.auth.Auth; +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.auth.IAuthenticator; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.UserOptions; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class AlterUserStatement extends AuthenticationStatement +{ + private final String username; + private final UserOptions opts; + private final Boolean superuser; + + public AlterUserStatement(String username, UserOptions opts, Boolean superuser) + { + this.username = username; + this.opts = opts; + this.superuser = superuser; + } + + public void validate(ClientState state) throws RequestValidationException + { + opts.validate(); + + if (superuser == null && opts.isEmpty()) + throw new InvalidRequestException("ALTER USER can't be empty"); + + // validate login here before checkAccess to avoid leaking user existence to anonymous users. + state.ensureNotAnonymous(); + + if (!Auth.isExistingUser(username)) + throw new InvalidRequestException(String.format("User %s doesn't exist", username)); + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + AuthenticatedUser user = state.getUser(); + + boolean isSuper = user.isSuper(); + + if (superuser != null && user.getName().equals(username)) + throw new UnauthorizedException("You aren't allowed to alter your own superuser status"); + + if (superuser != null && !isSuper) + throw new UnauthorizedException("Only superusers are allowed to alter superuser status"); + + if (!user.isSuper() && !user.getName().equals(username)) + throw new UnauthorizedException("You aren't allowed to alter this user"); + + if (!isSuper) + { + for (IAuthenticator.Option option : opts.getOptions().keySet()) + { + if (!DatabaseDescriptor.getAuthenticator().alterableOptions().contains(option)) + throw new UnauthorizedException(String.format("You aren't allowed to alter %s option", option)); + } + } + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + if (!opts.isEmpty()) + DatabaseDescriptor.getAuthenticator().alter(username, opts.getOptions()); + if (superuser != null) + Auth.insertUser(username, superuser.booleanValue()); + return null; + } +} diff --git a/cql3/statements/AuthenticationStatement.java b/cql3/statements/AuthenticationStatement.java new file mode 100644 index 0000000000..b47dd92eef --- /dev/null +++ b/cql3/statements/AuthenticationStatement.java @@ -0,0 +1,54 @@ +/* + * 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 org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement +{ + @Override + public Prepared prepare() + { + return new Prepared(this); + } + + public int getBoundTerms() + { + return 0; + } + + public ResultMessage execute(QueryState state, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + return execute(state.getClientState()); + } + + public abstract ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException; + + public ResultMessage executeInternal(QueryState state, QueryOptions options) + { + // executeInternal is for local query only, thus altering users doesn't make sense and is not supported + throw new UnsupportedOperationException(); + } +} + diff --git a/cql3/statements/AuthorizationStatement.java b/cql3/statements/AuthorizationStatement.java new file mode 100644 index 0000000000..2c7f2cb6fd --- /dev/null +++ b/cql3/statements/AuthorizationStatement.java @@ -0,0 +1,62 @@ +/* + * 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 org.apache.cassandra.auth.DataResource; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public abstract class AuthorizationStatement extends ParsedStatement implements CQLStatement +{ + @Override + public Prepared prepare() + { + return new Prepared(this); + } + + public int getBoundTerms() + { + return 0; + } + + public ResultMessage execute(QueryState state, QueryOptions options) + throws RequestValidationException, RequestExecutionException + { + return execute(state.getClientState()); + } + + public abstract ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException; + + public ResultMessage executeInternal(QueryState state, QueryOptions options) + { + // executeInternal is for local query only, thus altering permission doesn't make sense and is not supported + throw new UnsupportedOperationException(); + } + + public static DataResource maybeCorrectResource(DataResource resource, ClientState state) throws InvalidRequestException + { + if (resource.isColumnFamilyLevel() && resource.getKeyspace() == null) + return DataResource.columnFamily(state.getKeyspace(), resource.getColumnFamily()); + return resource; + } +} diff --git a/cql3/statements/BatchStatement.java b/cql3/statements/BatchStatement.java new file mode 100644 index 0000000000..a0aff3fee2 --- /dev/null +++ b/cql3/statements/BatchStatement.java @@ -0,0 +1,419 @@ +/* + * 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.*; + +import com.google.common.base.Function; +import com.google.common.collect.*; +import org.apache.cassandra.config.DatabaseDescriptor; + +import org.apache.cassandra.tracing.Tracing; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.transport.messages.ResultMessage; + +/** + * A BATCH statement parsed from a CQL query. + * + */ +public class BatchStatement implements CQLStatement +{ + public static enum Type + { + LOGGED, UNLOGGED, COUNTER + } + + private final int boundTerms; + public final Type type; + private final List statements; + private final Attributes attrs; + private final boolean hasConditions; + private static final Logger logger = LoggerFactory.getLogger(BatchStatement.class); + + /** + * Creates a new BatchStatement from a list of statements and a + * Thrift consistency level. + * + * @param type type of the batch + * @param statements a list of UpdateStatements + * @param attrs additional attributes for statement (CL, timestamp, timeToLive) + */ + public BatchStatement(int boundTerms, Type type, List statements, Attributes attrs) + { + boolean hasConditions = false; + for (ModificationStatement statement : statements) + hasConditions |= statement.hasConditions(); + + this.boundTerms = boundTerms; + this.type = type; + this.statements = statements; + this.attrs = attrs; + this.hasConditions = hasConditions; + } + + public boolean usesFunction(String ksName, String functionName) + { + if (attrs.usesFunction(ksName, functionName)) + return true; + for (ModificationStatement statement : statements) + if (statement.usesFunction(ksName, functionName)) + return true; + return false; + } + + public int getBoundTerms() + { + return boundTerms; + } + + public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException + { + for (ModificationStatement statement : statements) + statement.checkAccess(state); + } + + // Validates a prepared batch statement without validating its nested statements. + public void validate() throws InvalidRequestException + { + if (attrs.isTimeToLiveSet()) + throw new InvalidRequestException("Global TTL on the BATCH statement is not supported."); + + boolean timestampSet = attrs.isTimestampSet(); + if (timestampSet) + { + if (hasConditions) + throw new InvalidRequestException("Cannot provide custom timestamp for conditional BATCH"); + if (type == Type.COUNTER) + throw new InvalidRequestException("Cannot provide custom timestamp for counter BATCH"); + } + + boolean hasCounters = false; + boolean hasNonCounters = false; + + for (ModificationStatement statement : statements) + { + if (timestampSet && statement.isCounter()) + throw new InvalidRequestException("Cannot provide custom timestamp for a BATCH containing counters"); + + if (timestampSet && statement.isTimestampSet()) + throw new InvalidRequestException("Timestamp must be set either on BATCH or individual statements"); + + if (type == Type.COUNTER && !statement.isCounter()) + throw new InvalidRequestException("Cannot include non-counter statement in a counter batch"); + + if (type == Type.LOGGED && statement.isCounter()) + throw new InvalidRequestException("Cannot include a counter statement in a logged batch"); + + if (statement.isCounter()) + hasCounters = true; + else + hasNonCounters = true; + } + + if (hasCounters && hasNonCounters) + throw new InvalidRequestException("Counter and non-counter mutations cannot exist in the same batch"); + + if (hasConditions) + { + String ksName = null; + String cfName = null; + for (ModificationStatement stmt : statements) + { + if (ksName != null && (!stmt.keyspace().equals(ksName) || !stmt.columnFamily().equals(cfName))) + throw new InvalidRequestException("Batch with conditions cannot span multiple tables"); + ksName = stmt.keyspace(); + cfName = stmt.columnFamily(); + } + } + } + + // The batch itself will be validated in either Parsed#prepare() - for regular CQL3 batches, + // or in QueryProcessor.processBatch() - for native protocol batches. + public void validate(ClientState state) throws InvalidRequestException + { + for (ModificationStatement statement : statements) + statement.validate(state); + } + + public List getStatements() + { + return statements; + } + + private Collection getMutations(BatchQueryOptions options, boolean local, long now) + throws RequestExecutionException, RequestValidationException + { + Map> mutations = new HashMap<>(); + for (int i = 0; i < statements.size(); i++) + { + ModificationStatement statement = statements.get(i); + QueryOptions statementOptions = options.forStatement(i); + long timestamp = attrs.getTimestamp(now, statementOptions); + addStatementMutations(statement, statementOptions, local, timestamp, mutations); + } + return unzipMutations(mutations); + } + + private Collection unzipMutations(Map> mutations) + { + // The case where all statement where on the same keyspace is pretty common + if (mutations.size() == 1) + return mutations.values().iterator().next().values(); + + List ms = new ArrayList<>(); + for (Map ksMap : mutations.values()) + ms.addAll(ksMap.values()); + return ms; + } + + private void addStatementMutations(ModificationStatement statement, + QueryOptions options, + boolean local, + long now, + Map> mutations) + throws RequestExecutionException, RequestValidationException + { + String ksName = statement.keyspace(); + Map ksMap = mutations.get(ksName); + if (ksMap == null) + { + ksMap = new HashMap<>(); + mutations.put(ksName, ksMap); + } + + // The following does the same than statement.getMutations(), but we inline it here because + // we don't want to recreate mutations every time as this is particularly inefficient when applying + // multiple batch to the same partition (see #6737). + List keys = statement.buildPartitionKeyNames(options); + Composite clusteringPrefix = statement.createClusteringPrefix(options); + UpdateParameters params = statement.makeUpdateParameters(keys, clusteringPrefix, options, local, now); + + for (ByteBuffer key : keys) + { + IMutation mutation = ksMap.get(key); + Mutation mut; + if (mutation == null) + { + mut = new Mutation(ksName, key); + mutation = statement.cfm.isCounter() ? new CounterMutation(mut, options.getConsistency()) : mut; + ksMap.put(key, mutation); + } + else + { + mut = statement.cfm.isCounter() ? ((CounterMutation)mutation).getMutation() : (Mutation)mutation; + } + + statement.addUpdateForKey(mut.addOrGet(statement.cfm), key, clusteringPrefix, params); + } + } + + /** + * Checks batch size to ensure threshold is met. If not, a warning is logged. + * @param cfs ColumnFamilies that will store the batch's mutations. + */ + public static void verifyBatchSize(Iterable cfs) throws InvalidRequestException + { + long size = 0; + long warnThreshold = DatabaseDescriptor.getBatchSizeWarnThreshold(); + long failThreshold = DatabaseDescriptor.getBatchSizeFailThreshold(); + + for (ColumnFamily cf : cfs) + size += cf.dataSize(); + + if (size > warnThreshold) + { + Set ksCfPairs = new HashSet<>(); + for (ColumnFamily cf : cfs) + ksCfPairs.add(cf.metadata().ksName + "." + cf.metadata().cfName); + + String format = "Batch of prepared statements for {} is of size {}, exceeding specified threshold of {} by {}.{}"; + if (size > failThreshold) + { + Tracing.trace(format, new Object[] {ksCfPairs, size, failThreshold, size - failThreshold, " (see batch_size_fail_threshold_in_kb)"}); + logger.error(format, ksCfPairs, size, failThreshold, size - failThreshold, " (see batch_size_fail_threshold_in_kb)"); + throw new InvalidRequestException(String.format("Batch too large")); + } + else if (logger.isWarnEnabled()) + { + logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, ""); + } + } + } + + public ResultMessage execute(QueryState queryState, QueryOptions options) throws RequestExecutionException, RequestValidationException + { + return execute(queryState, BatchQueryOptions.withoutPerStatementVariables(options)); + } + + public ResultMessage execute(QueryState queryState, BatchQueryOptions options) throws RequestExecutionException, RequestValidationException + { + return execute(queryState, options, false, options.getTimestamp(queryState)); + } + + private ResultMessage execute(QueryState queryState, BatchQueryOptions options, boolean local, long now) + throws RequestExecutionException, RequestValidationException + { + if (options.getConsistency() == null) + throw new InvalidRequestException("Invalid empty consistency level"); + if (options.getSerialConsistency() == null) + throw new InvalidRequestException("Invalid empty serial consistency level"); + + if (hasConditions) + return executeWithConditions(options, queryState); + + executeWithoutConditions(getMutations(options, local, now), options.getConsistency()); + return new ResultMessage.Void(); + } + + private void executeWithoutConditions(Collection mutations, ConsistencyLevel cl) throws RequestExecutionException, RequestValidationException + { + // Extract each collection of cfs from it's IMutation and then lazily concatenate all of them into a single Iterable. + Iterable cfs = Iterables.concat(Iterables.transform(mutations, new Function>() + { + public Collection apply(IMutation im) + { + return im.getColumnFamilies(); + } + })); + verifyBatchSize(cfs); + + boolean mutateAtomic = (type == Type.LOGGED && mutations.size() > 1); + StorageProxy.mutateWithTriggers(mutations, cl, mutateAtomic); + } + + private ResultMessage executeWithConditions(BatchQueryOptions options, QueryState state) + throws RequestExecutionException, RequestValidationException + { + long now = state.getTimestamp(); + ByteBuffer key = null; + String ksName = null; + String cfName = null; + CQL3CasRequest casRequest = null; + Set columnsWithConditions = new LinkedHashSet<>(); + + for (int i = 0; i < statements.size(); i++) + { + ModificationStatement statement = statements.get(i); + QueryOptions statementOptions = options.forStatement(i); + long timestamp = attrs.getTimestamp(now, statementOptions); + List pks = statement.buildPartitionKeyNames(statementOptions); + if (pks.size() > 1) + throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)"); + if (key == null) + { + key = pks.get(0); + ksName = statement.cfm.ksName; + cfName = statement.cfm.cfName; + casRequest = new CQL3CasRequest(statement.cfm, key, true); + } + else if (!key.equals(pks.get(0))) + { + throw new InvalidRequestException("Batch with conditions cannot span multiple partitions"); + } + + Composite clusteringPrefix = statement.createClusteringPrefix(statementOptions); + if (statement.hasConditions()) + { + statement.addConditions(clusteringPrefix, casRequest, statementOptions); + // As soon as we have a ifNotExists, we set columnsWithConditions to null so that everything is in the resultSet + if (statement.hasIfNotExistCondition() || statement.hasIfExistCondition()) + columnsWithConditions = null; + else if (columnsWithConditions != null) + Iterables.addAll(columnsWithConditions, statement.getColumnsWithConditions()); + } + casRequest.addRowUpdate(clusteringPrefix, statement, statementOptions, timestamp); + } + + ColumnFamily result = StorageProxy.cas(ksName, cfName, key, casRequest, options.getSerialConsistency(), options.getConsistency(), state.getClientState()); + + return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true, options.forStatement(0))); + } + + public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException + { + assert !hasConditions; + for (IMutation mutation : getMutations(BatchQueryOptions.withoutPerStatementVariables(options), true, queryState.getTimestamp())) + { + // We don't use counters internally. + assert mutation instanceof Mutation; + ((Mutation) mutation).apply(); + } + return null; + } + + public interface BatchVariables + { + public List getVariablesForStatement(int statementInBatch); + } + + public String toString() + { + return String.format("BatchStatement(type=%s, statements=%s)", type, statements); + } + + public static class Parsed extends CFStatement + { + private final Type type; + private final Attributes.Raw attrs; + private final List parsedStatements; + + public Parsed(Type type, Attributes.Raw attrs, List parsedStatements) + { + super(null); + this.type = type; + this.attrs = attrs; + this.parsedStatements = parsedStatements; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + for (ModificationStatement.Parsed statement : parsedStatements) + statement.prepareKeyspace(state); + } + + public ParsedStatement.Prepared prepare() throws InvalidRequestException + { + VariableSpecifications boundNames = getBoundVariables(); + + List statements = new ArrayList<>(parsedStatements.size()); + for (ModificationStatement.Parsed parsed : parsedStatements) + statements.add(parsed.prepare(boundNames)); + + Attributes prepAttrs = attrs.prepare("[batch]", "[batch]"); + prepAttrs.collectMarkerSpecification(boundNames); + + BatchStatement batchStatement = new BatchStatement(boundNames.size(), type, statements, prepAttrs); + batchStatement.validate(); + + return new ParsedStatement.Prepared(batchStatement, boundNames); + } + } +} diff --git a/cql3/statements/Bound.java b/cql3/statements/Bound.java new file mode 100644 index 0000000000..7742642804 --- /dev/null +++ b/cql3/statements/Bound.java @@ -0,0 +1,45 @@ +/* + * 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; + +public enum Bound +{ + START(0), END(1); + + public final int idx; + + Bound(int idx) + { + this.idx = idx; + } + + public Bound reverse() + { + return isStart() ? END : START; + } + + public boolean isStart() + { + return this == START; + } + + public boolean isEnd() + { + return this == END; + } +} diff --git a/cql3/statements/CFPropDefs.java b/cql3/statements/CFPropDefs.java new file mode 100644 index 0000000000..d6d3f72911 --- /dev/null +++ b/cql3/statements/CFPropDefs.java @@ -0,0 +1,218 @@ +/* + * 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.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.cassandra.cache.CachingOptions; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.CFMetaData.SpeculativeRetry; +import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.SyntaxException; +import org.apache.cassandra.io.compress.CompressionParameters; + +public class CFPropDefs extends PropertyDefinitions +{ + public static final String KW_COMMENT = "comment"; + public static final String KW_READREPAIRCHANCE = "read_repair_chance"; + public static final String KW_DCLOCALREADREPAIRCHANCE = "dclocal_read_repair_chance"; + public static final String KW_GCGRACESECONDS = "gc_grace_seconds"; + public static final String KW_MINCOMPACTIONTHRESHOLD = "min_threshold"; + public static final String KW_MAXCOMPACTIONTHRESHOLD = "max_threshold"; + public static final String KW_CACHING = "caching"; + public static final String KW_DEFAULT_TIME_TO_LIVE = "default_time_to_live"; + public static final String KW_MIN_INDEX_INTERVAL = "min_index_interval"; + public static final String KW_MAX_INDEX_INTERVAL = "max_index_interval"; + public static final String KW_SPECULATIVE_RETRY = "speculative_retry"; + public static final String KW_BF_FP_CHANCE = "bloom_filter_fp_chance"; + public static final String KW_MEMTABLE_FLUSH_PERIOD = "memtable_flush_period_in_ms"; + + public static final String KW_COMPACTION = "compaction"; + public static final String KW_COMPRESSION = "compression"; + + public static final String COMPACTION_STRATEGY_CLASS_KEY = "class"; + + public static final Set keywords = new HashSet<>(); + public static final Set obsoleteKeywords = new HashSet<>(); + + static + { + keywords.add(KW_COMMENT); + keywords.add(KW_READREPAIRCHANCE); + keywords.add(KW_DCLOCALREADREPAIRCHANCE); + keywords.add(KW_GCGRACESECONDS); + keywords.add(KW_CACHING); + keywords.add(KW_DEFAULT_TIME_TO_LIVE); + keywords.add(KW_MIN_INDEX_INTERVAL); + keywords.add(KW_MAX_INDEX_INTERVAL); + keywords.add(KW_SPECULATIVE_RETRY); + keywords.add(KW_BF_FP_CHANCE); + keywords.add(KW_COMPACTION); + keywords.add(KW_COMPRESSION); + keywords.add(KW_MEMTABLE_FLUSH_PERIOD); + + obsoleteKeywords.add("index_interval"); + obsoleteKeywords.add("replicate_on_write"); + obsoleteKeywords.add("populate_io_cache_on_flush"); + } + + private Class compactionStrategyClass = null; + + public void validate() throws ConfigurationException, SyntaxException + { + // Skip validation if the comapction strategy class is already set as it means we've alreayd + // prepared (and redoing it would set strategyClass back to null, which we don't want) + if (compactionStrategyClass != null) + return; + + validate(keywords, obsoleteKeywords); + + Map compactionOptions = getCompactionOptions(); + if (!compactionOptions.isEmpty()) + { + String strategy = compactionOptions.get(COMPACTION_STRATEGY_CLASS_KEY); + if (strategy == null) + throw new ConfigurationException("Missing sub-option '" + COMPACTION_STRATEGY_CLASS_KEY + "' for the '" + KW_COMPACTION + "' option."); + + compactionStrategyClass = CFMetaData.createCompactionStrategy(strategy); + compactionOptions.remove(COMPACTION_STRATEGY_CLASS_KEY); + + CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionOptions); + } + + Map compressionOptions = getCompressionOptions(); + if (!compressionOptions.isEmpty()) + { + String sstableCompressionClass = compressionOptions.get(CompressionParameters.SSTABLE_COMPRESSION); + if (sstableCompressionClass == null) + throw new ConfigurationException("Missing sub-option '" + CompressionParameters.SSTABLE_COMPRESSION + "' for the '" + KW_COMPRESSION + "' option."); + + Integer chunkLength = CompressionParameters.DEFAULT_CHUNK_LENGTH; + if (compressionOptions.containsKey(CompressionParameters.CHUNK_LENGTH_KB)) + chunkLength = CompressionParameters.parseChunkLength(compressionOptions.get(CompressionParameters.CHUNK_LENGTH_KB)); + + Map remainingOptions = new HashMap<>(compressionOptions); + remainingOptions.remove(CompressionParameters.SSTABLE_COMPRESSION); + remainingOptions.remove(CompressionParameters.CHUNK_LENGTH_KB); + CompressionParameters cp = new CompressionParameters(sstableCompressionClass, chunkLength, remainingOptions); + cp.validate(); + } + + validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE); + + Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null); + Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null); + if (minIndexInterval != null && minIndexInterval < 1) + throw new ConfigurationException(KW_MIN_INDEX_INTERVAL + " must be greater than 0"); + if (maxIndexInterval != null && minIndexInterval != null && maxIndexInterval < minIndexInterval) + throw new ConfigurationException(KW_MAX_INDEX_INTERVAL + " must be greater than " + KW_MIN_INDEX_INTERVAL); + + SpeculativeRetry.fromString(getString(KW_SPECULATIVE_RETRY, SpeculativeRetry.RetryType.NONE.name())); + } + + public Class getCompactionStrategy() + { + return compactionStrategyClass; + } + + public Map getCompactionOptions() throws SyntaxException + { + Map compactionOptions = getMap(KW_COMPACTION); + if (compactionOptions == null) + return Collections.emptyMap(); + return compactionOptions; + } + + public Map getCompressionOptions() throws SyntaxException + { + Map compressionOptions = getMap(KW_COMPRESSION); + if (compressionOptions == null) + return Collections.emptyMap(); + return compressionOptions; + } + public CachingOptions getCachingOptions() throws SyntaxException, ConfigurationException + { + CachingOptions options = null; + Object val = properties.get(KW_CACHING); + if (val == null) + return null; + else if (val instanceof Map) + options = CachingOptions.fromMap(getMap(KW_CACHING)); + else if (val instanceof String) // legacy syntax + { + options = CachingOptions.fromString(getSimple(KW_CACHING)); + logger.warn("Setting caching options with deprecated syntax."); + } + return options; + } + + public void applyToCFMetadata(CFMetaData cfm) throws ConfigurationException, SyntaxException + { + if (hasProperty(KW_COMMENT)) + cfm.comment(getString(KW_COMMENT, "")); + + cfm.readRepairChance(getDouble(KW_READREPAIRCHANCE, cfm.getReadRepairChance())); + cfm.dcLocalReadRepairChance(getDouble(KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepairChance())); + cfm.gcGraceSeconds(getInt(KW_GCGRACESECONDS, cfm.getGcGraceSeconds())); + int minCompactionThreshold = toInt(KW_MINCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MINCOMPACTIONTHRESHOLD), cfm.getMinCompactionThreshold()); + int maxCompactionThreshold = toInt(KW_MAXCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MAXCOMPACTIONTHRESHOLD), cfm.getMaxCompactionThreshold()); + if (minCompactionThreshold <= 0 || maxCompactionThreshold <= 0) + throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been deprecated, set the compaction option 'enabled' to false instead."); + cfm.minCompactionThreshold(minCompactionThreshold); + cfm.maxCompactionThreshold(maxCompactionThreshold); + cfm.defaultTimeToLive(getInt(KW_DEFAULT_TIME_TO_LIVE, cfm.getDefaultTimeToLive())); + cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(getString(KW_SPECULATIVE_RETRY, cfm.getSpeculativeRetry().toString()))); + cfm.memtableFlushPeriod(getInt(KW_MEMTABLE_FLUSH_PERIOD, cfm.getMemtableFlushPeriod())); + cfm.minIndexInterval(getInt(KW_MIN_INDEX_INTERVAL, cfm.getMinIndexInterval())); + cfm.maxIndexInterval(getInt(KW_MAX_INDEX_INTERVAL, cfm.getMaxIndexInterval())); + + if (compactionStrategyClass != null) + { + cfm.compactionStrategyClass(compactionStrategyClass); + cfm.compactionStrategyOptions(new HashMap<>(getCompactionOptions())); + } + + cfm.bloomFilterFpChance(getDouble(KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance())); + + if (!getCompressionOptions().isEmpty()) + cfm.compressionParameters(CompressionParameters.create(getCompressionOptions())); + CachingOptions cachingOptions = getCachingOptions(); + if (cachingOptions != null) + cfm.caching(cachingOptions); + } + + @Override + public String toString() + { + return String.format("CFPropDefs(%s)", properties.toString()); + } + + private void validateMinimumInt(String field, int minimumValue, int defaultValue) throws SyntaxException, ConfigurationException + { + Integer val = getInt(field, null); + if (val != null && val < minimumValue) + throw new ConfigurationException(String.format("%s cannot be smaller than %s, (default %s)", + field, minimumValue, defaultValue)); + + } +} diff --git a/cql3/statements/CFStatement.java b/cql3/statements/CFStatement.java new file mode 100644 index 0000000000..9b2987cb9d --- /dev/null +++ b/cql3/statements/CFStatement.java @@ -0,0 +1,64 @@ +/* + * 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 org.apache.cassandra.cql3.CFName; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Abstract class for statements that apply on a given column family. + */ +public abstract class CFStatement extends ParsedStatement +{ + protected final CFName cfName; + + protected CFStatement(CFName cfName) + { + this.cfName = cfName; + } + + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!cfName.hasKeyspace()) + { + // XXX: We explicitely only want to call state.getKeyspace() in this case, as we don't want to throw + // if not logged in any keyspace but a keyspace is explicitely set on the statement. So don't move + // the call outside the 'if' or replace the method by 'prepareKeyspace(state.getKeyspace())' + cfName.setKeyspace(state.getKeyspace(), true); + } + } + + // Only for internal calls, use the version with ClientState for user queries + public void prepareKeyspace(String keyspace) + { + if (!cfName.hasKeyspace()) + cfName.setKeyspace(keyspace, true); + } + + public String keyspace() + { + assert cfName.hasKeyspace() : "The statement hasn't be prepared correctly"; + return cfName.getKeyspace(); + } + + public String columnFamily() + { + return cfName.getColumnFamily(); + } +} diff --git a/cql3/statements/CQL3CasRequest.java b/cql3/statements/CQL3CasRequest.java new file mode 100644 index 0000000000..4ff9c27c97 --- /dev/null +++ b/cql3/statements/CQL3CasRequest.java @@ -0,0 +1,265 @@ +/* + * 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.*; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.filter.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.CASRequest; +import org.apache.cassandra.utils.Pair; + +/** + * Processed CAS conditions and update on potentially multiple rows of the same partition. + */ +public class CQL3CasRequest implements CASRequest +{ + private final CFMetaData cfm; + private final ByteBuffer key; + private final long now; + private final boolean isBatch; + + // We index RowCondition by the prefix of the row they applied to for 2 reasons: + // 1) this allows to keep things sorted to build the ColumnSlice array below + // 2) this allows to detect when contradictory conditions are set (not exists with some other conditions on the same row) + private final SortedMap conditions; + + private final List updates = new ArrayList<>(); + + public CQL3CasRequest(CFMetaData cfm, ByteBuffer key, boolean isBatch) + { + this.cfm = cfm; + // When checking if conditions apply, we want to use a fixed reference time for a whole request to check + // for expired cells. Note that this is unrelated to the cell timestamp. + this.now = System.currentTimeMillis(); + this.key = key; + this.conditions = new TreeMap<>(cfm.comparator); + this.isBatch = isBatch; + } + + public void addRowUpdate(Composite prefix, ModificationStatement stmt, QueryOptions options, long timestamp) + { + updates.add(new RowUpdate(prefix, stmt, options, timestamp)); + } + + public void addNotExist(Composite prefix) throws InvalidRequestException + { + RowCondition previous = conditions.put(prefix, new NotExistCondition(prefix, now)); + if (previous != null && !(previous instanceof NotExistCondition)) + { + // these should be prevented by the parser, but it doesn't hurt to check + if (previous instanceof ExistCondition) + throw new InvalidRequestException("Cannot mix IF EXISTS and IF NOT EXISTS conditions for the same row"); + else + throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row"); + } + } + + public void addExist(Composite prefix) throws InvalidRequestException + { + RowCondition previous = conditions.put(prefix, new ExistCondition(prefix, now)); + // this should be prevented by the parser, but it doesn't hurt to check + if (previous instanceof NotExistCondition) + throw new InvalidRequestException("Cannot mix IF EXISTS and IF NOT EXISTS conditions for the same row"); + } + + public void addConditions(Composite prefix, Collection conds, QueryOptions options) throws InvalidRequestException + { + RowCondition condition = conditions.get(prefix); + if (condition == null) + { + condition = new ColumnsConditions(prefix, now); + conditions.put(prefix, condition); + } + else if (!(condition instanceof ColumnsConditions)) + { + throw new InvalidRequestException("Cannot mix IF conditions and IF NOT EXISTS for the same row"); + } + ((ColumnsConditions)condition).addConditions(conds, options); + } + + public IDiskAtomFilter readFilter() + { + assert !conditions.isEmpty(); + ColumnSlice[] slices = new ColumnSlice[conditions.size()]; + int i = 0; + // We always read CQL rows entirely as on CAS failure we want to be able to distinguish between "row exists + // but all values for which there were conditions are null" and "row doesn't exists", and we can't rely on the + // row marker for that (see #6623) + for (Composite prefix : conditions.keySet()) + slices[i++] = prefix.slice(); + + int toGroup = cfm.comparator.isDense() ? -1 : cfm.clusteringColumns().size(); + slices = ColumnSlice.deoverlapSlices(slices, cfm.comparator); + assert ColumnSlice.validateSlices(slices, cfm.comparator, false); + return new SliceQueryFilter(slices, false, slices.length, toGroup); + } + + public boolean appliesTo(ColumnFamily current) throws InvalidRequestException + { + for (RowCondition condition : conditions.values()) + { + if (!condition.appliesTo(current)) + return false; + } + return true; + } + + public ColumnFamily makeUpdates(ColumnFamily current) throws InvalidRequestException + { + ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfm); + for (RowUpdate upd : updates) + upd.applyUpdates(current, cf); + + if (isBatch) + BatchStatement.verifyBatchSize(Collections.singleton(cf)); + + return cf; + } + + /** + * Due to some operation on lists, we can't generate the update that a given Modification statement does before + * we get the values read by the initial read of Paxos. A RowUpdate thus just store the relevant information + * (include the statement iself) to generate those updates. We'll have multiple RowUpdate for a Batch, otherwise + * we'll have only one. + */ + private class RowUpdate + { + private final Composite rowPrefix; + private final ModificationStatement stmt; + private final QueryOptions options; + private final long timestamp; + + private RowUpdate(Composite rowPrefix, ModificationStatement stmt, QueryOptions options, long timestamp) + { + this.rowPrefix = rowPrefix; + this.stmt = stmt; + this.options = options; + this.timestamp = timestamp; + } + + public void applyUpdates(ColumnFamily current, ColumnFamily updates) throws InvalidRequestException + { + Map map = null; + if (stmt.requiresRead()) + { + // Uses the "current" values read by Paxos for lists operation that requires a read + Iterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(current.iterator(new ColumnSlice[]{ rowPrefix.slice() })); + if (iter.hasNext()) + { + map = Collections.singletonMap(key, iter.next()); + assert !iter.hasNext() : "We shoudn't be updating more than one CQL row per-ModificationStatement"; + } + } + + UpdateParameters params = new UpdateParameters(cfm, options, timestamp, stmt.getTimeToLive(options), map); + stmt.addUpdateForKey(updates, key, rowPrefix, params); + } + } + + private static abstract class RowCondition + { + public final Composite rowPrefix; + protected final long now; + + protected RowCondition(Composite rowPrefix, long now) + { + this.rowPrefix = rowPrefix; + this.now = now; + } + + public abstract boolean appliesTo(ColumnFamily current) throws InvalidRequestException; + } + + private static class NotExistCondition extends RowCondition + { + private NotExistCondition(Composite rowPrefix, long now) + { + super(rowPrefix, now); + } + + public boolean appliesTo(ColumnFamily current) + { + if (current == null) + return true; + + Iterator iter = current.iterator(new ColumnSlice[]{ rowPrefix.slice() }); + while (iter.hasNext()) + if (iter.next().isLive(now)) + return false; + return true; + } + } + + private static class ExistCondition extends RowCondition + { + private ExistCondition(Composite rowPrefix, long now) + { + super (rowPrefix, now); + } + + public boolean appliesTo(ColumnFamily current) + { + if (current == null) + return false; + + Iterator iter = current.iterator(new ColumnSlice[]{ rowPrefix.slice() }); + while (iter.hasNext()) + if (iter.next().isLive(now)) + return true; + return false; + } + } + + private static class ColumnsConditions extends RowCondition + { + private final Multimap, ColumnCondition.Bound> conditions = HashMultimap.create(); + + private ColumnsConditions(Composite rowPrefix, long now) + { + super(rowPrefix, now); + } + + public void addConditions(Collection conds, QueryOptions options) throws InvalidRequestException + { + for (ColumnCondition condition : conds) + { + ColumnCondition.Bound current = condition.bind(options); + conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current); + } + } + + public boolean appliesTo(ColumnFamily current) throws InvalidRequestException + { + if (current == null) + return conditions.isEmpty(); + + for (ColumnCondition.Bound condition : conditions.values()) + if (!condition.appliesTo(rowPrefix, current, now)) + return false; + return true; + } + } +} diff --git a/cql3/statements/CreateAggregateStatement.java b/cql3/statements/CreateAggregateStatement.java new file mode 100644 index 0000000000..9816e5848c --- /dev/null +++ b/cql3/statements/CreateAggregateStatement.java @@ -0,0 +1,194 @@ +/* + * 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.List; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.cql3.functions.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** + * A CREATE AGGREGATE statement parsed from a CQL query. + */ +public final class CreateAggregateStatement extends SchemaAlteringStatement +{ + private final boolean orReplace; + private final boolean ifNotExists; + private FunctionName functionName; + private String stateFunc; + private String finalFunc; + private final CQL3Type.Raw stateTypeRaw; + + private final List argRawTypes; + private final Term.Raw ival; + + public CreateAggregateStatement(FunctionName functionName, + List argRawTypes, + String stateFunc, + CQL3Type.Raw stateType, + String finalFunc, + Term.Raw ival, + boolean orReplace, + boolean ifNotExists) + { + this.functionName = functionName; + this.argRawTypes = argRawTypes; + this.stateFunc = stateFunc; + this.finalFunc = finalFunc; + this.stateTypeRaw = stateType; + this.ival = ival; + this.orReplace = orReplace; + this.ifNotExists = ifNotExists; + } + + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!functionName.hasKeyspace() && state.getRawKeyspace() != null) + functionName = new FunctionName(state.getKeyspace(), functionName.name); + + if (!functionName.hasKeyspace()) + throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session"); + + ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + // TODO CASSANDRA-7557 (function DDL permission) + + state.hasKeyspaceAccess(functionName.keyspace, Permission.CREATE); + } + + public void validate(ClientState state) throws InvalidRequestException + { + if (ifNotExists && orReplace) + throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives"); + + if (Schema.instance.getKSMetaData(functionName.keyspace) == null) + throw new InvalidRequestException(String.format("Cannot add aggregate '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace)); + } + + public Event.SchemaChange changeEvent() + { + return null; + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + List> argTypes = new ArrayList<>(argRawTypes.size()); + for (CQL3Type.Raw rawType : argRawTypes) + argTypes.add(rawType.prepare(functionName.keyspace).getType()); + + FunctionName stateFuncName = new FunctionName(functionName.keyspace, stateFunc); + FunctionName finalFuncName; + + ScalarFunction fFinal = null; + AbstractType stateType = stateTypeRaw.prepare(functionName.keyspace).getType(); + Function f = Functions.find(stateFuncName, stateArguments(stateType, argTypes)); + if (!(f instanceof ScalarFunction)) + throw new InvalidRequestException("State function " + stateFuncSig(stateFuncName, stateTypeRaw, argRawTypes) + " does not exist or is not a scalar function"); + ScalarFunction fState = (ScalarFunction)f; + + AbstractType returnType; + if (finalFunc != null) + { + finalFuncName = new FunctionName(functionName.keyspace, finalFunc); + f = Functions.find(finalFuncName, Collections.>singletonList(stateType)); + if (!(f instanceof ScalarFunction)) + throw new InvalidRequestException("Final function " + finalFuncName + "(" + stateTypeRaw + ") does not exist"); + fFinal = (ScalarFunction) f; + returnType = fFinal.returnType(); + } + else + { + returnType = fState.returnType(); + if (!returnType.equals(stateType)) + throw new InvalidRequestException("State function " + stateFuncSig(stateFuncName, stateTypeRaw, argRawTypes) + " return type must be the same as the first argument type (if no final function is used)"); + } + + Function old = Functions.find(functionName, argTypes); + if (old != null) + { + if (ifNotExists) + return false; + if (!orReplace) + throw new InvalidRequestException(String.format("Function %s already exists", old)); + if (!(old instanceof AggregateFunction)) + throw new InvalidRequestException(String.format("Aggregate %s can only replace an aggregate", old)); + + // Means we're replacing the function. We still need to validate that 1) it's not a native function and 2) that the return type + // matches (or that could break existing code badly) + if (old.isNative()) + throw new InvalidRequestException(String.format("Cannot replace native aggregate %s", old)); + if (!old.returnType().isValueCompatibleWith(returnType)) + throw new InvalidRequestException(String.format("Cannot replace aggregate %s, the new return type %s is not compatible with the return type %s of existing function", + functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type())); + } + + ByteBuffer initcond = null; + if (ival != null) + { + ColumnSpecification receiver = new ColumnSpecification(functionName.keyspace, "--dummy--", new ColumnIdentifier("(aggregate_initcond)", true), stateType); + initcond = ival.prepare(functionName.keyspace, receiver).bindAndGet(QueryOptions.DEFAULT); + } + + UDAggregate udAggregate = new UDAggregate(functionName, argTypes, returnType, + fState, + fFinal, + initcond); + + MigrationManager.announceNewAggregate(udAggregate, isLocalOnly); + + return true; + } + + private String stateFuncSig(FunctionName stateFuncName, CQL3Type.Raw stateTypeRaw, List argRawTypes) + { + StringBuilder sb = new StringBuilder(); + sb.append(stateFuncName.toString()).append('(').append(stateTypeRaw); + for (CQL3Type.Raw argRawType : argRawTypes) + sb.append(", ").append(argRawType); + sb.append(')'); + return sb.toString(); + } + + private List> stateArguments(AbstractType stateType, List> argTypes) + { + List> r = new ArrayList<>(argTypes.size() + 1); + r.add(stateType); + r.addAll(argTypes); + return r; + } +} diff --git a/cql3/statements/CreateFunctionStatement.java b/cql3/statements/CreateFunctionStatement.java new file mode 100644 index 0000000000..dbdecf91f2 --- /dev/null +++ b/cql3/statements/CreateFunctionStatement.java @@ -0,0 +1,145 @@ +/* + * 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.util.ArrayList; +import java.util.HashSet; +import java.util.List; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.functions.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** + * A CREATE FUNCTION statement parsed from a CQL query. + */ +public final class CreateFunctionStatement extends SchemaAlteringStatement +{ + private final boolean orReplace; + private final boolean ifNotExists; + private FunctionName functionName; + private final String language; + private final String body; + private final boolean deterministic; + + private final List argNames; + private final List argRawTypes; + private final CQL3Type.Raw rawReturnType; + + public CreateFunctionStatement(FunctionName functionName, + String language, + String body, + boolean deterministic, + List argNames, + List argRawTypes, + CQL3Type.Raw rawReturnType, + boolean orReplace, + boolean ifNotExists) + { + this.functionName = functionName; + this.language = language; + this.body = body; + this.deterministic = deterministic; + this.argNames = argNames; + this.argRawTypes = argRawTypes; + this.rawReturnType = rawReturnType; + this.orReplace = orReplace; + this.ifNotExists = ifNotExists; + } + + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!functionName.hasKeyspace() && state.getRawKeyspace() != null) + functionName = new FunctionName(state.getRawKeyspace(), functionName.name); + + if (!functionName.hasKeyspace()) + throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session"); + + ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + // TODO CASSANDRA-7557 (function DDL permission) + + state.hasKeyspaceAccess(functionName.keyspace, Permission.CREATE); + } + + public void validate(ClientState state) throws InvalidRequestException + { + if (ifNotExists && orReplace) + throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives"); + + if (Schema.instance.getKSMetaData(functionName.keyspace) == null) + throw new InvalidRequestException(String.format("Cannot add function '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace)); + } + + public Event.SchemaChange changeEvent() + { + return null; + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + if (new HashSet<>(argNames).size() != argNames.size()) + throw new InvalidRequestException(String.format("duplicate argument names for given function %s with argument names %s", + functionName, argNames)); + + List> argTypes = new ArrayList<>(argRawTypes.size()); + for (CQL3Type.Raw rawType : argRawTypes) + argTypes.add(rawType.prepare(typeKeyspace(rawType)).getType()); + + AbstractType returnType = rawReturnType.prepare(typeKeyspace(rawReturnType)).getType(); + + Function old = Functions.find(functionName, argTypes); + if (old != null) + { + if (ifNotExists) + return false; + if (!orReplace) + throw new InvalidRequestException(String.format("Function %s already exists", old)); + if (!(old instanceof ScalarFunction)) + throw new InvalidRequestException(String.format("Function %s can only replace a function", old)); + + if (!Functions.typeEquals(old.returnType(), returnType)) + throw new InvalidRequestException(String.format("Cannot replace function %s, the new return type %s is not compatible with the return type %s of existing function", + functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type())); + } + + MigrationManager.announceNewFunction(UDFunction.create(functionName, argNames, argTypes, returnType, language, body, deterministic), isLocalOnly); + return true; + } + + private String typeKeyspace(CQL3Type.Raw rawType) + { + String ks = rawType.keyspace(); + if (ks != null) + return ks; + return functionName.keyspace; + } +} diff --git a/cql3/statements/CreateIndexStatement.java b/cql3/statements/CreateIndexStatement.java new file mode 100644 index 0000000000..ee08acaf17 --- /dev/null +++ b/cql3/statements/CreateIndexStatement.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.statements; + +import java.util.Collections; +import java.util.Map; + +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.IndexType; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.index.SecondaryIndex; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** A CREATE INDEX statement parsed from a CQL query. */ +public class CreateIndexStatement extends SchemaAlteringStatement +{ + private static final Logger logger = LoggerFactory.getLogger(CreateIndexStatement.class); + + private final String indexName; + private final IndexTarget.Raw rawTarget; + private final IndexPropDefs properties; + private final boolean ifNotExists; + + public CreateIndexStatement(CFName name, + String indexName, + IndexTarget.Raw target, + IndexPropDefs properties, + boolean ifNotExists) + { + super(name); + this.indexName = indexName; + this.rawTarget = target; + this.properties = properties; + this.ifNotExists = ifNotExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.ALTER); + } + + public void validate(ClientState state) throws RequestValidationException + { + CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + if (cfm.isCounter()) + throw new InvalidRequestException("Secondary indexes are not supported on counter tables"); + + IndexTarget target = rawTarget.prepare(cfm); + ColumnDefinition cd = cfm.getColumnDefinition(target.column); + + if (cd == null) + throw new InvalidRequestException("No column definition found for column " + target.column); + + boolean isMap = cd.type instanceof MapType; + boolean isFrozenCollection = cd.type.isCollection() && !cd.type.isMultiCell(); + + if (isFrozenCollection) + { + validateForFrozenCollection(target); + } + else + { + validateNotFullIndex(target); + validateIsValuesIndexIfTargetColumnNotCollection(cd, target); + validateTargetColumnIsMapIfIndexInvolvesKeys(isMap, target); + } + + if (cd.getIndexType() != null) + { + IndexTarget.TargetType prevType = IndexTarget.TargetType.fromColumnDefinition(cd); + if (isMap && target.type != prevType) + { + String msg = "Cannot create index on %s(%s): an index on %s(%s) already exists and indexing " + + "a map on more than one dimension at the same time is not currently supported"; + throw new InvalidRequestException(String.format(msg, + target.type, target.column, + prevType, target.column)); + } + + if (ifNotExists) + return; + else + throw new InvalidRequestException("Index already exists"); + } + + properties.validate(); + + // TODO: we could lift that limitation + if ((cfm.comparator.isDense() || !cfm.comparator.isCompound()) && cd.kind != ColumnDefinition.Kind.REGULAR) + throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables"); + + // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and + // CompositesIndex) and maybe we should, but that means a query like: + // SELECT * FROM foo WHERE static_column = 'bar' + // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a + // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where + // such indexing is actually useful. + if (cd.isStatic()) + throw new InvalidRequestException("Secondary indexes are not allowed on static columns"); + + if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cd.isOnAllComponents()) + throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column)); + } + + private void validateForFrozenCollection(IndexTarget target) throws InvalidRequestException + { + if (target.type != IndexTarget.TargetType.FULL) + throw new InvalidRequestException(String.format("Cannot create index on %s of frozen column %s", target.type, target.column)); + } + + private void validateNotFullIndex(IndexTarget target) throws InvalidRequestException + { + if (target.type == IndexTarget.TargetType.FULL) + throw new InvalidRequestException("full() indexes can only be created on frozen collections"); + } + + private void validateIsValuesIndexIfTargetColumnNotCollection(ColumnDefinition cd, IndexTarget target) throws InvalidRequestException + { + if (!cd.type.isCollection() && target.type != IndexTarget.TargetType.VALUES) + throw new InvalidRequestException(String.format("Cannot create index on %s of column %s; only non-frozen collections support %s indexes", + target.type, target.column, target.type)); + } + + private void validateTargetColumnIsMapIfIndexInvolvesKeys(boolean isMap, IndexTarget target) throws InvalidRequestException + { + if (target.type == IndexTarget.TargetType.KEYS || target.type == IndexTarget.TargetType.KEYS_AND_VALUES) + { + if (!isMap) + throw new InvalidRequestException(String.format("Cannot create index on %s of column %s with non-map type", + target.type, target.column)); + } + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy(); + IndexTarget target = rawTarget.prepare(cfm); + logger.debug("Updating column {} definition for index {}", target.column, indexName); + ColumnDefinition cd = cfm.getColumnDefinition(target.column); + + if (cd.getIndexType() != null && ifNotExists) + return false; + + if (properties.isCustom) + { + cd.setIndexType(IndexType.CUSTOM, properties.getOptions()); + } + else if (cfm.comparator.isCompound()) + { + Map options = Collections.emptyMap(); + // For now, we only allow indexing values for collections, but we could later allow + // to also index map keys, so we record that this is the values we index to make our + // lives easier then. + if (cd.type.isCollection() && cd.type.isMultiCell()) + options = ImmutableMap.of(target.type.indexOption(), ""); + cd.setIndexType(IndexType.COMPOSITES, options); + } + else + { + cd.setIndexType(IndexType.KEYS, Collections.emptyMap()); + } + + cd.setIndexName(indexName); + cfm.addDefaultIndexNames(); + MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly); + return true; + } + + public Event.SchemaChange changeEvent() + { + // Creating an index is akin to updating the CF + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } +} diff --git a/cql3/statements/CreateKeyspaceStatement.java b/cql3/statements/CreateKeyspaceStatement.java new file mode 100644 index 0000000000..8281cbd0f7 --- /dev/null +++ b/cql3/statements/CreateKeyspaceStatement.java @@ -0,0 +1,119 @@ +/* + * 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 org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** A CREATE KEYSPACE statement parsed from a CQL query. */ +public class CreateKeyspaceStatement extends SchemaAlteringStatement +{ + private final String name; + private final KSPropDefs attrs; + private final boolean ifNotExists; + + /** + * Creates a new CreateKeyspaceStatement instance for a given + * keyspace name and keyword arguments. + * + * @param name the name of the keyspace to create + * @param attrs map of the raw keyword arguments that followed the WITH keyword. + */ + public CreateKeyspaceStatement(String name, KSPropDefs attrs, boolean ifNotExists) + { + super(); + this.name = name; + this.attrs = attrs; + this.ifNotExists = ifNotExists; + } + + @Override + public String keyspace() + { + return name; + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + state.hasAllKeyspacesAccess(Permission.CREATE); + } + + /** + * The CqlParser only goes as far as extracting the keyword arguments + * from these statements, so this method is responsible for processing and + * validating. + * + * @throws InvalidRequestException if arguments are missing or unacceptable + */ + public void validate(ClientState state) throws RequestValidationException + { + ThriftValidation.validateKeyspaceNotSystem(name); + + // keyspace name + if (!name.matches("\\w+")) + throw new InvalidRequestException(String.format("\"%s\" is not a valid keyspace name", name)); + if (name.length() > Schema.NAME_LENGTH) + throw new InvalidRequestException(String.format("Keyspace names shouldn't be more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, name)); + + attrs.validate(); + + if (attrs.getReplicationStrategyClass() == null) + throw new ConfigurationException("Missing mandatory replication strategy class"); + + // The strategy is validated through KSMetaData.validate() in announceNewKeyspace below. + // However, for backward compatibility with thrift, this doesn't validate unexpected options yet, + // so doing proper validation here. + AbstractReplicationStrategy.validateReplicationStrategy(name, + AbstractReplicationStrategy.getClass(attrs.getReplicationStrategyClass()), + StorageService.instance.getTokenMetadata(), + DatabaseDescriptor.getEndpointSnitch(), + attrs.getReplicationOptions()); + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + try + { + MigrationManager.announceNewKeyspace(attrs.asKSMetadata(name), isLocalOnly); + return true; + } + catch (AlreadyExistsException e) + { + if (ifNotExists) + return false; + throw e; + } + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, keyspace()); + } +} diff --git a/cql3/statements/CreateTableStatement.java b/cql3/statements/CreateTableStatement.java new file mode 100644 index 0000000000..c8c24743e4 --- /dev/null +++ b/cql3/statements/CreateTableStatement.java @@ -0,0 +1,442 @@ +/* + * 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.*; + +import org.apache.cassandra.exceptions.*; +import org.apache.commons.lang3.StringUtils; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.Multiset; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.composites.*; +import org.apache.cassandra.db.ColumnFamilyType; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.io.compress.CompressionParameters; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** A CREATE TABLE parsed from a CQL query statement. */ +public class CreateTableStatement extends SchemaAlteringStatement +{ + public CellNameType comparator; + private AbstractType defaultValidator; + private AbstractType keyValidator; + + private final List keyAliases = new ArrayList(); + private final List columnAliases = new ArrayList(); + private ByteBuffer valueAlias; + + private boolean isDense; + + private final Map columns = new HashMap(); + private final Set staticColumns; + private final CFPropDefs properties; + private final boolean ifNotExists; + + public CreateTableStatement(CFName name, CFPropDefs properties, boolean ifNotExists, Set staticColumns) + { + super(name); + this.properties = properties; + this.ifNotExists = ifNotExists; + this.staticColumns = staticColumns; + + try + { + if (!this.properties.hasProperty(CFPropDefs.KW_COMPRESSION) && CFMetaData.DEFAULT_COMPRESSOR != null) + this.properties.addProperty(CFPropDefs.KW_COMPRESSION, + new HashMap() + {{ + put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR); + }}); + } + catch (SyntaxException e) + { + throw new AssertionError(e); + } + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(keyspace(), Permission.CREATE); + } + + public void validate(ClientState state) + { + // validated in announceMigration() + } + + // Column definitions + private List getColumns(CFMetaData cfm) + { + List columnDefs = new ArrayList<>(columns.size()); + Integer componentIndex = comparator.isCompound() ? comparator.clusteringPrefixSize() : null; + for (Map.Entry col : columns.entrySet()) + { + ColumnIdentifier id = col.getKey(); + columnDefs.add(staticColumns.contains(id) + ? ColumnDefinition.staticDef(cfm, col.getKey().bytes, col.getValue(), componentIndex) + : ColumnDefinition.regularDef(cfm, col.getKey().bytes, col.getValue(), componentIndex)); + } + + return columnDefs; + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + try + { + MigrationManager.announceNewColumnFamily(getCFMetaData(), isLocalOnly); + return true; + } + catch (AlreadyExistsException e) + { + if (ifNotExists) + return false; + throw e; + } + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } + + /** + * Returns a CFMetaData instance based on the parameters parsed from this + * CREATE statement, or defaults where applicable. + * + * @return a CFMetaData instance corresponding to the values parsed from this statement + * @throws InvalidRequestException on failure to validate parsed parameters + */ + public CFMetaData getCFMetaData() throws RequestValidationException + { + CFMetaData newCFMD; + newCFMD = new CFMetaData(keyspace(), + columnFamily(), + ColumnFamilyType.Standard, + comparator); + applyPropertiesTo(newCFMD); + return newCFMD; + } + + public void applyPropertiesTo(CFMetaData cfmd) throws RequestValidationException + { + cfmd.defaultValidator(defaultValidator) + .keyValidator(keyValidator) + .addAllColumnDefinitions(getColumns(cfmd)) + .isDense(isDense); + + addColumnMetadataFromAliases(cfmd, keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY); + addColumnMetadataFromAliases(cfmd, columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN); + if (valueAlias != null) + addColumnMetadataFromAliases(cfmd, Collections.singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE); + + properties.applyToCFMetadata(cfmd); + } + + private void addColumnMetadataFromAliases(CFMetaData cfm, List aliases, AbstractType comparator, ColumnDefinition.Kind kind) + { + if (comparator instanceof CompositeType) + { + CompositeType ct = (CompositeType)comparator; + for (int i = 0; i < aliases.size(); ++i) + if (aliases.get(i) != null) + cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(i), ct.types.get(i), i, kind)); + } + else + { + assert aliases.size() <= 1; + if (!aliases.isEmpty() && aliases.get(0) != null) + cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(0), comparator, null, kind)); + } + } + + + public static class RawStatement extends CFStatement + { + private final Map definitions = new HashMap<>(); + public final CFPropDefs properties = new CFPropDefs(); + + private final List> keyAliases = new ArrayList>(); + private final List columnAliases = new ArrayList(); + private final Map definedOrdering = new LinkedHashMap(); // Insertion ordering is important + private final Set staticColumns = new HashSet(); + + private boolean useCompactStorage; + private final Multiset definedNames = HashMultiset.create(1); + + private final boolean ifNotExists; + + public RawStatement(CFName name, boolean ifNotExists) + { + super(name); + this.ifNotExists = ifNotExists; + } + + /** + * Transform this raw statement into a CreateTableStatement. + */ + public ParsedStatement.Prepared prepare() throws RequestValidationException + { + // Column family name + if (!columnFamily().matches("\\w+")) + throw new InvalidRequestException(String.format("\"%s\" is not a valid table name (must be alphanumeric character only: [0-9A-Za-z]+)", columnFamily())); + if (columnFamily().length() > Schema.NAME_LENGTH) + throw new InvalidRequestException(String.format("Table names shouldn't be more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, columnFamily())); + + for (Multiset.Entry entry : definedNames.entrySet()) + if (entry.getCount() > 1) + throw new InvalidRequestException(String.format("Multiple definition of identifier %s", entry.getElement())); + + properties.validate(); + + CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists, staticColumns); + + Map definedMultiCellCollections = null; + for (Map.Entry entry : definitions.entrySet()) + { + ColumnIdentifier id = entry.getKey(); + CQL3Type pt = entry.getValue().prepare(keyspace()); + if (pt.isCollection() && ((CollectionType) pt.getType()).isMultiCell()) + { + if (definedMultiCellCollections == null) + definedMultiCellCollections = new HashMap<>(); + definedMultiCellCollections.put(id.bytes, (CollectionType) pt.getType()); + } + stmt.columns.put(id, pt.getType()); // we'll remove what is not a column below + } + + if (keyAliases.isEmpty()) + throw new InvalidRequestException("No PRIMARY KEY specifed (exactly one required)"); + else if (keyAliases.size() > 1) + throw new InvalidRequestException("Multiple PRIMARY KEYs specifed (exactly one required)"); + + List kAliases = keyAliases.get(0); + + List> keyTypes = new ArrayList>(kAliases.size()); + for (ColumnIdentifier alias : kAliases) + { + stmt.keyAliases.add(alias.bytes); + AbstractType t = getTypeAndRemove(stmt.columns, alias); + if (t instanceof CounterColumnType) + throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias)); + if (staticColumns.contains(alias)) + throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias)); + keyTypes.add(t); + } + stmt.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes); + + // Dense means that no part of the comparator stores a CQL column name. This means + // COMPACT STORAGE with at least one columnAliases (otherwise it's a thrift "static" CF). + stmt.isDense = useCompactStorage && !columnAliases.isEmpty(); + + // Handle column aliases + if (columnAliases.isEmpty()) + { + if (useCompactStorage) + { + // There should remain some column definition since it is a non-composite "static" CF + if (stmt.columns.isEmpty()) + throw new InvalidRequestException("No definition found that is not part of the PRIMARY KEY"); + + if (definedMultiCellCollections != null) + throw new InvalidRequestException("Non-frozen collection types are not supported with COMPACT STORAGE"); + + stmt.comparator = new SimpleSparseCellNameType(UTF8Type.instance); + } + else + { + stmt.comparator = definedMultiCellCollections == null + ? new CompoundSparseCellNameType(Collections.>emptyList()) + : new CompoundSparseCellNameType.WithCollection(Collections.>emptyList(), ColumnToCollectionType.getInstance(definedMultiCellCollections)); + } + } + else + { + // If we use compact storage and have only one alias, it is a + // standard "dynamic" CF, otherwise it's a composite + if (useCompactStorage && columnAliases.size() == 1) + { + if (definedMultiCellCollections != null) + throw new InvalidRequestException("Collection types are not supported with COMPACT STORAGE"); + + ColumnIdentifier alias = columnAliases.get(0); + if (staticColumns.contains(alias)) + throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias)); + + stmt.columnAliases.add(alias.bytes); + AbstractType at = getTypeAndRemove(stmt.columns, alias); + if (at instanceof CounterColumnType) + throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", stmt.columnAliases.get(0))); + stmt.comparator = new SimpleDenseCellNameType(at); + } + else + { + List> types = new ArrayList>(columnAliases.size() + 1); + for (ColumnIdentifier t : columnAliases) + { + stmt.columnAliases.add(t.bytes); + + AbstractType type = getTypeAndRemove(stmt.columns, t); + if (type instanceof CounterColumnType) + throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t)); + if (staticColumns.contains(t)) + throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", t)); + types.add(type); + } + + if (useCompactStorage) + { + if (definedMultiCellCollections != null) + throw new InvalidRequestException("Collection types are not supported with COMPACT STORAGE"); + + stmt.comparator = new CompoundDenseCellNameType(types); + } + else + { + stmt.comparator = definedMultiCellCollections == null + ? new CompoundSparseCellNameType(types) + : new CompoundSparseCellNameType.WithCollection(types, ColumnToCollectionType.getInstance(definedMultiCellCollections)); + } + } + } + + if (!staticColumns.isEmpty()) + { + // Only CQL3 tables can have static columns + if (useCompactStorage) + throw new InvalidRequestException("Static columns are not supported in COMPACT STORAGE tables"); + // Static columns only make sense if we have at least one clustering column. Otherwise everything is static anyway + if (columnAliases.isEmpty()) + throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column"); + } + + if (useCompactStorage && !stmt.columnAliases.isEmpty()) + { + if (stmt.columns.isEmpty()) + { + // The only value we'll insert will be the empty one, so the default validator don't matter + stmt.defaultValidator = BytesType.instance; + // We need to distinguish between + // * I'm upgrading from thrift so the valueAlias is null + // * I've defined my table with only a PK (and the column value will be empty) + // So, we use an empty valueAlias (rather than null) for the second case + stmt.valueAlias = ByteBufferUtil.EMPTY_BYTE_BUFFER; + } + else + { + if (stmt.columns.size() > 1) + throw new InvalidRequestException(String.format("COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: %s)", StringUtils.join(stmt.columns.keySet(), ", "))); + + Map.Entry lastEntry = stmt.columns.entrySet().iterator().next(); + stmt.defaultValidator = lastEntry.getValue(); + stmt.valueAlias = lastEntry.getKey().bytes; + stmt.columns.remove(lastEntry.getKey()); + } + } + else + { + // For compact, we are in the "static" case, so we need at least one column defined. For non-compact however, having + // just the PK is fine since we have CQL3 row marker. + if (useCompactStorage && stmt.columns.isEmpty()) + throw new InvalidRequestException("COMPACT STORAGE with non-composite PRIMARY KEY require one column not part of the PRIMARY KEY, none given"); + + // There is no way to insert/access a column that is not defined for non-compact storage, so + // the actual validator don't matter much (except that we want to recognize counter CF as limitation apply to them). + stmt.defaultValidator = !stmt.columns.isEmpty() && (stmt.columns.values().iterator().next() instanceof CounterColumnType) + ? CounterColumnType.instance + : BytesType.instance; + } + + + // If we give a clustering order, we must explicitly do so for all aliases and in the order of the PK + if (!definedOrdering.isEmpty()) + { + if (definedOrdering.size() > columnAliases.size()) + throw new InvalidRequestException("Only clustering key columns can be defined in CLUSTERING ORDER directive"); + + int i = 0; + for (ColumnIdentifier id : definedOrdering.keySet()) + { + ColumnIdentifier c = columnAliases.get(i); + if (!id.equals(c)) + { + if (definedOrdering.containsKey(c)) + throw new InvalidRequestException(String.format("The order of columns in the CLUSTERING ORDER directive must be the one of the clustering key (%s must appear before %s)", c, id)); + else + throw new InvalidRequestException(String.format("Missing CLUSTERING ORDER for column %s", c)); + } + ++i; + } + } + + return new ParsedStatement.Prepared(stmt); + } + + private AbstractType getTypeAndRemove(Map columns, ColumnIdentifier t) throws InvalidRequestException + { + AbstractType type = columns.get(t); + if (type == null) + throw new InvalidRequestException(String.format("Unknown definition %s referenced in PRIMARY KEY", t)); + if (type.isCollection() && type.isMultiCell()) + throw new InvalidRequestException(String.format("Invalid collection type for PRIMARY KEY component %s", t)); + + columns.remove(t); + Boolean isReversed = definedOrdering.get(t); + return isReversed != null && isReversed ? ReversedType.getInstance(type) : type; + } + + public void addDefinition(ColumnIdentifier def, CQL3Type.Raw type, boolean isStatic) + { + definedNames.add(def); + definitions.put(def, type); + if (isStatic) + staticColumns.add(def); + } + + public void addKeyAliases(List aliases) + { + keyAliases.add(aliases); + } + + public void addColumnAlias(ColumnIdentifier alias) + { + columnAliases.add(alias); + } + + public void setOrdering(ColumnIdentifier alias, boolean reversed) + { + definedOrdering.put(alias, reversed); + } + + public void setCompactStorage() + { + useCompactStorage = true; + } + } +} diff --git a/cql3/statements/CreateTriggerStatement.java b/cql3/statements/CreateTriggerStatement.java new file mode 100644 index 0000000000..6ebe0d38b5 --- /dev/null +++ b/cql3/statements/CreateTriggerStatement.java @@ -0,0 +1,91 @@ +/* + * 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.config.TriggerDefinition; +import org.apache.cassandra.cql3.CFName; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; +import org.apache.cassandra.triggers.TriggerExecutor; + +public class CreateTriggerStatement extends SchemaAlteringStatement +{ + private static final Logger logger = LoggerFactory.getLogger(CreateTriggerStatement.class); + + private final String triggerName; + private final String triggerClass; + private final boolean ifNotExists; + + public CreateTriggerStatement(CFName name, String triggerName, String clazz, boolean ifNotExists) + { + super(name); + this.triggerName = triggerName; + this.triggerClass = clazz; + this.ifNotExists = ifNotExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + state.ensureIsSuper("Only superusers are allowed to perform CREATE TRIGGER queries"); + } + + public void validate(ClientState state) throws RequestValidationException + { + ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + try + { + TriggerExecutor.instance.loadTriggerInstance(triggerClass); + } + catch (Exception e) + { + throw new ConfigurationException(String.format("Trigger class '%s' doesn't exist", triggerClass)); + } + } + + public boolean announceMigration(boolean isLocalOnly) throws ConfigurationException, InvalidRequestException + { + CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy(); + + TriggerDefinition triggerDefinition = TriggerDefinition.create(triggerName, triggerClass); + + if (!ifNotExists || !cfm.containsTriggerDefinition(triggerDefinition)) + { + cfm.addTriggerDefinition(triggerDefinition); + logger.info("Adding trigger with name {} and class {}", triggerName, triggerClass); + MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly); + return true; + } + return false; + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } +} diff --git a/cql3/statements/CreateTypeStatement.java b/cql3/statements/CreateTypeStatement.java new file mode 100644 index 0000000000..82c28084cf --- /dev/null +++ b/cql3/statements/CreateTypeStatement.java @@ -0,0 +1,133 @@ +/* + * 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.*; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.*; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; + +public class CreateTypeStatement extends SchemaAlteringStatement +{ + private final UTName name; + private final List columnNames = new ArrayList<>(); + private final List columnTypes = new ArrayList<>(); + private final boolean ifNotExists; + + public CreateTypeStatement(UTName name, boolean ifNotExists) + { + super(); + this.name = name; + this.ifNotExists = ifNotExists; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!name.hasKeyspace()) + name.setKeyspace(state.getKeyspace()); + } + + public void addDefinition(ColumnIdentifier name, CQL3Type.Raw type) + { + columnNames.add(name); + columnTypes.add(type); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(keyspace(), Permission.CREATE); + } + + public void validate(ClientState state) throws RequestValidationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + if (ksm == null) + throw new InvalidRequestException(String.format("Cannot add type in unknown keyspace %s", name.getKeyspace())); + + if (ksm.userTypes.getType(name.getUserTypeName()) != null && !ifNotExists) + throw new InvalidRequestException(String.format("A user type of name %s already exists", name)); + + for (CQL3Type.Raw type : columnTypes) + if (type.isCounter()) + throw new InvalidRequestException("A user type cannot contain counters"); + } + + public static void checkForDuplicateNames(UserType type) throws InvalidRequestException + { + for (int i = 0; i < type.size() - 1; i++) + { + ByteBuffer fieldName = type.fieldName(i); + for (int j = i+1; j < type.size(); j++) + { + if (fieldName.equals(type.fieldName(j))) + throw new InvalidRequestException(String.format("Duplicate field name %s in type %s", + UTF8Type.instance.getString(fieldName), + UTF8Type.instance.getString(type.name))); + } + } + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName()); + } + + @Override + public String keyspace() + { + return name.getKeyspace(); + } + + private UserType createType() throws InvalidRequestException + { + List names = new ArrayList<>(columnNames.size()); + for (ColumnIdentifier name : columnNames) + names.add(name.bytes); + + List> types = new ArrayList<>(columnTypes.size()); + for (CQL3Type.Raw type : columnTypes) + types.add(type.prepare(keyspace()).getType()); + + return new UserType(name.getKeyspace(), name.getUserTypeName(), names, types); + } + + public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + assert ksm != null; // should haven't validate otherwise + + // Can happen with ifNotExists + if (ksm.userTypes.getType(name.getUserTypeName()) != null) + return false; + + UserType type = createType(); + checkForDuplicateNames(type); + MigrationManager.announceNewType(type, isLocalOnly); + return true; + } +} diff --git a/cql3/statements/CreateUserStatement.java b/cql3/statements/CreateUserStatement.java new file mode 100644 index 0000000000..090365920d --- /dev/null +++ b/cql3/statements/CreateUserStatement.java @@ -0,0 +1,75 @@ +/* + * 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 org.apache.cassandra.auth.Auth; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.UserOptions; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class CreateUserStatement extends AuthenticationStatement +{ + private final String username; + private final UserOptions opts; + private final boolean superuser; + private final boolean ifNotExists; + + public CreateUserStatement(String username, UserOptions opts, boolean superuser, boolean ifNotExists) + { + this.username = username; + this.opts = opts; + this.superuser = superuser; + this.ifNotExists = ifNotExists; + } + + public void validate(ClientState state) throws RequestValidationException + { + if (username.isEmpty()) + throw new InvalidRequestException("Username can't be an empty string"); + + opts.validate(); + + // validate login here before checkAccess to avoid leaking user existence to anonymous users. + state.ensureNotAnonymous(); + + if (!ifNotExists && Auth.isExistingUser(username)) + throw new InvalidRequestException(String.format("User %s already exists", username)); + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + if (!state.getUser().isSuper()) + throw new UnauthorizedException("Only superusers are allowed to perform CREATE USER queries"); + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + // not rejected in validate() + if (ifNotExists && Auth.isExistingUser(username)) + return null; + + DatabaseDescriptor.getAuthenticator().create(username, opts.getOptions()); + Auth.insertUser(username, superuser); + return null; + } +} diff --git a/cql3/statements/DeleteStatement.java b/cql3/statements/DeleteStatement.java new file mode 100644 index 0000000000..ff685cfde4 --- /dev/null +++ b/cql3/statements/DeleteStatement.java @@ -0,0 +1,149 @@ +/* + * 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.*; + +import com.google.common.collect.Iterators; + +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.utils.Pair; + +/** + * A DELETE parsed from a CQL query statement. + */ +public class DeleteStatement extends ModificationStatement +{ + private DeleteStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs) + { + super(type, boundTerms, cfm, attrs); + } + + public boolean requireFullClusteringKey() + { + return false; + } + + public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, Composite prefix, UpdateParameters params) + throws InvalidRequestException + { + List deletions = getOperations(); + + if (prefix.size() < cfm.clusteringColumns().size() && !deletions.isEmpty()) + { + // In general, we can't delete specific columns if not all clustering columns have been specified. + // However, if we delete only static colums, it's fine since we won't really use the prefix anyway. + for (Operation deletion : deletions) + if (!deletion.column.isStatic()) + throw new InvalidRequestException(String.format("Primary key column '%s' must be specified in order to delete column '%s'", getFirstEmptyKey().name, deletion.column.name)); + } + + if (deletions.isEmpty()) + { + // We delete the slice selected by the prefix. + // However, for performance reasons, we distinguish 2 cases: + // - It's a full internal row delete + // - It's a full cell name (i.e it's a dense layout and the prefix is full) + if (prefix.isEmpty()) + { + // No columns specified, delete the row + cf.delete(new DeletionInfo(params.timestamp, params.localDeletionTime)); + } + else if (cfm.comparator.isDense() && prefix.size() == cfm.clusteringColumns().size()) + { + cf.addAtom(params.makeTombstone(cfm.comparator.create(prefix, null))); + } + else + { + cf.addAtom(params.makeRangeTombstone(prefix.slice())); + } + } + else + { + for (Operation op : deletions) + op.execute(key, cf, prefix, params); + } + } + + protected void validateWhereClauseForConditions() throws InvalidRequestException + { + Iterator iterator = Iterators.concat(cfm.partitionKeyColumns().iterator(), cfm.clusteringColumns().iterator()); + while (iterator.hasNext()) + { + ColumnDefinition def = iterator.next(); + Restriction restriction = processedKeys.get(def.name); + if (restriction == null || !(restriction.isEQ() || restriction.isIN())) + { + throw new InvalidRequestException( + String.format("DELETE statements must restrict all PRIMARY KEY columns with equality relations in order " + + "to use IF conditions, but column '%s' is not restricted", def.name)); + } + } + + } + + public static class Parsed extends ModificationStatement.Parsed + { + private final List deletions; + private final List whereClause; + + public Parsed(CFName name, + Attributes.Raw attrs, + List deletions, + List whereClause, + List> conditions, + boolean ifExists) + { + super(name, attrs, conditions, false, ifExists); + this.deletions = deletions; + this.whereClause = whereClause; + } + + protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException + { + DeleteStatement stmt = new DeleteStatement(ModificationStatement.StatementType.DELETE, boundNames.size(), cfm, attrs); + + for (Operation.RawDeletion deletion : deletions) + { + ColumnIdentifier id = deletion.affectedColumn().prepare(cfm); + ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) + throw new InvalidRequestException(String.format("Unknown identifier %s", id)); + + // For compact, we only have one value except the key, so the only form of DELETE that make sense is without a column + // list. However, we support having the value name for coherence with the static/sparse case + if (def.isPrimaryKeyColumn()) + throw new InvalidRequestException(String.format("Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", def.name)); + + Operation op = deletion.prepare(cfm.ksName, def); + op.collectMarkerSpecification(boundNames); + stmt.addOperation(op); + } + + stmt.processWhereClause(whereClause, boundNames); + return stmt; + } + } +} diff --git a/cql3/statements/DropAggregateStatement.java b/cql3/statements/DropAggregateStatement.java new file mode 100644 index 0000000000..118f89d3ee --- /dev/null +++ b/cql3/statements/DropAggregateStatement.java @@ -0,0 +1,136 @@ +/* + * 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.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.functions.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** + * A DROP AGGREGATE statement parsed from a CQL query. + */ +public final class DropAggregateStatement extends SchemaAlteringStatement +{ + private FunctionName functionName; + private final boolean ifExists; + private final List argRawTypes; + private final boolean argsPresent; + + public DropAggregateStatement(FunctionName functionName, + List argRawTypes, + boolean argsPresent, + boolean ifExists) + { + this.functionName = functionName; + this.argRawTypes = argRawTypes; + this.argsPresent = argsPresent; + this.ifExists = ifExists; + } + + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!functionName.hasKeyspace() && state.getRawKeyspace() != null) + functionName = new FunctionName(state.getKeyspace(), functionName.name); + + if (!functionName.hasKeyspace()) + throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session"); + + ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + // TODO CASSANDRA-7557 (function DDL permission) + + state.hasKeyspaceAccess(functionName.keyspace, Permission.DROP); + } + + public void validate(ClientState state) throws RequestValidationException + { + } + + public Event.SchemaChange changeEvent() + { + return null; + } + + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + List olds = Functions.find(functionName); + + if (!argsPresent && olds != null && olds.size() > 1) + throw new InvalidRequestException(String.format("'DROP AGGREGATE %s' matches multiple function definitions; " + + "specify the argument types by issuing a statement like " + + "'DROP AGGREGATE %s (type, type, ...)'. Hint: use cqlsh " + + "'DESCRIBE AGGREGATE %s' command to find all overloads", + functionName, functionName, functionName)); + + List> argTypes = new ArrayList<>(argRawTypes.size()); + for (CQL3Type.Raw rawType : argRawTypes) + argTypes.add(rawType.prepare(functionName.keyspace).getType()); + + Function old; + if (argsPresent) + { + old = Functions.find(functionName, argTypes); + if (old == null || !(old instanceof AggregateFunction)) + { + if (ifExists) + return false; + // just build a nicer error message + StringBuilder sb = new StringBuilder(); + for (CQL3Type.Raw rawType : argRawTypes) + { + if (sb.length() > 0) + sb.append(", "); + sb.append(rawType); + } + throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s(%s)'", + functionName, sb)); + } + } + else + { + if (olds == null || olds.isEmpty() || !(olds.get(0) instanceof AggregateFunction)) + { + if (ifExists) + return false; + throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s'", functionName)); + } + old = olds.get(0); + } + + if (old.isNative()) + throw new InvalidRequestException(String.format("Cannot drop aggregate '%s' because it is a " + + "native (built-in) function", functionName)); + + MigrationManager.announceAggregateDrop((UDAggregate)old, isLocalOnly); + return true; + } +} diff --git a/cql3/statements/DropFunctionStatement.java b/cql3/statements/DropFunctionStatement.java new file mode 100644 index 0000000000..394aca0f30 --- /dev/null +++ b/cql3/statements/DropFunctionStatement.java @@ -0,0 +1,149 @@ +/* + * 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.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.functions.*; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +/** + * A DROP FUNCTION statement parsed from a CQL query. + */ +public final class DropFunctionStatement extends SchemaAlteringStatement +{ + private FunctionName functionName; + private final boolean ifExists; + private final List argRawTypes; + private final boolean argsPresent; + + public DropFunctionStatement(FunctionName functionName, + List argRawTypes, + boolean argsPresent, + boolean ifExists) + { + this.functionName = functionName; + this.argRawTypes = argRawTypes; + this.argsPresent = argsPresent; + this.ifExists = ifExists; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!functionName.hasKeyspace() && state.getRawKeyspace() != null) + functionName = new FunctionName(state.getKeyspace(), functionName.name); + + if (!functionName.hasKeyspace()) + throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session"); + + ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace); + } + + @Override + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + // TODO CASSANDRA-7557 (function DDL permission) + + state.hasKeyspaceAccess(functionName.keyspace, Permission.DROP); + } + + @Override + public void validate(ClientState state) + { + } + + @Override + public Event.SchemaChange changeEvent() + { + return null; + } + + @Override + public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException + { + List olds = Functions.find(functionName); + + if (!argsPresent && olds != null && olds.size() > 1) + throw new InvalidRequestException(String.format("'DROP FUNCTION %s' matches multiple function definitions; " + + "specify the argument types by issuing a statement like " + + "'DROP FUNCTION %s (type, type, ...)'. Hint: use cqlsh " + + "'DESCRIBE FUNCTION %s' command to find all overloads", + functionName, functionName, functionName)); + + List> argTypes = new ArrayList<>(argRawTypes.size()); + for (CQL3Type.Raw rawType : argRawTypes) + argTypes.add(rawType.prepare(typeKeyspace(rawType)).getType()); + + Function old; + if (argsPresent) + { + old = Functions.find(functionName, argTypes); + if (old == null || !(old instanceof ScalarFunction)) + { + if (ifExists) + return false; + // just build a nicer error message + StringBuilder sb = new StringBuilder(); + for (CQL3Type.Raw rawType : argRawTypes) + { + if (sb.length() > 0) + sb.append(", "); + sb.append(rawType); + } + throw new InvalidRequestException(String.format("Cannot drop non existing function '%s(%s)'", + functionName, sb)); + } + } + else + { + if (olds == null || olds.isEmpty() || !(olds.get(0) instanceof ScalarFunction)) + { + if (ifExists) + return false; + throw new InvalidRequestException(String.format("Cannot drop non existing function '%s'", functionName)); + } + old = olds.get(0); + } + + List references = Functions.getReferencesTo(old); + if (!references.isEmpty()) + throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", functionName, references)); + + MigrationManager.announceFunctionDrop((UDFunction) old, isLocalOnly); + return true; + } + + private String typeKeyspace(CQL3Type.Raw rawType) + { + String ks = rawType.keyspace(); + if (ks != null) + return ks; + return functionName.keyspace; + } +} diff --git a/cql3/statements/DropIndexStatement.java b/cql3/statements/DropIndexStatement.java new file mode 100644 index 0000000000..5df8188d24 --- /dev/null +++ b/cql3/statements/DropIndexStatement.java @@ -0,0 +1,133 @@ +/* + * 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 org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.KeyspaceNotDefinedException; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.Event; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class DropIndexStatement extends SchemaAlteringStatement +{ + public final String indexName; + public final boolean ifExists; + + // initialized in announceMigration() + private String indexedCF; + + public DropIndexStatement(IndexName indexName, boolean ifExists) + { + super(indexName.getCfName()); + this.indexName = indexName.getIdx(); + this.ifExists = ifExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + CFMetaData cfm = findIndexedCF(); + if (cfm == null) + return; + + state.hasColumnFamilyAccess(cfm.ksName, cfm.cfName, Permission.ALTER); + } + + public void validate(ClientState state) + { + // validated in findIndexedCf() + } + + public Event.SchemaChange changeEvent() + { + // Dropping an index is akin to updating the CF + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } + + @Override + public ResultMessage execute(QueryState state, QueryOptions options) throws RequestValidationException + { + announceMigration(false); + return indexedCF == null ? null : new ResultMessage.SchemaChange(changeEvent()); + } + + public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException + { + CFMetaData cfm = findIndexedCF(); + if (cfm == null) + return false; + + CFMetaData updatedCfm = updateCFMetadata(cfm); + indexedCF = updatedCfm.cfName; + MigrationManager.announceColumnFamilyUpdate(updatedCfm, false, isLocalOnly); + return true; + } + + private CFMetaData updateCFMetadata(CFMetaData cfm) + { + ColumnDefinition column = findIndexedColumn(cfm); + assert column != null; + CFMetaData cloned = cfm.copy(); + ColumnDefinition toChange = cloned.getColumnDefinition(column.name); + assert toChange.getIndexName() != null && toChange.getIndexName().equals(indexName); + toChange.setIndexName(null); + toChange.setIndexType(null, null); + return cloned; + } + + private CFMetaData findIndexedCF() throws InvalidRequestException + { + KSMetaData ksm = Schema.instance.getKSMetaData(keyspace()); + if (ksm == null) + throw new KeyspaceNotDefinedException("Keyspace " + keyspace() + " does not exist"); + for (CFMetaData cfm : ksm.cfMetaData().values()) + { + if (findIndexedColumn(cfm) != null) + return cfm; + } + + if (ifExists) + return null; + else + throw new InvalidRequestException("Index '" + indexName + "' could not be found in any of the tables of keyspace '" + keyspace() + '\''); + } + + private ColumnDefinition findIndexedColumn(CFMetaData cfm) + { + for (ColumnDefinition column : cfm.allColumns()) + { + if (column.getIndexType() != null && column.getIndexName() != null && column.getIndexName().equals(indexName)) + return column; + } + return null; + } + + @Override + public String columnFamily() + { + assert indexedCF != null; + return indexedCF; + } +} diff --git a/cql3/statements/DropKeyspaceStatement.java b/cql3/statements/DropKeyspaceStatement.java new file mode 100644 index 0000000000..ba6b917e84 --- /dev/null +++ b/cql3/statements/DropKeyspaceStatement.java @@ -0,0 +1,77 @@ +/* + * 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 org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +public class DropKeyspaceStatement extends SchemaAlteringStatement +{ + private final String keyspace; + private final boolean ifExists; + + public DropKeyspaceStatement(String keyspace, boolean ifExists) + { + super(); + this.keyspace = keyspace; + this.ifExists = ifExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(keyspace, Permission.DROP); + } + + public void validate(ClientState state) throws RequestValidationException + { + ThriftValidation.validateKeyspaceNotSystem(keyspace); + } + + @Override + public String keyspace() + { + return keyspace; + } + + public boolean announceMigration(boolean isLocalOnly) throws ConfigurationException + { + try + { + MigrationManager.announceKeyspaceDrop(keyspace, isLocalOnly); + return true; + } + catch(ConfigurationException e) + { + if (ifExists) + return false; + throw e; + } + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, keyspace()); + } +} diff --git a/cql3/statements/DropTableStatement.java b/cql3/statements/DropTableStatement.java new file mode 100644 index 0000000000..e690c3e4fa --- /dev/null +++ b/cql3/statements/DropTableStatement.java @@ -0,0 +1,76 @@ +/* + * 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 org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.CFName; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; + +public class DropTableStatement extends SchemaAlteringStatement +{ + private final boolean ifExists; + + public DropTableStatement(CFName name, boolean ifExists) + { + super(name); + this.ifExists = ifExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + try + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.DROP); + } + catch (InvalidRequestException e) + { + if (!ifExists) + throw e; + } + } + + public void validate(ClientState state) + { + // validated in announceMigration() + } + + public boolean announceMigration(boolean isLocalOnly) throws ConfigurationException + { + try + { + MigrationManager.announceColumnFamilyDrop(keyspace(), columnFamily(), isLocalOnly); + return true; + } + catch (ConfigurationException e) + { + if (ifExists) + return false; + throw e; + } + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } +} diff --git a/cql3/statements/DropTriggerStatement.java b/cql3/statements/DropTriggerStatement.java new file mode 100644 index 0000000000..e3db1e1ce6 --- /dev/null +++ b/cql3/statements/DropTriggerStatement.java @@ -0,0 +1,78 @@ +/* + * 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.CFName; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.Event; + +public class DropTriggerStatement extends SchemaAlteringStatement +{ + private static final Logger logger = LoggerFactory.getLogger(DropTriggerStatement.class); + + private final String triggerName; + + private final boolean ifExists; + + public DropTriggerStatement(CFName name, String triggerName, boolean ifExists) + { + super(name); + this.triggerName = triggerName; + this.ifExists = ifExists; + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + state.ensureIsSuper("Only superusers are allowed to perfrom DROP TRIGGER queries"); + } + + public void validate(ClientState state) throws RequestValidationException + { + ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + } + + public boolean announceMigration(boolean isLocalOnly) throws ConfigurationException, InvalidRequestException + { + CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy(); + if (cfm.removeTrigger(triggerName)) + { + logger.info("Dropping trigger with name {}", triggerName); + MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly); + return true; + } + if (!ifExists) + throw new InvalidRequestException(String.format("Trigger %s was not found", triggerName)); + return false; + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily()); + } +} diff --git a/cql3/statements/DropTypeStatement.java b/cql3/statements/DropTypeStatement.java new file mode 100644 index 0000000000..ed21957077 --- /dev/null +++ b/cql3/statements/DropTypeStatement.java @@ -0,0 +1,164 @@ +/* + * 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 org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.*; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.functions.Function; +import org.apache.cassandra.cql3.functions.Functions; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.MigrationManager; +import org.apache.cassandra.transport.Event; + +public class DropTypeStatement extends SchemaAlteringStatement +{ + private final UTName name; + private final boolean ifExists; + + public DropTypeStatement(UTName name, boolean ifExists) + { + super(); + this.name = name; + this.ifExists = ifExists; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (!name.hasKeyspace()) + name.setKeyspace(state.getKeyspace()); + } + + public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException + { + state.hasKeyspaceAccess(keyspace(), Permission.DROP); + } + + public void validate(ClientState state) throws RequestValidationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + if (ksm == null) + throw new InvalidRequestException(String.format("Cannot drop type in unknown keyspace %s", name.getKeyspace())); + + UserType old = ksm.userTypes.getType(name.getUserTypeName()); + if (old == null) + { + if (ifExists) + return; + else + throw new InvalidRequestException(String.format("No user type named %s exists.", name)); + } + + // We don't want to drop a type unless it's not used anymore (mainly because + // if someone drops a type and recreates one with the same name but different + // definition with the previous name still in use, things can get messy). + // We have two places to check: 1) other user type that can nest the one + // we drop and 2) existing tables referencing the type (maybe in a nested + // way). + + for (Function function : Functions.all()) + { + if (isUsedBy(function.returnType())) + throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function)); + for (AbstractType argType : function.argTypes()) + if (isUsedBy(argType)) + throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function)); + } + + for (KSMetaData ksm2 : Schema.instance.getKeyspaceDefinitions()) + { + for (UserType ut : ksm2.userTypes.getAllTypes().values()) + { + if (ut.keyspace.equals(name.getKeyspace()) && ut.name.equals(name.getUserTypeName())) + continue; + if (isUsedBy(ut)) + throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by user type %s", name, ut.asCQL3Type())); + } + + for (CFMetaData cfm : ksm2.cfMetaData().values()) + for (ColumnDefinition def : cfm.allColumns()) + if (isUsedBy(def.type)) + throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by table %s.%s", name, cfm.ksName, cfm.cfName)); + } + } + + private boolean isUsedBy(AbstractType toCheck) throws RequestValidationException + { + if (toCheck instanceof UserType) + { + UserType ut = (UserType)toCheck; + if (name.getKeyspace().equals(ut.keyspace) && name.getUserTypeName().equals(ut.name)) + return true; + + for (AbstractType subtype : ut.fieldTypes()) + if (isUsedBy(subtype)) + return true; + } + else if (toCheck instanceof CompositeType) + { + CompositeType ct = (CompositeType)toCheck; + for (AbstractType subtype : ct.types) + if (isUsedBy(subtype)) + return true; + } + else if (toCheck instanceof ColumnToCollectionType) + { + for (CollectionType collection : ((ColumnToCollectionType)toCheck).defined.values()) + if (isUsedBy(collection)) + return true; + } + else if (toCheck instanceof CollectionType) + { + if (toCheck instanceof ListType) + return isUsedBy(((ListType)toCheck).getElementsType()); + else if (toCheck instanceof SetType) + return isUsedBy(((SetType)toCheck).getElementsType()); + else + return isUsedBy(((MapType)toCheck).getKeysType()) || isUsedBy(((MapType)toCheck).getKeysType()); + } + return false; + } + + public Event.SchemaChange changeEvent() + { + return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName()); + } + + @Override + public String keyspace() + { + return name.getKeyspace(); + } + + public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException + { + KSMetaData ksm = Schema.instance.getKSMetaData(name.getKeyspace()); + assert ksm != null; + + UserType toDrop = ksm.userTypes.getType(name.getUserTypeName()); + // Can be null with ifExists + if (toDrop == null) + return false; + + MigrationManager.announceTypeDrop(toDrop, isLocalOnly); + return true; + } +} diff --git a/cql3/statements/DropUserStatement.java b/cql3/statements/DropUserStatement.java new file mode 100644 index 0000000000..9e92b269f4 --- /dev/null +++ b/cql3/statements/DropUserStatement.java @@ -0,0 +1,72 @@ +/* + * 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 org.apache.cassandra.auth.Auth; +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class DropUserStatement extends AuthenticationStatement +{ + private final String username; + private final boolean ifExists; + + public DropUserStatement(String username, boolean ifExists) + { + this.username = username; + this.ifExists = ifExists; + } + + public void validate(ClientState state) throws RequestValidationException + { + // validate login here before checkAccess to avoid leaking user existence to anonymous users. + state.ensureNotAnonymous(); + + if (!ifExists && !Auth.isExistingUser(username)) + throw new InvalidRequestException(String.format("User %s doesn't exist", username)); + + AuthenticatedUser user = state.getUser(); + if (user != null && user.getName().equals(username)) + throw new InvalidRequestException("Users aren't allowed to DROP themselves"); + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + if (!state.getUser().isSuper()) + throw new UnauthorizedException("Only superusers are allowed to perform DROP USER queries"); + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + // not rejected in validate() + if (ifExists && !Auth.isExistingUser(username)) + return null; + + // clean up permissions after the dropped user. + DatabaseDescriptor.getAuthorizer().revokeAll(username); + Auth.deleteUser(username); + DatabaseDescriptor.getAuthenticator().drop(username); + return null; + } +} diff --git a/cql3/statements/GrantStatement.java b/cql3/statements/GrantStatement.java new file mode 100644 index 0000000000..67b651ceca --- /dev/null +++ b/cql3/statements/GrantStatement.java @@ -0,0 +1,43 @@ +/** + * 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.util.Set; + +import org.apache.cassandra.auth.IResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class GrantStatement extends PermissionAlteringStatement +{ + public GrantStatement(Set permissions, IResource resource, String username) + { + super(permissions, resource, username); + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, username); + return null; + } +} diff --git a/cql3/statements/IndexPropDefs.java b/cql3/statements/IndexPropDefs.java new file mode 100644 index 0000000000..6790611296 --- /dev/null +++ b/cql3/statements/IndexPropDefs.java @@ -0,0 +1,70 @@ +/* + * 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.util.*; + +import org.apache.cassandra.db.index.SecondaryIndex; +import org.apache.cassandra.exceptions.*; + +public class IndexPropDefs extends PropertyDefinitions +{ + public static final String KW_OPTIONS = "options"; + + public static final Set keywords = new HashSet<>(); + public static final Set obsoleteKeywords = new HashSet<>(); + + public boolean isCustom; + public String customClass; + + static + { + keywords.add(KW_OPTIONS); + } + + public void validate() throws RequestValidationException + { + validate(keywords, obsoleteKeywords); + + if (isCustom && customClass == null) + throw new InvalidRequestException("CUSTOM index requires specifiying the index class"); + + if (!isCustom && customClass != null) + throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index"); + + if (!isCustom && !properties.isEmpty()) + throw new InvalidRequestException("Cannot specify options for a non-CUSTOM index"); + + if (getRawOptions().containsKey(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME)) + throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option", + SecondaryIndex.CUSTOM_INDEX_OPTION_NAME)); + } + + public Map getRawOptions() throws SyntaxException + { + Map options = getMap(KW_OPTIONS); + return options == null ? Collections.emptyMap() : options; + } + + public Map getOptions() throws SyntaxException + { + Map options = new HashMap<>(getRawOptions()); + options.put(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, customClass); + return options; + } +} diff --git a/cql3/statements/IndexTarget.java b/cql3/statements/IndexTarget.java new file mode 100644 index 0000000000..d6023882bd --- /dev/null +++ b/cql3/statements/IndexTarget.java @@ -0,0 +1,114 @@ +/* + * 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.util.Map; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.index.SecondaryIndex; + +public class IndexTarget +{ + public final ColumnIdentifier column; + public final TargetType type; + + private IndexTarget(ColumnIdentifier column, TargetType type) + { + this.column = column; + this.type = type; + } + + public static class Raw + { + private final ColumnIdentifier.Raw column; + private final TargetType type; + + private Raw(ColumnIdentifier.Raw column, TargetType type) + { + this.column = column; + this.type = type; + } + + public static Raw valuesOf(ColumnIdentifier.Raw c) + { + return new Raw(c, TargetType.VALUES); + } + + public static Raw keysOf(ColumnIdentifier.Raw c) + { + return new Raw(c, TargetType.KEYS); + } + + public static Raw keysAndValuesOf(ColumnIdentifier.Raw c) + { + return new Raw(c, TargetType.KEYS_AND_VALUES); + } + + public static Raw fullCollection(ColumnIdentifier.Raw c) + { + return new Raw(c, TargetType.FULL); + } + + public IndexTarget prepare(CFMetaData cfm) + { + return new IndexTarget(column.prepare(cfm), type); + } + } + + public static enum TargetType + { + VALUES, KEYS, KEYS_AND_VALUES, FULL; + + public String toString() + { + switch (this) + { + case KEYS: return "keys"; + case KEYS_AND_VALUES: return "entries"; + case FULL: return "full"; + default: return "values"; + } + } + + public String indexOption() + { + switch (this) + { + case KEYS: return SecondaryIndex.INDEX_KEYS_OPTION_NAME; + case KEYS_AND_VALUES: return SecondaryIndex.INDEX_ENTRIES_OPTION_NAME; + case VALUES: return SecondaryIndex.INDEX_VALUES_OPTION_NAME; + default: throw new AssertionError(); + } + } + + public static TargetType fromColumnDefinition(ColumnDefinition cd) + { + Map options = cd.getIndexOptions(); + if (options.containsKey(SecondaryIndex.INDEX_KEYS_OPTION_NAME)) + return KEYS; + else if (options.containsKey(SecondaryIndex.INDEX_ENTRIES_OPTION_NAME)) + return KEYS_AND_VALUES; + else if (cd.type.isCollection() && !cd.type.isMultiCell()) + return FULL; + else + return VALUES; + } + } +} diff --git a/cql3/statements/KSPropDefs.java b/cql3/statements/KSPropDefs.java new file mode 100644 index 0000000000..7c054351df --- /dev/null +++ b/cql3/statements/KSPropDefs.java @@ -0,0 +1,89 @@ +/* + * 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.util.*; + +import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.exceptions.*; + +public class KSPropDefs extends PropertyDefinitions +{ + public static final String KW_DURABLE_WRITES = "durable_writes"; + public static final String KW_REPLICATION = "replication"; + + public static final String REPLICATION_STRATEGY_CLASS_KEY = "class"; + + public static final Set keywords = new HashSet<>(); + public static final Set obsoleteKeywords = new HashSet<>(); + + static + { + keywords.add(KW_DURABLE_WRITES); + keywords.add(KW_REPLICATION); + } + + private String strategyClass; + + public void validate() throws SyntaxException + { + // Skip validation if the strategy class is already set as it means we've alreayd + // prepared (and redoing it would set strategyClass back to null, which we don't want) + if (strategyClass != null) + return; + + validate(keywords, obsoleteKeywords); + + Map replicationOptions = getReplicationOptions(); + if (!replicationOptions.isEmpty()) + { + strategyClass = replicationOptions.get(REPLICATION_STRATEGY_CLASS_KEY); + replicationOptions.remove(REPLICATION_STRATEGY_CLASS_KEY); + } + } + + public Map getReplicationOptions() throws SyntaxException + { + Map replicationOptions = getMap(KW_REPLICATION); + if (replicationOptions == null) + return Collections.emptyMap(); + return replicationOptions; + } + + public String getReplicationStrategyClass() + { + return strategyClass; + } + + public KSMetaData asKSMetadata(String ksName) throws RequestValidationException + { + return KSMetaData.newKeyspace(ksName, getReplicationStrategyClass(), getReplicationOptions(), getBoolean(KW_DURABLE_WRITES, true)); + } + + public KSMetaData asKSMetadataUpdate(KSMetaData old) throws RequestValidationException + { + String sClass = strategyClass; + Map sOptions = getReplicationOptions(); + if (sClass == null) + { + sClass = old.strategyClass.getName(); + sOptions = old.strategyOptions; + } + return KSMetaData.newKeyspace(old.name, sClass, sOptions, getBoolean(KW_DURABLE_WRITES, old.durableWrites)); + } +} diff --git a/cql3/statements/ListPermissionsStatement.java b/cql3/statements/ListPermissionsStatement.java new file mode 100644 index 0000000000..d8db871a96 --- /dev/null +++ b/cql3/statements/ListPermissionsStatement.java @@ -0,0 +1,123 @@ +/* + * 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.util.*; + +import org.apache.cassandra.auth.*; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.ResultSet; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class ListPermissionsStatement extends AuthorizationStatement +{ + private static final String KS = Auth.AUTH_KS; + private static final String CF = "permissions"; // virtual cf to use for now. + + private static final List metadata; + + static + { + List columns = new ArrayList(4); + columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("username", true), UTF8Type.instance)); + columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("resource", true), UTF8Type.instance)); + columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("permission", true), UTF8Type.instance)); + metadata = Collections.unmodifiableList(columns); + } + + private final Set permissions; + private DataResource resource; + private final String username; + private final boolean recursive; + + public ListPermissionsStatement(Set permissions, IResource resource, String username, boolean recursive) + { + this.permissions = permissions; + this.resource = (DataResource) resource; + this.username = username; + this.recursive = recursive; + } + + public void validate(ClientState state) throws RequestValidationException + { + // a check to ensure the existence of the user isn't being leaked by user existence check. + state.ensureNotAnonymous(); + + if (username != null && !Auth.isExistingUser(username)) + throw new InvalidRequestException(String.format("User %s doesn't exist", username)); + + if (resource != null) + { + resource = maybeCorrectResource(resource, state); + if (!resource.exists()) + throw new InvalidRequestException(String.format("%s doesn't exist", resource)); + } + } + + public void checkAccess(ClientState state) + { + // checked in validate + } + + // TODO: Create a new ResultMessage type (?). Rows will do for now. + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + List details = new ArrayList(); + + if (resource != null && recursive) + { + for (IResource r : Resources.chain(resource)) + details.addAll(list(state, r)); + } + else + { + details.addAll(list(state, resource)); + } + + Collections.sort(details); + return resultMessage(details); + } + + private ResultMessage resultMessage(List details) + { + if (details.isEmpty()) + return new ResultMessage.Void(); + + ResultSet result = new ResultSet(metadata); + for (PermissionDetails pd : details) + { + result.addColumnValue(UTF8Type.instance.decompose(pd.username)); + result.addColumnValue(UTF8Type.instance.decompose(pd.resource.toString())); + result.addColumnValue(UTF8Type.instance.decompose(pd.permission.toString())); + } + return new ResultMessage.Rows(result); + } + + private Set list(ClientState state, IResource resource) + throws RequestValidationException, RequestExecutionException + { + return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, username); + } +} diff --git a/cql3/statements/ListUsersStatement.java b/cql3/statements/ListUsersStatement.java new file mode 100644 index 0000000000..8acbcab327 --- /dev/null +++ b/cql3/statements/ListUsersStatement.java @@ -0,0 +1,47 @@ +/* + * 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 org.apache.cassandra.auth.Auth; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class ListUsersStatement extends AuthenticationStatement +{ + public void validate(ClientState state) + { + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + state.ensureNotAnonymous(); + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + return QueryProcessor.process(String.format("SELECT * FROM %s.%s", Auth.AUTH_KS, Auth.USERS_CF), + ConsistencyLevel.QUORUM, + QueryState.forInternalCalls()); + } +} diff --git a/cql3/statements/ModificationStatement.java b/cql3/statements/ModificationStatement.java new file mode 100644 index 0000000000..8347ef5825 --- /dev/null +++ b/cql3/statements/ModificationStatement.java @@ -0,0 +1,770 @@ +/* + * 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.*; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.restrictions.Restriction; +import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction; +import org.apache.cassandra.cql3.selection.Selection; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CBuilder; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.db.filter.SliceQueryFilter; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.Pair; + +/* + * Abstract parent class of individual modifications, i.e. INSERT, UPDATE and DELETE. + */ +public abstract class ModificationStatement implements CQLStatement +{ + private static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false); + + public static enum StatementType { INSERT, UPDATE, DELETE } + public final StatementType type; + + private final int boundTerms; + public final CFMetaData cfm; + public final Attributes attrs; + + protected final Map processedKeys = new HashMap<>(); + private final List columnOperations = new ArrayList(); + + // Separating normal and static conditions makes things somewhat easier + private List columnConditions; + private List staticConditions; + private boolean ifNotExists; + private boolean ifExists; + + private boolean hasNoClusteringColumns = true; + + private boolean setsStaticColumns; + private boolean setsRegularColumns; + + private final Function getColumnForCondition = new Function() + { + public ColumnDefinition apply(ColumnCondition cond) + { + return cond.column; + } + }; + + public ModificationStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs) + { + this.type = type; + this.boundTerms = boundTerms; + this.cfm = cfm; + this.attrs = attrs; + } + + public boolean usesFunction(String ksName, String functionName) + { + if (attrs.usesFunction(ksName, functionName)) + return true; + for (Restriction restriction : processedKeys.values()) + if (restriction != null && restriction.usesFunction(ksName, functionName)) + return true; + for (Operation operation : columnOperations) + if (operation != null && operation.usesFunction(ksName, functionName)) + return true; + for (ColumnCondition condition : columnConditions) + if (condition != null && condition.usesFunction(ksName, functionName)) + return true; + for (ColumnCondition condition : staticConditions) + if (condition != null && condition.usesFunction(ksName, functionName)) + return true; + return false; + } + + public abstract boolean requireFullClusteringKey(); + public abstract void addUpdateForKey(ColumnFamily updates, ByteBuffer key, Composite prefix, UpdateParameters params) throws InvalidRequestException; + + public int getBoundTerms() + { + return boundTerms; + } + + public String keyspace() + { + return cfm.ksName; + } + + public String columnFamily() + { + return cfm.cfName; + } + + public boolean isCounter() + { + return cfm.isCounter(); + } + + public long getTimestamp(long now, QueryOptions options) throws InvalidRequestException + { + return attrs.getTimestamp(now, options); + } + + public boolean isTimestampSet() + { + return attrs.isTimestampSet(); + } + + public int getTimeToLive(QueryOptions options) throws InvalidRequestException + { + return attrs.getTimeToLive(options); + } + + public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.MODIFY); + + // CAS updates can be used to simulate a SELECT query, so should require Permission.SELECT as well. + if (hasConditions()) + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT); + } + + public void validate(ClientState state) throws InvalidRequestException + { + if (hasConditions() && attrs.isTimestampSet()) + throw new InvalidRequestException("Cannot provide custom timestamp for conditional updates"); + + if (isCounter() && attrs.isTimestampSet()) + throw new InvalidRequestException("Cannot provide custom timestamp for counter updates"); + + if (isCounter() && attrs.isTimeToLiveSet()) + throw new InvalidRequestException("Cannot provide custom TTL for counter updates"); + } + + public void addOperation(Operation op) + { + if (op.column.isStatic()) + setsStaticColumns = true; + else + setsRegularColumns = true; + columnOperations.add(op); + } + + public List getOperations() + { + return columnOperations; + } + + public Iterable getColumnsWithConditions() + { + if (ifNotExists || ifExists) + return null; + + return Iterables.concat(columnConditions == null ? Collections.emptyList() : Iterables.transform(columnConditions, getColumnForCondition), + staticConditions == null ? Collections.emptyList() : Iterables.transform(staticConditions, getColumnForCondition)); + } + + public void addCondition(ColumnCondition cond) + { + List conds = null; + if (cond.column.isStatic()) + { + setsStaticColumns = true; + if (staticConditions == null) + staticConditions = new ArrayList(); + conds = staticConditions; + } + else + { + setsRegularColumns = true; + if (columnConditions == null) + columnConditions = new ArrayList(); + conds = columnConditions; + } + conds.add(cond); + } + + public void setIfNotExistCondition() + { + ifNotExists = true; + } + + public boolean hasIfNotExistCondition() + { + return ifNotExists; + } + + public void setIfExistCondition() + { + ifExists = true; + } + + public boolean hasIfExistCondition() + { + return ifExists; + } + + private void addKeyValues(ColumnDefinition def, Restriction values) throws InvalidRequestException + { + if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN) + hasNoClusteringColumns = false; + if (processedKeys.put(def.name, values) != null) + throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", def.name)); + } + + public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException + { + addKeyValues(def, new SingleColumnRestriction.EQ(def, value)); + } + + public void processWhereClause(List whereClause, VariableSpecifications names) throws InvalidRequestException + { + for (Relation relation : whereClause) + { + if (relation.isMultiColumn()) + { + throw new InvalidRequestException( + String.format("Multi-column relations cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation)); + } + SingleColumnRelation rel = (SingleColumnRelation) relation; + + if (rel.onToken()) + throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation)); + + ColumnIdentifier id = rel.getEntity().prepare(cfm); + ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) + throw new InvalidRequestException(String.format("Unknown key identifier %s", id)); + + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + Restriction restriction; + + if (rel.isEQ() || (def.isPartitionKey() && rel.isIN())) + { + restriction = rel.toRestriction(cfm, names); + } + else + { + throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), def.name)); + } + + addKeyValues(def, restriction); + break; + default: + throw new InvalidRequestException(String.format("Non PRIMARY KEY %s found in where clause", def.name)); + } + } + } + + public List buildPartitionKeyNames(QueryOptions options) + throws InvalidRequestException + { + CBuilder keyBuilder = cfm.getKeyValidatorAsCType().builder(); + List keys = new ArrayList(); + for (ColumnDefinition def : cfm.partitionKeyColumns()) + { + Restriction r = processedKeys.get(def.name); + if (r == null) + throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name)); + + List values = r.values(options); + + if (keyBuilder.remainingCount() == 1) + { + for (ByteBuffer val : values) + { + if (val == null) + throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name)); + ByteBuffer key = keyBuilder.buildWith(val).toByteBuffer(); + ThriftValidation.validateKey(cfm, key); + keys.add(key); + } + } + else + { + if (values.size() != 1) + throw new InvalidRequestException("IN is only supported on the last column of the partition key"); + ByteBuffer val = values.get(0); + if (val == null) + throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name)); + keyBuilder.add(val); + } + } + return keys; + } + + public Composite createClusteringPrefix(QueryOptions options) + throws InvalidRequestException + { + // If the only updated/deleted columns are static, then we don't need clustering columns. + // And in fact, unless it is an INSERT, we reject if clustering colums are provided as that + // suggest something unintended. For instance, given: + // CREATE TABLE t (k int, v int, s int static, PRIMARY KEY (k, v)) + // it can make sense to do: + // INSERT INTO t(k, v, s) VALUES (0, 1, 2) + // but both + // UPDATE t SET s = 3 WHERE k = 0 AND v = 1 + // DELETE v FROM t WHERE k = 0 AND v = 1 + // sounds like you don't really understand what your are doing. + if (setsStaticColumns && !setsRegularColumns) + { + // If we set no non-static columns, then it's fine not to have clustering columns + if (hasNoClusteringColumns) + return cfm.comparator.staticPrefix(); + + // If we do have clustering columns however, then either it's an INSERT and the query is valid + // but we still need to build a proper prefix, or it's not an INSERT, and then we want to reject + // (see above) + if (type != StatementType.INSERT) + { + for (ColumnDefinition def : cfm.clusteringColumns()) + if (processedKeys.get(def.name) != null) + throw new InvalidRequestException(String.format("Invalid restriction on clustering column %s since the %s statement modifies only static columns", def.name, type)); + // we should get there as it contradicts hasNoClusteringColumns == false + throw new AssertionError(); + } + } + + return createClusteringPrefixBuilderInternal(options); + } + + private Composite createClusteringPrefixBuilderInternal(QueryOptions options) + throws InvalidRequestException + { + CBuilder builder = cfm.comparator.prefixBuilder(); + ColumnDefinition firstEmptyKey = null; + for (ColumnDefinition def : cfm.clusteringColumns()) + { + Restriction r = processedKeys.get(def.name); + if (r == null) + { + firstEmptyKey = def; + if (requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound()) + throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name)); + } + else if (firstEmptyKey != null) + { + throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s since %s is set", firstEmptyKey.name, def.name)); + } + else + { + List values = r.values(options); + assert values.size() == 1; // We only allow IN for row keys so far + ByteBuffer val = values.get(0); + if (val == null) + throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name)); + builder.add(val); + } + } + return builder.build(); + } + + protected ColumnDefinition getFirstEmptyKey() + { + for (ColumnDefinition def : cfm.clusteringColumns()) + { + if (processedKeys.get(def.name) == null) + return def; + } + return null; + } + + public boolean requiresRead() + { + // Lists SET operation incurs a read. + for (Operation op : columnOperations) + if (op.requiresRead()) + return true; + + return false; + } + + protected Map readRequiredRows(Collection partitionKeys, Composite clusteringPrefix, boolean local, ConsistencyLevel cl) + throws RequestExecutionException, RequestValidationException + { + if (!requiresRead()) + return null; + + try + { + cl.validateForRead(keyspace()); + } + catch (InvalidRequestException e) + { + throw new InvalidRequestException(String.format("Write operation require a read but consistency %s is not supported on reads", cl)); + } + + ColumnSlice[] slices = new ColumnSlice[]{ clusteringPrefix.slice() }; + List commands = new ArrayList(partitionKeys.size()); + long now = System.currentTimeMillis(); + for (ByteBuffer key : partitionKeys) + commands.add(new SliceFromReadCommand(keyspace(), + key, + columnFamily(), + now, + new SliceQueryFilter(slices, false, Integer.MAX_VALUE))); + + List rows = local + ? SelectStatement.readLocally(keyspace(), commands) + : StorageProxy.read(commands, cl); + + Map map = new HashMap(); + for (Row row : rows) + { + if (row.cf == null || row.cf.isEmpty()) + continue; + + Iterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(row.cf.getSortedColumns().iterator()); + if (iter.hasNext()) + { + map.put(row.key.getKey(), iter.next()); + // We can only update one CQ3Row per partition key at a time (we don't allow IN for clustering key) + assert !iter.hasNext(); + } + } + return map; + } + + public boolean hasConditions() + { + return ifNotExists + || ifExists + || (columnConditions != null && !columnConditions.isEmpty()) + || (staticConditions != null && !staticConditions.isEmpty()); + } + + public ResultMessage execute(QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + if (options.getConsistency() == null) + throw new InvalidRequestException("Invalid empty consistency level"); + + if (hasConditions() && options.getProtocolVersion() == 1) + throw new InvalidRequestException("Conditional updates are not supported by the protocol version in use. You need to upgrade to a driver using the native protocol v2."); + + return hasConditions() + ? executeWithCondition(queryState, options) + : executeWithoutCondition(queryState, options); + } + + private ResultMessage executeWithoutCondition(QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + ConsistencyLevel cl = options.getConsistency(); + if (isCounter()) + cl.validateCounterForWrite(cfm); + else + cl.validateForWrite(cfm.ksName); + + Collection mutations = getMutations(options, false, options.getTimestamp(queryState)); + if (!mutations.isEmpty()) + StorageProxy.mutateWithTriggers(mutations, cl, false); + + return null; + } + + public ResultMessage executeWithCondition(QueryState queryState, QueryOptions options) + throws RequestExecutionException, RequestValidationException + { + List keys = buildPartitionKeyNames(options); + // We don't support IN for CAS operation so far + if (keys.size() > 1) + throw new InvalidRequestException("IN on the partition key is not supported with conditional updates"); + + ByteBuffer key = keys.get(0); + long now = options.getTimestamp(queryState); + Composite prefix = createClusteringPrefix(options); + + CQL3CasRequest request = new CQL3CasRequest(cfm, key, false); + addConditions(prefix, request, options); + request.addRowUpdate(prefix, this, options, now); + + ColumnFamily result = StorageProxy.cas(keyspace(), + columnFamily(), + key, + request, + options.getSerialConsistency(), + options.getConsistency(), + queryState.getClientState()); + return new ResultMessage.Rows(buildCasResultSet(key, result, options)); + } + + public void addConditions(Composite clusteringPrefix, CQL3CasRequest request, QueryOptions options) throws InvalidRequestException + { + if (ifNotExists) + { + // If we use ifNotExists, if the statement applies to any non static columns, then the condition is on the row of the non-static + // columns and the prefix should be the clusteringPrefix. But if only static columns are set, then the ifNotExists apply to the existence + // of any static columns and we should use the prefix for the "static part" of the partition. + request.addNotExist(clusteringPrefix); + } + else if (ifExists) + { + request.addExist(clusteringPrefix); + } + else + { + if (columnConditions != null) + request.addConditions(clusteringPrefix, columnConditions, options); + if (staticConditions != null) + request.addConditions(cfm.comparator.staticPrefix(), staticConditions, options); + } + } + + private ResultSet buildCasResultSet(ByteBuffer key, ColumnFamily cf, QueryOptions options) throws InvalidRequestException + { + return buildCasResultSet(keyspace(), key, columnFamily(), cf, getColumnsWithConditions(), false, options); + } + + public static ResultSet buildCasResultSet(String ksName, ByteBuffer key, String cfName, ColumnFamily cf, Iterable columnsWithConditions, boolean isBatch, QueryOptions options) + throws InvalidRequestException + { + boolean success = cf == null; + + ColumnSpecification spec = new ColumnSpecification(ksName, cfName, CAS_RESULT_COLUMN, BooleanType.instance); + ResultSet.Metadata metadata = new ResultSet.Metadata(Collections.singletonList(spec)); + List> rows = Collections.singletonList(Collections.singletonList(BooleanType.instance.decompose(success))); + + ResultSet rs = new ResultSet(metadata, rows); + return success ? rs : merge(rs, buildCasFailureResultSet(key, cf, columnsWithConditions, isBatch, options)); + } + + private static ResultSet merge(ResultSet left, ResultSet right) + { + if (left.size() == 0) + return right; + else if (right.size() == 0) + return left; + + assert left.size() == 1; + int size = left.metadata.names.size() + right.metadata.names.size(); + List specs = new ArrayList(size); + specs.addAll(left.metadata.names); + specs.addAll(right.metadata.names); + List> rows = new ArrayList<>(right.size()); + for (int i = 0; i < right.size(); i++) + { + List row = new ArrayList(size); + row.addAll(left.rows.get(0)); + row.addAll(right.rows.get(i)); + rows.add(row); + } + return new ResultSet(new ResultSet.Metadata(specs), rows); + } + + private static ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf, Iterable columnsWithConditions, boolean isBatch, QueryOptions options) + throws InvalidRequestException + { + CFMetaData cfm = cf.metadata(); + Selection selection; + if (columnsWithConditions == null) + { + selection = Selection.wildcard(cfm); + } + else + { + // We can have multiple conditions on the same columns (for collections) so use a set + // to avoid duplicate, but preserve the order just to it follows the order of IF in the query in general + Set defs = new LinkedHashSet<>(); + // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside + // of batches for compatibility sakes). + if (isBatch) + { + defs.addAll(cfm.partitionKeyColumns()); + defs.addAll(cfm.clusteringColumns()); + } + for (ColumnDefinition def : columnsWithConditions) + defs.add(def); + selection = Selection.forColumns(cfm, new ArrayList<>(defs)); + + } + + long now = System.currentTimeMillis(); + Selection.ResultSetBuilder builder = selection.resultSetBuilder(now); + SelectStatement.forSelection(cfm, selection).processColumnFamily(key, cf, options, now, builder); + + return builder.build(options.getProtocolVersion()); + } + + public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException + { + if (hasConditions()) + throw new UnsupportedOperationException(); + + for (IMutation mutation : getMutations(options, true, queryState.getTimestamp())) + { + // We don't use counters internally. + assert mutation instanceof Mutation; + + ((Mutation) mutation).apply(); + } + return null; + } + + /** + * Convert statement into a list of mutations to apply on the server + * + * @param options value for prepared statement markers + * @param local if true, any requests (for collections) performed by getMutation should be done locally only. + * @param now the current timestamp in microseconds to use if no timestamp is user provided. + * + * @return list of the mutations + * @throws InvalidRequestException on invalid requests + */ + private Collection getMutations(QueryOptions options, boolean local, long now) + throws RequestExecutionException, RequestValidationException + { + List keys = buildPartitionKeyNames(options); + Composite clusteringPrefix = createClusteringPrefix(options); + + UpdateParameters params = makeUpdateParameters(keys, clusteringPrefix, options, local, now); + + Collection mutations = new ArrayList(keys.size()); + for (ByteBuffer key: keys) + { + ThriftValidation.validateKey(cfm, key); + ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfm); + addUpdateForKey(cf, key, clusteringPrefix, params); + Mutation mut = new Mutation(cfm.ksName, key, cf); + + mutations.add(isCounter() ? new CounterMutation(mut, options.getConsistency()) : mut); + } + return mutations; + } + + public UpdateParameters makeUpdateParameters(Collection keys, + Composite prefix, + QueryOptions options, + boolean local, + long now) + throws RequestExecutionException, RequestValidationException + { + // Some lists operation requires reading + Map rows = readRequiredRows(keys, prefix, local, options.getConsistency()); + return new UpdateParameters(cfm, options, getTimestamp(now, options), getTimeToLive(options), rows); + } + + /** + * If there are conditions on the statement, this is called after the where clause and conditions have been + * processed to check that they are compatible. + * @throws InvalidRequestException + */ + protected void validateWhereClauseForConditions() throws InvalidRequestException + { + // no-op by default + } + + public static abstract class Parsed extends CFStatement + { + protected final Attributes.Raw attrs; + protected final List> conditions; + private final boolean ifNotExists; + private final boolean ifExists; + + protected Parsed(CFName name, Attributes.Raw attrs, List> conditions, boolean ifNotExists, boolean ifExists) + { + super(name); + this.attrs = attrs; + this.conditions = conditions == null ? Collections.>emptyList() : conditions; + this.ifNotExists = ifNotExists; + this.ifExists = ifExists; + } + + public ParsedStatement.Prepared prepare() throws InvalidRequestException + { + VariableSpecifications boundNames = getBoundVariables(); + ModificationStatement statement = prepare(boundNames); + return new ParsedStatement.Prepared(statement, boundNames); + } + + public ModificationStatement prepare(VariableSpecifications boundNames) throws InvalidRequestException + { + CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + + Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily()); + preparedAttributes.collectMarkerSpecification(boundNames); + + ModificationStatement stmt = prepareInternal(metadata, boundNames, preparedAttributes); + + if (ifNotExists || ifExists || !conditions.isEmpty()) + { + if (stmt.isCounter()) + throw new InvalidRequestException("Conditional updates are not supported on counter tables"); + + if (attrs.timestamp != null) + throw new InvalidRequestException("Cannot provide custom timestamp for conditional updates"); + + if (ifNotExists) + { + // To have both 'IF NOT EXISTS' and some other conditions doesn't make sense. + // So far this is enforced by the parser, but let's assert it for sanity if ever the parse changes. + assert conditions.isEmpty(); + assert !ifExists; + stmt.setIfNotExistCondition(); + } + else if (ifExists) + { + assert conditions.isEmpty(); + assert !ifNotExists; + stmt.setIfExistCondition(); + } + else + { + for (Pair entry : conditions) + { + ColumnIdentifier id = entry.left.prepare(metadata); + ColumnDefinition def = metadata.getColumnDefinition(id); + if (def == null) + throw new InvalidRequestException(String.format("Unknown identifier %s", id)); + + ColumnCondition condition = entry.right.prepare(keyspace(), def); + condition.collectMarkerSpecification(boundNames); + + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + throw new InvalidRequestException(String.format("PRIMARY KEY column '%s' cannot have IF conditions", id)); + default: + stmt.addCondition(condition); + break; + } + } + } + + stmt.validateWhereClauseForConditions(); + } + return stmt; + } + + protected abstract ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException; + } +} diff --git a/cql3/statements/ParsedStatement.java b/cql3/statements/ParsedStatement.java new file mode 100644 index 0000000000..bcce9ce0b9 --- /dev/null +++ b/cql3/statements/ParsedStatement.java @@ -0,0 +1,69 @@ +/* + * 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.util.Collections; +import java.util.List; + +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.exceptions.RequestValidationException; + +public abstract class ParsedStatement +{ + private VariableSpecifications variables; + + public VariableSpecifications getBoundVariables() + { + return variables; + } + + // Used by the parser and preparable statement + public void setBoundVariables(List boundNames) + { + this.variables = new VariableSpecifications(boundNames); + } + + public abstract Prepared prepare() throws RequestValidationException; + + public static class Prepared + { + public final CQLStatement statement; + public final List boundNames; + + public Prepared(CQLStatement statement, List boundNames) + { + this.statement = statement; + this.boundNames = boundNames; + } + + public Prepared(CQLStatement statement, VariableSpecifications names) + { + this(statement, names.getSpecifications()); + } + + public Prepared(CQLStatement statement) + { + this(statement, Collections.emptyList()); + } + } + + public boolean usesFunction(String ksName, String functionName) + { + return false; + } +} diff --git a/cql3/statements/PermissionAlteringStatement.java b/cql3/statements/PermissionAlteringStatement.java new file mode 100644 index 0000000000..5f3b70c35d --- /dev/null +++ b/cql3/statements/PermissionAlteringStatement.java @@ -0,0 +1,66 @@ +/* + * 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.util.Set; + +import org.apache.cassandra.auth.Auth; +import org.apache.cassandra.auth.DataResource; +import org.apache.cassandra.auth.IResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.service.ClientState; + +public abstract class PermissionAlteringStatement extends AuthorizationStatement +{ + protected final Set permissions; + protected DataResource resource; + protected final String username; + + protected PermissionAlteringStatement(Set permissions, IResource resource, String username) + { + this.permissions = permissions; + this.resource = (DataResource) resource; + this.username = username; + } + + public void validate(ClientState state) throws RequestValidationException + { + // validate login here before checkAccess to avoid leaking user existence to anonymous users. + state.ensureNotAnonymous(); + + if (!Auth.isExistingUser(username)) + throw new InvalidRequestException(String.format("User %s doesn't exist", username)); + + // if a keyspace is omitted when GRANT/REVOKE ON TABLE , we need to correct the resource. + resource = maybeCorrectResource(resource, state); + if (!resource.exists()) + throw new InvalidRequestException(String.format("%s doesn't exist", resource)); + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + // check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject GRANT/REVOKE. + state.ensureHasPermission(Permission.AUTHORIZE, resource); + // check that the user has [a single permission or all in case of ALL] on the resource or its parents. + for (Permission p : permissions) + state.ensureHasPermission(p, resource); + } +} diff --git a/cql3/statements/PropertyDefinitions.java b/cql3/statements/PropertyDefinitions.java new file mode 100644 index 0000000000..793285bbdb --- /dev/null +++ b/cql3/statements/PropertyDefinitions.java @@ -0,0 +1,143 @@ +/* + * 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.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.SyntaxException; + +public class PropertyDefinitions +{ + protected static final Logger logger = LoggerFactory.getLogger(PropertyDefinitions.class); + + protected final Map properties = new HashMap(); + + public void addProperty(String name, String value) throws SyntaxException + { + if (properties.put(name, value) != null) + throw new SyntaxException(String.format("Multiple definition for property '%s'", name)); + } + + public void addProperty(String name, Map value) throws SyntaxException + { + if (properties.put(name, value) != null) + throw new SyntaxException(String.format("Multiple definition for property '%s'", name)); + } + + public void validate(Set keywords, Set obsolete) throws SyntaxException + { + for (String name : properties.keySet()) + { + if (keywords.contains(name)) + continue; + + if (obsolete.contains(name)) + logger.warn("Ignoring obsolete property {}", name); + else + throw new SyntaxException(String.format("Unknown property '%s'", name)); + } + } + + protected String getSimple(String name) throws SyntaxException + { + Object val = properties.get(name); + if (val == null) + return null; + if (!(val instanceof String)) + throw new SyntaxException(String.format("Invalid value for property '%s'. It should be a string", name)); + return (String)val; + } + + protected Map getMap(String name) throws SyntaxException + { + Object val = properties.get(name); + if (val == null) + return null; + if (!(val instanceof Map)) + throw new SyntaxException(String.format("Invalid value for property '%s'. It should be a map.", name)); + return (Map)val; + } + + public Boolean hasProperty(String name) + { + return properties.containsKey(name); + } + + public String getString(String key, String defaultValue) throws SyntaxException + { + String value = getSimple(key); + return value != null ? value : defaultValue; + } + + // Return a property value, typed as a Boolean + public Boolean getBoolean(String key, Boolean defaultValue) throws SyntaxException + { + String value = getSimple(key); + return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)"); + } + + // Return a property value, typed as a double + public double getDouble(String key, double defaultValue) throws SyntaxException + { + String value = getSimple(key); + if (value == null) + { + return defaultValue; + } + else + { + try + { + return Double.parseDouble(value); + } + catch (NumberFormatException e) + { + throw new SyntaxException(String.format("Invalid double value %s for '%s'", value, key)); + } + } + } + + // Return a property value, typed as an Integer + public Integer getInt(String key, Integer defaultValue) throws SyntaxException + { + String value = getSimple(key); + return toInt(key, value, defaultValue); + } + + public static Integer toInt(String key, String value, Integer defaultValue) throws SyntaxException + { + if (value == null) + { + return defaultValue; + } + else + { + try + { + return Integer.valueOf(value); + } + catch (NumberFormatException e) + { + throw new SyntaxException(String.format("Invalid integer value %s for '%s'", value, key)); + } + } + } +} diff --git a/cql3/statements/RequestValidations.java b/cql3/statements/RequestValidations.java new file mode 100644 index 0000000000..ce15e159ef --- /dev/null +++ b/cql3/statements/RequestValidations.java @@ -0,0 +1,194 @@ +/* + * 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.util.ArrayList; +import java.util.HashSet; +import java.util.List; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.commons.lang3.ArrayUtils.EMPTY_OBJECT_ARRAY; + +/** + * Utility methods use to perform request validation. + */ +public final class RequestValidations +{ + /** + * Checks that the specified expression is true. If not an InvalidRequestException will + * be thrown. + * + * @param expression the expression to test + * @param messageTemplate the error message + * @param messageArgs the message arguments + * @throws InvalidRequestException if the specified expression is false. + */ + public static void checkTrue(boolean expression, String message) throws InvalidRequestException + { + checkTrue(expression, message, EMPTY_OBJECT_ARRAY); + } + + /** + * Checks that the specified expression is true. If not an InvalidRequestException will + * be thrown. + * + * @param expression the expression to test + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @throws InvalidRequestException if the specified expression is false. + */ + public static void checkTrue(boolean expression, + String messageTemplate, + Object... messageArgs) + throws InvalidRequestException + { + if (!expression) + throw invalidRequest(messageTemplate, messageArgs); + } + + /** + * Checks that the specified list does not contains duplicates. + * + * @param list the list to test + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @throws InvalidRequestException if the specified list contains duplicates. + */ + public static void checkContainsNoDuplicates(List list, String message) throws InvalidRequestException + { + if (new HashSet<>(list).size() != list.size()) + throw invalidRequest(message); + } + + /** + * Checks that the specified list contains only the specified elements. + * + * @param list the list to test + * @param expectedElements the expected elements + * @param message the error message + * @throws InvalidRequestException if the specified list contains duplicates. + */ + public static void checkContainsOnly(List list, + List expectedElements, + String message) throws InvalidRequestException + { + List copy = new ArrayList<>(list); + copy.removeAll(expectedElements); + if (!copy.isEmpty()) + throw invalidRequest(message); + } + + /** + * Checks that the specified expression is false. If not an InvalidRequestException will + * be thrown. + * + * @param expression the expression to test + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @throws InvalidRequestException if the specified expression is true. + */ + public static void checkFalse(boolean expression, + String messageTemplate, + Object... messageArgs) + throws InvalidRequestException + { + checkTrue(!expression, messageTemplate, messageArgs); + } + + /** + * Checks that the specified expression is false. If not an InvalidRequestException will + * be thrown. + * + * @param expression the expression to test + * @param message the error message + * @throws InvalidRequestException if the specified expression is true. + */ + public static void checkFalse(boolean expression, String message) throws InvalidRequestException + { + checkTrue(!expression, message); + } + + /** + * Checks that the specified object is NOT null. + * If it is an InvalidRequestException will be throws. + * + * @param object the object to test + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @return the object + * @throws InvalidRequestException if the specified object is null. + */ + public static T checkNotNull(T object, String messageTemplate, Object... messageArgs) + throws InvalidRequestException + { + checkTrue(object != null, messageTemplate, messageArgs); + return object; + } + + /** + * Checks that the specified object is null. + * If it is not an InvalidRequestException will be throws. + * + * @param object the object to test + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @return the object + * @throws InvalidRequestException if the specified object is not null. + */ + public static T checkNull(T object, String messageTemplate, Object... messageArgs) + throws InvalidRequestException + { + checkTrue(object == null, messageTemplate, messageArgs); + return object; + } + + /** + * Checks that the specified object is null. + * If it is not an InvalidRequestException will be throws. + * + * @param object the object to test + * @param message the error message + * @return the object + * @throws InvalidRequestException if the specified object is not null. + */ + public static T checkNull(T object, String message) throws InvalidRequestException + { + return checkNull(object, message, EMPTY_OBJECT_ARRAY); + } + + /** + * Returns an InvalidRequestException with the specified message. + * + * @param messageTemplate the template used to build the error message + * @param messageArgs the message arguments + * @return an InvalidRequestException with the specified message. + */ + public static InvalidRequestException invalidRequest(String messageTemplate, Object... messageArgs) + { + return new InvalidRequestException(String.format(messageTemplate, messageArgs)); + } + + /** + * This class must not be instantiated as it only contains static methods. + */ + private RequestValidations() + { + + } +} diff --git a/cql3/statements/RevokeStatement.java b/cql3/statements/RevokeStatement.java new file mode 100644 index 0000000000..e81a385651 --- /dev/null +++ b/cql3/statements/RevokeStatement.java @@ -0,0 +1,43 @@ +/** + * 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.util.Set; + +import org.apache.cassandra.auth.IResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.transport.messages.ResultMessage; + +public class RevokeStatement extends PermissionAlteringStatement +{ + public RevokeStatement(Set permissions, IResource resource, String username) + { + super(permissions, resource, username); + } + + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException + { + DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, username); + return null; + } +} diff --git a/cql3/statements/SchemaAlteringStatement.java b/cql3/statements/SchemaAlteringStatement.java new file mode 100644 index 0000000000..018d24400e --- /dev/null +++ b/cql3/statements/SchemaAlteringStatement.java @@ -0,0 +1,104 @@ +/* + * 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 org.apache.cassandra.cql3.CFName; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.Event; +import org.apache.cassandra.transport.messages.ResultMessage; + +/** + * Abstract class for statements that alter the schema. + */ +public abstract class SchemaAlteringStatement extends CFStatement implements CQLStatement +{ + private final boolean isColumnFamilyLevel; + + protected SchemaAlteringStatement() + { + super(null); + this.isColumnFamilyLevel = false; + } + + protected SchemaAlteringStatement(CFName name) + { + super(name); + this.isColumnFamilyLevel = true; + } + + public int getBoundTerms() + { + return 0; + } + + @Override + public void prepareKeyspace(ClientState state) throws InvalidRequestException + { + if (isColumnFamilyLevel) + super.prepareKeyspace(state); + } + + @Override + public Prepared prepare() + { + return new Prepared(this); + } + + public abstract Event.SchemaChange changeEvent(); + + /** + * Announces the migration to other nodes in the cluster. + * @return true if the execution of this statement resulted in a schema change, false otherwise (when IF NOT EXISTS + * is used, for example) + * @throws RequestValidationException + */ + public abstract boolean announceMigration(boolean isLocalOnly) throws RequestValidationException; + + public ResultMessage execute(QueryState state, QueryOptions options) throws RequestValidationException + { + // If an IF [NOT] EXISTS clause was used, this may not result in an actual schema change. To avoid doing + // extra work in the drivers to handle schema changes, we return an empty message in this case. (CASSANDRA-7600) + boolean didChangeSchema = announceMigration(false); + if (!didChangeSchema) + return new ResultMessage.Void(); + + Event.SchemaChange ce = changeEvent(); + return ce == null ? new ResultMessage.Void() : new ResultMessage.SchemaChange(ce); + } + + public ResultMessage executeInternal(QueryState state, QueryOptions options) + { + try + { + boolean didChangeSchema = announceMigration(true); + if (!didChangeSchema) + return new ResultMessage.Void(); + + Event.SchemaChange ce = changeEvent(); + return ce == null ? new ResultMessage.Void() : new ResultMessage.SchemaChange(ce); + } + catch (RequestValidationException e) + { + throw new RuntimeException(e); + } + } +} diff --git a/cql3/statements/SelectStatement.java b/cql3/statements/SelectStatement.java new file mode 100644 index 0000000000..f06055ab46 --- /dev/null +++ b/cql3/statements/SelectStatement.java @@ -0,0 +1,1074 @@ +/* + * 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.*; + +import com.google.common.base.Objects; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.restrictions.StatementRestrictions; +import org.apache.cassandra.cql3.selection.RawSelector; +import org.apache.cassandra.cql3.selection.Selection; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.CellName; +import org.apache.cassandra.db.composites.CellNameType; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.composites.Composites; +import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.db.filter.IDiskAtomFilter; +import org.apache.cassandra.db.filter.NamesQueryFilter; +import org.apache.cassandra.db.filter.SliceQueryFilter; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.pager.Pageable; +import org.apache.cassandra.service.pager.QueryPager; +import org.apache.cassandra.service.pager.QueryPagers; +import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; + +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.cql3.statements.RequestValidations.invalidRequest; + +/** + * Encapsulates a completely parsed SELECT query, including the target + * column family, expression, result count, and ordering clause. + * + */ +public class SelectStatement implements CQLStatement +{ + private static final int DEFAULT_COUNT_PAGE_SIZE = 10000; + + private final int boundTerms; + public final CFMetaData cfm; + public final Parameters parameters; + private final Selection selection; + private final Term limit; + + private final StatementRestrictions restrictions; + + private final boolean isReversed; + + /** + * The comparator used to orders results when multiple keys are selected (using IN). + */ + private final Comparator> orderingComparator; + + // Used by forSelection below + private static final Parameters defaultParameters = new Parameters(Collections.emptyMap(), false, false); + + public SelectStatement(CFMetaData cfm, + int boundTerms, + Parameters parameters, + Selection selection, + StatementRestrictions restrictions, + boolean isReversed, + Comparator> orderingComparator, + Term limit) + { + this.cfm = cfm; + this.boundTerms = boundTerms; + this.selection = selection; + this.restrictions = restrictions; + this.isReversed = isReversed; + this.orderingComparator = orderingComparator; + this.parameters = parameters; + this.limit = limit; + } + + @Override + public boolean usesFunction(String ksName, String functionName) + { + return selection.usesFunction(ksName, functionName) + || restrictions.usesFunction(ksName, functionName) + || (limit != null && limit.usesFunction(ksName, functionName)); + } + + // Creates a simple select based on the given selection. + // Note that the results select statement should not be used for actual queries, but only for processing already + // queried data through processColumnFamily. + static SelectStatement forSelection(CFMetaData cfm, Selection selection) + { + return new SelectStatement(cfm, + 0, + defaultParameters, + selection, + StatementRestrictions.empty(cfm), + false, + null, + null); + } + + public ResultSet.Metadata getResultMetadata() + { + return selection.getResultMetadata(); + } + + public int getBoundTerms() + { + return boundTerms; + } + + public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT); + } + + public void validate(ClientState state) throws InvalidRequestException + { + // Nothing to do, all validation has been done by RawStatement.prepare() + } + + public ResultMessage.Rows execute(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException + { + ConsistencyLevel cl = options.getConsistency(); + checkNotNull(cl, "Invalid empty consistency level"); + + cl.validateForRead(keyspace()); + + int limit = getLimit(options); + long now = System.currentTimeMillis(); + Pageable command = getPageableCommand(options, limit, now); + + int pageSize = options.getPageSize(); + + // An aggregation query will never be paged for the user, but we always page it internally to avoid OOM. + // If we user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default + // Note that if there are some nodes in the cluster with a version less than 2.0, we can't use paging (CASSANDRA-6707). + if (selection.isAggregate() && pageSize <= 0) + pageSize = DEFAULT_COUNT_PAGE_SIZE; + + if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize)) + { + return execute(command, options, limit, now, state); + } + + QueryPager pager = QueryPagers.pager(command, cl, state.getClientState(), options.getPagingState()); + + if (selection.isAggregate()) + return pageAggregateQuery(pager, options, pageSize, now); + + // We can't properly do post-query ordering if we page (see #6722) + checkFalse(needsPostQueryOrdering(), + "Cannot page queries with both ORDER BY and a IN restriction on the partition key;" + + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query"); + + List page = pager.fetchPage(pageSize); + ResultMessage.Rows msg = processResults(page, options, limit, now); + + if (!pager.isExhausted()) + msg.result.metadata.setHasMorePages(pager.state()); + + return msg; + } + + private Pageable getPageableCommand(QueryOptions options, int limit, long now) throws RequestValidationException + { + int limitForQuery = updateLimitForQuery(limit); + if (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing()) + return getRangeCommand(options, limitForQuery, now); + + List commands = getSliceCommands(options, limitForQuery, now); + return commands == null ? null : new Pageable.ReadCommands(commands, limitForQuery); + } + + public Pageable getPageableCommand(QueryOptions options) throws RequestValidationException + { + return getPageableCommand(options, getLimit(options), System.currentTimeMillis()); + } + + private ResultMessage.Rows execute(Pageable command, QueryOptions options, int limit, long now, QueryState state) throws RequestValidationException, RequestExecutionException + { + List rows; + if (command == null) + { + rows = Collections.emptyList(); + } + else + { + rows = command instanceof Pageable.ReadCommands + ? StorageProxy.read(((Pageable.ReadCommands)command).commands, options.getConsistency(), state.getClientState()) + : StorageProxy.getRangeSlice((RangeSliceCommand)command, options.getConsistency()); + } + + return processResults(rows, options, limit, now); + } + + private ResultMessage.Rows pageAggregateQuery(QueryPager pager, QueryOptions options, int pageSize, long now) + throws RequestValidationException, RequestExecutionException + { + Selection.ResultSetBuilder result = selection.resultSetBuilder(now); + while (!pager.isExhausted()) + { + for (org.apache.cassandra.db.Row row : pager.fetchPage(pageSize)) + { + // Not columns match the query, skip + if (row.cf == null) + continue; + + processColumnFamily(row.key.getKey(), row.cf, options, now, result); + } + } + return new ResultMessage.Rows(result.build(options.getProtocolVersion())); + } + + public ResultMessage.Rows processResults(List rows, QueryOptions options, int limit, long now) throws RequestValidationException + { + ResultSet rset = process(rows, options, limit, now); + return new ResultMessage.Rows(rset); + } + + static List readLocally(String keyspaceName, List cmds) + { + Keyspace keyspace = Keyspace.open(keyspaceName); + List rows = new ArrayList(cmds.size()); + for (ReadCommand cmd : cmds) + rows.add(cmd.getRow(keyspace)); + return rows; + } + + public ResultMessage.Rows executeInternal(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException + { + int limit = getLimit(options); + long now = System.currentTimeMillis(); + Pageable command = getPageableCommand(options, limit, now); + List rows = command == null + ? Collections.emptyList() + : (command instanceof Pageable.ReadCommands + ? readLocally(keyspace(), ((Pageable.ReadCommands)command).commands) + : ((RangeSliceCommand)command).executeLocally()); + + return processResults(rows, options, limit, now); + } + + public ResultSet process(List rows) throws InvalidRequestException + { + QueryOptions options = QueryOptions.DEFAULT; + return process(rows, options, getLimit(options), System.currentTimeMillis()); + } + + public String keyspace() + { + return cfm.ksName; + } + + public String columnFamily() + { + return cfm.cfName; + } + + private List getSliceCommands(QueryOptions options, int limit, long now) throws RequestValidationException + { + Collection keys = restrictions.getPartitionKeys(options); + + List commands = new ArrayList<>(keys.size()); + + IDiskAtomFilter filter = makeFilter(options, limit); + if (filter == null) + return null; + + // Note that we use the total limit for every key, which is potentially inefficient. + // However, IN + LIMIT is not a very sensible choice. + for (ByteBuffer key : keys) + { + QueryProcessor.validateKey(key); + // We should not share the slice filter amongst the commands (hence the cloneShallow), due to + // SliceQueryFilter not being immutable due to its columnCounter used by the lastCounted() method + // (this is fairly ugly and we should change that but that's probably not a tiny refactor to do that cleanly) + commands.add(ReadCommand.create(keyspace(), ByteBufferUtil.clone(key), columnFamily(), now, filter.cloneShallow())); + } + + return commands; + } + + private RangeSliceCommand getRangeCommand(QueryOptions options, int limit, long now) throws RequestValidationException + { + IDiskAtomFilter filter = makeFilter(options, limit); + if (filter == null) + return null; + + List expressions = getValidatedIndexExpressions(options); + // The LIMIT provided by the user is the number of CQL row he wants returned. + // We want to have getRangeSlice to count the number of columns, not the number of keys. + AbstractBounds keyBounds = restrictions.getPartitionKeyBounds(options); + return keyBounds == null + ? null + : new RangeSliceCommand(keyspace(), columnFamily(), now, filter, keyBounds, expressions, limit, !parameters.isDistinct, false); + } + + private ColumnSlice makeStaticSlice() + { + // Note: we could use staticPrefix.start() for the start bound, but EMPTY gives us the + // same effect while saving a few CPU cycles. + return isReversed + ? new ColumnSlice(cfm.comparator.staticPrefix().end(), Composites.EMPTY) + : new ColumnSlice(Composites.EMPTY, cfm.comparator.staticPrefix().end()); + } + + private IDiskAtomFilter makeFilter(QueryOptions options, int limit) + throws InvalidRequestException + { + int toGroup = cfm.comparator.isDense() ? -1 : cfm.clusteringColumns().size(); + if (parameters.isDistinct) + { + // For distinct, we only care about fetching the beginning of each partition. If we don't have + // static columns, we in fact only care about the first cell, so we query only that (we don't "group"). + // If we do have static columns, we do need to fetch the first full group (to have the static columns values). + return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, selection.containsStaticColumns() ? toGroup : -1); + } + else if (restrictions.isColumnRange()) + { + List startBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.START, options); + List endBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.END, options); + assert startBounds.size() == endBounds.size(); + + // Handles fetching static columns. Note that for 2i, the filter is just used to restrict + // the part of the index to query so adding the static slice would be useless and confusing. + // For 2i, static columns are retrieve in CompositesSearcher with each index hit. + ColumnSlice staticSlice = selection.containsStaticColumns() && !restrictions.usesSecondaryIndexing() + ? makeStaticSlice() + : null; + + // The case where startBounds == 1 is common enough that it's worth optimizing + if (startBounds.size() == 1) + { + ColumnSlice slice = new ColumnSlice(startBounds.get(0), endBounds.get(0)); + if (slice.isAlwaysEmpty(cfm.comparator, isReversed)) + return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup); + + if (staticSlice == null) + return sliceFilter(slice, limit, toGroup); + + if (isReversed) + return slice.includes(cfm.comparator.reverseComparator(), staticSlice.start) + ? sliceFilter(new ColumnSlice(slice.start, staticSlice.finish), limit, toGroup) + : sliceFilter(new ColumnSlice[]{ slice, staticSlice }, limit, toGroup); + else + return slice.includes(cfm.comparator, staticSlice.finish) + ? sliceFilter(new ColumnSlice(staticSlice.start, slice.finish), limit, toGroup) + : sliceFilter(new ColumnSlice[]{ staticSlice, slice }, limit, toGroup); + } + + List l = new ArrayList(startBounds.size()); + for (int i = 0; i < startBounds.size(); i++) + { + ColumnSlice slice = new ColumnSlice(startBounds.get(i), endBounds.get(i)); + if (!slice.isAlwaysEmpty(cfm.comparator, isReversed)) + l.add(slice); + } + + if (l.isEmpty()) + return staticSlice == null ? null : sliceFilter(staticSlice, limit, toGroup); + if (staticSlice == null) + return sliceFilter(l.toArray(new ColumnSlice[l.size()]), limit, toGroup); + + // The slices should not overlap. We know the slices built from startBounds/endBounds don't, but if there is + // a static slice, it could overlap with the 2nd slice. Check for it and correct if that's the case + ColumnSlice[] slices; + if (isReversed) + { + if (l.get(l.size() - 1).includes(cfm.comparator.reverseComparator(), staticSlice.start)) + { + slices = l.toArray(new ColumnSlice[l.size()]); + slices[slices.length-1] = new ColumnSlice(slices[slices.length-1].start, Composites.EMPTY); + } + else + { + slices = l.toArray(new ColumnSlice[l.size()+1]); + slices[slices.length-1] = staticSlice; + } + } + else + { + if (l.get(0).includes(cfm.comparator, staticSlice.finish)) + { + slices = new ColumnSlice[l.size()]; + slices[0] = new ColumnSlice(Composites.EMPTY, l.get(0).finish); + for (int i = 1; i < l.size(); i++) + slices[i] = l.get(i); + } + else + { + slices = new ColumnSlice[l.size()+1]; + slices[0] = staticSlice; + for (int i = 0; i < l.size(); i++) + slices[i+1] = l.get(i); + } + } + return sliceFilter(slices, limit, toGroup); + } + else + { + SortedSet cellNames = getRequestedColumns(options); + if (cellNames == null) // in case of IN () for the last column of the key + return null; + QueryProcessor.validateCellNames(cellNames, cfm.comparator); + return new NamesQueryFilter(cellNames, true); + } + } + + private SliceQueryFilter sliceFilter(ColumnSlice slice, int limit, int toGroup) + { + return sliceFilter(new ColumnSlice[]{ slice }, limit, toGroup); + } + + private SliceQueryFilter sliceFilter(ColumnSlice[] slices, int limit, int toGroup) + { + assert ColumnSlice.validateSlices(slices, cfm.comparator, isReversed) : String.format("Invalid slices: " + Arrays.toString(slices) + (isReversed ? " (reversed)" : "")); + return new SliceQueryFilter(slices, isReversed, limit, toGroup); + } + + private int getLimit(QueryOptions options) throws InvalidRequestException + { + if (limit != null) + { + ByteBuffer b = checkNotNull(limit.bindAndGet(options), "Invalid null value of limit"); + + try + { + Int32Type.instance.validate(b); + int l = Int32Type.instance.compose(b); + checkTrue(l > 0, "LIMIT must be strictly positive"); + return l; + } + catch (MarshalException e) + { + throw new InvalidRequestException("Invalid limit value"); + } + } + return Integer.MAX_VALUE; + } + + private int updateLimitForQuery(int limit) + { + // Internally, we don't support exclusive bounds for slices. Instead, we query one more element if necessary + // and exclude it later (in processColumnFamily) + return restrictions.isNonCompositeSliceWithExclusiveBounds() && limit != Integer.MAX_VALUE + ? limit + 1 + : limit; + } + + private SortedSet getRequestedColumns(QueryOptions options) throws InvalidRequestException + { + // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762 + // we always do a slice for CQL3 tables, so it's ok to ignore them here + assert !restrictions.isColumnRange(); + SortedSet columns = new TreeSet(cfm.comparator); + for (Composite composite : restrictions.getClusteringColumnsAsComposites(options)) + columns.addAll(addSelectedColumns(composite)); + return columns; + } + + private SortedSet addSelectedColumns(Composite prefix) + { + if (cfm.comparator.isDense()) + { + return FBUtilities.singleton(cfm.comparator.create(prefix, null), cfm.comparator); + } + else + { + SortedSet columns = new TreeSet(cfm.comparator); + + // We need to query the selected column as well as the marker + // column (for the case where the row exists but has no columns outside the PK) + // Two exceptions are "static CF" (non-composite non-compact CF) and "super CF" + // that don't have marker and for which we must query all columns instead + if (cfm.comparator.isCompound() && !cfm.isSuper()) + { + // marker + columns.add(cfm.comparator.rowMarker(prefix)); + + // selected columns + for (ColumnDefinition def : selection.getColumns()) + if (def.isRegular() || def.isStatic()) + columns.add(cfm.comparator.create(prefix, def)); + } + else + { + // We now that we're not composite so we can ignore static columns + for (ColumnDefinition def : cfm.regularColumns()) + columns.add(cfm.comparator.create(prefix, def)); + } + return columns; + } + } + + public List getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException + { + if (!restrictions.usesSecondaryIndexing()) + return Collections.emptyList(); + + List expressions = restrictions.getIndexExpressions(options); + + ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily()); + SecondaryIndexManager secondaryIndexManager = cfs.indexManager; + secondaryIndexManager.validateIndexSearchersForQuery(expressions); + + return expressions; + } + + private CellName makeExclusiveSliceBound(Bound bound, CellNameType type, QueryOptions options) throws InvalidRequestException + { + if (restrictions.areRequestedBoundsInclusive(bound)) + return null; + + return type.makeCellName(restrictions.getClusteringColumnsBounds(bound, options).get(0)); + } + + private Iterator applySliceRestriction(final Iterator cells, final QueryOptions options) throws InvalidRequestException + { + final CellNameType type = cfm.comparator; + + final CellName excludedStart = makeExclusiveSliceBound(Bound.START, type, options); + final CellName excludedEnd = makeExclusiveSliceBound(Bound.END, type, options); + + return Iterators.filter(cells, new Predicate() + { + public boolean apply(Cell c) + { + // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless + // the comparator is composite that is)), filter here + return !((excludedStart != null && type.compare(c.name(), excludedStart) == 0) + || (excludedEnd != null && type.compare(c.name(), excludedEnd) == 0)); + } + }); + } + + private ResultSet process(List rows, QueryOptions options, int limit, long now) throws InvalidRequestException + { + Selection.ResultSetBuilder result = selection.resultSetBuilder(now); + for (org.apache.cassandra.db.Row row : rows) + { + // Not columns match the query, skip + if (row.cf == null) + continue; + + processColumnFamily(row.key.getKey(), row.cf, options, now, result); + } + + ResultSet cqlRows = result.build(options.getProtocolVersion()); + + orderResults(cqlRows); + + // Internal calls always return columns in the comparator order, even when reverse was set + if (isReversed) + cqlRows.reverse(); + + // Trim result if needed to respect the user limit + cqlRows.trim(limit); + return cqlRows; + } + + // Used by ModificationStatement for CAS operations + void processColumnFamily(ByteBuffer key, ColumnFamily cf, QueryOptions options, long now, Selection.ResultSetBuilder result) + throws InvalidRequestException + { + CFMetaData cfm = cf.metadata(); + ByteBuffer[] keyComponents = null; + if (cfm.getKeyValidator() instanceof CompositeType) + { + keyComponents = ((CompositeType)cfm.getKeyValidator()).split(key); + } + else + { + keyComponents = new ByteBuffer[]{ key }; + } + + Iterator cells = cf.getSortedColumns().iterator(); + if (restrictions.isNonCompositeSliceWithExclusiveBounds()) + cells = applySliceRestriction(cells, options); + + CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells); + + // If there is static columns but there is no non-static row, then provided the select was a full + // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns) + // then we want to include the static columns in the result set (and we're done). + CQL3Row staticRow = iter.getStaticRow(); + if (staticRow != null && !iter.hasNext() && !restrictions.usesSecondaryIndexing() && restrictions.hasNoClusteringColumnsRestriction()) + { + result.newRow(options.getProtocolVersion()); + for (ColumnDefinition def : selection.getColumns()) + { + switch (def.kind) + { + case PARTITION_KEY: + result.add(keyComponents[def.position()]); + break; + case STATIC: + addValue(result, def, staticRow, options); + break; + default: + result.add((ByteBuffer)null); + } + } + return; + } + + while (iter.hasNext()) + { + CQL3Row cql3Row = iter.next(); + + // Respect requested order + result.newRow(options.getProtocolVersion()); + // Respect selection order + for (ColumnDefinition def : selection.getColumns()) + { + switch (def.kind) + { + case PARTITION_KEY: + result.add(keyComponents[def.position()]); + break; + case CLUSTERING_COLUMN: + result.add(cql3Row.getClusteringColumn(def.position())); + break; + case COMPACT_VALUE: + result.add(cql3Row.getColumn(null)); + break; + case REGULAR: + addValue(result, def, cql3Row, options); + break; + case STATIC: + addValue(result, def, staticRow, options); + break; + } + } + } + } + + private static void addValue(Selection.ResultSetBuilder result, ColumnDefinition def, CQL3Row row, QueryOptions options) + { + if (row == null) + { + result.add((ByteBuffer)null); + return; + } + + if (def.type.isMultiCell()) + { + List cells = row.getMultiCellColumn(def.name); + ByteBuffer buffer = cells == null + ? null + : ((CollectionType)def.type).serializeForNativeProtocol(cells, options.getProtocolVersion()); + result.add(buffer); + return; + } + + result.add(row.getColumn(def.name)); + } + + private boolean needsPostQueryOrdering() + { + // We need post-query ordering only for queries with IN on the partition key and an ORDER BY. + return restrictions.keyIsInRelation() && !parameters.orderings.isEmpty(); + } + + /** + * Orders results when multiple keys are selected (using IN) + */ + private void orderResults(ResultSet cqlRows) + { + if (cqlRows.size() == 0 || !needsPostQueryOrdering()) + return; + + Collections.sort(cqlRows.rows, orderingComparator); + } + + public static class RawStatement extends CFStatement + { + private final Parameters parameters; + private final List selectClause; + private final List whereClause; + private final Term.Raw limit; + + public RawStatement(CFName cfName, Parameters parameters, List selectClause, List whereClause, Term.Raw limit) + { + super(cfName); + this.parameters = parameters; + this.selectClause = selectClause; + this.whereClause = whereClause == null ? Collections.emptyList() : whereClause; + this.limit = limit; + } + + public ParsedStatement.Prepared prepare() throws InvalidRequestException + { + CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + VariableSpecifications boundNames = getBoundVariables(); + + Selection selection = selectClause.isEmpty() + ? Selection.wildcard(cfm) + : Selection.fromSelectors(cfm, selectClause); + + StatementRestrictions restrictions = prepareRestrictions(cfm, boundNames, selection); + + if (parameters.isDistinct) + validateDistinctSelection(cfm, selection, restrictions); + + Comparator> orderingComparator = null; + boolean isReversed = false; + + if (!parameters.orderings.isEmpty()) + { + verifyOrderingIsAllowed(restrictions); + orderingComparator = getOrderingComparator(cfm, selection, restrictions); + isReversed = isReversed(cfm); + } + + if (isReversed) + restrictions.reverse(); + + checkNeedsFiltering(restrictions); + + SelectStatement stmt = new SelectStatement(cfm, + boundNames.size(), + parameters, + selection, + restrictions, + isReversed, + orderingComparator, + prepareLimit(boundNames)); + + return new ParsedStatement.Prepared(stmt, boundNames); + } + + /** + * Prepares the restrictions. + * + * @param cfm the column family meta data + * @param boundNames the variable specifications + * @param selection the selection + * @return the restrictions + * @throws InvalidRequestException if a problem occurs while building the restrictions + */ + private StatementRestrictions prepareRestrictions(CFMetaData cfm, + VariableSpecifications boundNames, + Selection selection) throws InvalidRequestException + { + try + { + return new StatementRestrictions(cfm, + whereClause, + boundNames, + selection.containsOnlyStaticColumns(), + selection.containsACollection()); + } + catch (UnrecognizedEntityException e) + { + if (containsAlias(e.entity)) + throw invalidRequest("Aliases aren't allowed in the where clause ('%s')", e.relation); + throw e; + } + } + + /** Returns a Term for the limit or null if no limit is set */ + private Term prepareLimit(VariableSpecifications boundNames) throws InvalidRequestException + { + if (limit == null) + return null; + + Term prepLimit = limit.prepare(keyspace(), limitReceiver()); + prepLimit.collectMarkerSpecification(boundNames); + return prepLimit; + } + + private static void verifyOrderingIsAllowed(StatementRestrictions restrictions) throws InvalidRequestException + { + checkFalse(restrictions.usesSecondaryIndexing(), "ORDER BY with 2ndary indexes is not supported."); + checkFalse(restrictions.isKeyRange(), "ORDER BY is only supported when the partition key is restricted by an EQ or an IN."); + } + + private static void validateDistinctSelection(CFMetaData cfm, + Selection selection, + StatementRestrictions restrictions) + throws InvalidRequestException + { + Collection requestedColumns = selection.getColumns(); + for (ColumnDefinition def : requestedColumns) + checkFalse(!def.isPartitionKey() && !def.isStatic(), + "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", + def.name); + + // If it's a key range, we require that all partition key columns are selected so we don't have to bother + // with post-query grouping. + if (!restrictions.isKeyRange()) + return; + + for (ColumnDefinition def : cfm.partitionKeyColumns()) + checkTrue(requestedColumns.contains(def), + "SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name); + } + + private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException + { + checkFalse(containsAlias(column), "Aliases are not allowed in order by clause ('%s')", column); + checkFalse(true, "Order by on unknown column %s", column); + } + + private Comparator> getOrderingComparator(CFMetaData cfm, + Selection selection, + StatementRestrictions restrictions) + throws InvalidRequestException + { + if (!restrictions.keyIsInRelation()) + return null; + + Map orderingIndexes = getOrderingIndex(cfm, selection); + + List idToSort = new ArrayList(); + List> sorters = new ArrayList>(); + + for (ColumnIdentifier.Raw raw : parameters.orderings.keySet()) + { + ColumnIdentifier identifier = raw.prepare(cfm); + ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier); + idToSort.add(orderingIndexes.get(orderingColumn.name)); + sorters.add(orderingColumn.type); + } + return idToSort.size() == 1 ? new SingleColumnComparator(idToSort.get(0), sorters.get(0)) + : new CompositeComparator(sorters, idToSort); + } + + private Map getOrderingIndex(CFMetaData cfm, Selection selection) + throws InvalidRequestException + { + // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, + // even if we don't + // ultimately ship them to the client (CASSANDRA-4911). + Map orderingIndexes = new HashMap<>(); + for (ColumnIdentifier.Raw raw : parameters.orderings.keySet()) + { + ColumnIdentifier column = raw.prepare(cfm); + final ColumnDefinition def = cfm.getColumnDefinition(column); + if (def == null) + handleUnrecognizedOrderingColumn(column); + int index = selection.indexOf(def); + if (index < 0) + index = selection.addColumnForOrdering(def); + orderingIndexes.put(def.name, index); + } + return orderingIndexes; + } + + private boolean isReversed(CFMetaData cfm) throws InvalidRequestException + { + Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()]; + int i = 0; + for (Map.Entry entry : parameters.orderings.entrySet()) + { + ColumnIdentifier column = entry.getKey().prepare(cfm); + boolean reversed = entry.getValue(); + + ColumnDefinition def = cfm.getColumnDefinition(column); + if (def == null) + handleUnrecognizedOrderingColumn(column); + + checkTrue(def.isClusteringColumn(), + "Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column); + + checkTrue(i++ == def.position(), + "Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"); + + reversedMap[def.position()] = (reversed != def.isReversedType()); + } + + // Check that all boolean in reversedMap, if set, agrees + Boolean isReversed = null; + for (Boolean b : reversedMap) + { + // Column on which order is specified can be in any order + if (b == null) + continue; + + if (isReversed == null) + { + isReversed = b; + continue; + } + checkTrue(isReversed.equals(b), "Unsupported order by relation"); + } + assert isReversed != null; + return isReversed; + } + + /** If ALLOW FILTERING was not specified, this verifies that it is not needed */ + private void checkNeedsFiltering(StatementRestrictions restrictions) throws InvalidRequestException + { + // non-key-range non-indexed queries cannot involve filtering underneath + if (!parameters.allowFiltering && (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())) + { + // We will potentially filter data if either: + // - Have more than one IndexExpression + // - Have no index expression and the column filter is not the identity + checkFalse(restrictions.needFiltering(), + "Cannot execute this query as it might involve data filtering and " + + "thus may have unpredictable performance. If you want to execute " + + "this query despite the performance unpredictability, use ALLOW FILTERING"); + } + + // We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an + // inclusive slice and remove post-query the value that shouldn't be returned. One problem however is that + // if there is a user limit, that limit may make the query return before the end of the slice is reached, + // in which case, once we'll have removed bound post-query, we might end up with less results than + // requested which would be incorrect. For single-partition query, this is not a problem, we just ask for + // one more result (see updateLimitForQuery()) since that's enough to compensate for that problem. For key + // range however, each returned row may include one result that will have to be trimmed, so we would have + // to bump the query limit by N where N is the number of rows we will return, but we don't know that in + // advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather + // than answering with something that is wrong. + if (restrictions.isNonCompositeSliceWithExclusiveBounds() && restrictions.isKeyRange() && limit != null) + { + SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(restrictions.cfm); + throw invalidRequest("The query requests a restriction of rows with a strict bound (%s) over a range of partitions. " + + "This is not supported by the underlying storage engine for COMPACT tables if a LIMIT is provided. " + + "Please either make the condition non strict (%s) or remove the user LIMIT", rel, rel.withNonStrictOperator()); + } + } + + private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFMetaData cfm) + { + for (Relation r : whereClause) + { + // We only call this when sliceRestriction != null, i.e. for compact table with non composite comparator, + // so it can't be a MultiColumnRelation. + SingleColumnRelation rel = (SingleColumnRelation)r; + + if (cfm.getColumnDefinition(rel.getEntity().prepare(cfm)).isClusteringColumn() + && (rel.operator() == Operator.GT || rel.operator() == Operator.LT)) + return rel; + } + + // We're not supposed to call this method unless we know this can't happen + throw new AssertionError(); + } + + private boolean containsAlias(final ColumnIdentifier name) + { + return Iterables.any(selectClause, new Predicate() + { + public boolean apply(RawSelector raw) + { + return name.equals(raw.alias); + } + }); + } + + private ColumnSpecification limitReceiver() + { + return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance); + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("name", cfName) + .add("selectClause", selectClause) + .add("whereClause", whereClause) + .add("isDistinct", parameters.isDistinct) + .toString(); + } + } + + public static class Parameters + { + private final Map orderings; + private final boolean isDistinct; + private final boolean allowFiltering; + + public Parameters(Map orderings, + boolean isDistinct, + boolean allowFiltering) + { + this.orderings = orderings; + this.isDistinct = isDistinct; + this.allowFiltering = allowFiltering; + } + } + + /** + * Used in orderResults(...) method when single 'ORDER BY' condition where given + */ + private static class SingleColumnComparator implements Comparator> + { + private final int index; + private final Comparator comparator; + + public SingleColumnComparator(int columnIndex, Comparator orderer) + { + index = columnIndex; + comparator = orderer; + } + + public int compare(List a, List b) + { + return comparator.compare(a.get(index), b.get(index)); + } + } + + /** + * Used in orderResults(...) method when multiple 'ORDER BY' conditions where given + */ + private static class CompositeComparator implements Comparator> + { + private final List> orderTypes; + private final List positions; + + private CompositeComparator(List> orderTypes, List positions) + { + this.orderTypes = orderTypes; + this.positions = positions; + } + + public int compare(List a, List b) + { + for (int i = 0; i < positions.size(); i++) + { + Comparator type = orderTypes.get(i); + int columnPos = positions.get(i); + + ByteBuffer aValue = a.get(columnPos); + ByteBuffer bValue = b.get(columnPos); + + int comparison = type.compare(aValue, bValue); + + if (comparison != 0) + return comparison; + } + + return 0; + } + } +} diff --git a/cql3/statements/TruncateStatement.java b/cql3/statements/TruncateStatement.java new file mode 100644 index 0000000000..ef1c4a4d55 --- /dev/null +++ b/cql3/statements/TruncateStatement.java @@ -0,0 +1,84 @@ +/* + * 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.io.IOException; +import java.util.concurrent.TimeoutException; + +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.thrift.ThriftValidation; + +public class TruncateStatement extends CFStatement implements CQLStatement +{ + public TruncateStatement(CFName name) + { + super(name); + } + + public int getBoundTerms() + { + return 0; + } + + public Prepared prepare() throws InvalidRequestException + { + return new Prepared(this); + } + + public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException + { + state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.MODIFY); + } + + public void validate(ClientState state) throws InvalidRequestException + { + ThriftValidation.validateColumnFamily(keyspace(), columnFamily()); + } + + public ResultMessage execute(QueryState state, QueryOptions options) throws InvalidRequestException, TruncateException + { + try + { + StorageProxy.truncateBlocking(keyspace(), columnFamily()); + } + catch (UnavailableException e) + { + throw new TruncateException(e); + } + catch (TimeoutException e) + { + throw new TruncateException(e); + } + catch (IOException e) + { + throw new TruncateException(e); + } + return null; + } + + public ResultMessage executeInternal(QueryState state, QueryOptions options) + { + throw new UnsupportedOperationException(); + } +} diff --git a/cql3/statements/UpdateStatement.java b/cql3/statements/UpdateStatement.java new file mode 100644 index 0000000000..4c7660f42f --- /dev/null +++ b/cql3/statements/UpdateStatement.java @@ -0,0 +1,241 @@ +/* + * 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.*; + +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.composites.Composite; +import org.apache.cassandra.db.index.SecondaryIndexManager; +import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Pair; + +/** + * An UPDATE statement parsed from a CQL query statement. + * + */ +public class UpdateStatement extends ModificationStatement +{ + private static final Constants.Value EMPTY = new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER); + + private UpdateStatement(StatementType type, int boundTerms, CFMetaData cfm, Attributes attrs) + { + super(type, boundTerms, cfm, attrs); + } + + public boolean requireFullClusteringKey() + { + return true; + } + + public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, Composite prefix, UpdateParameters params) + throws InvalidRequestException + { + // Inserting the CQL row marker (see #4361) + // We always need to insert a marker for INSERT, because of the following situation: + // CREATE TABLE t ( k int PRIMARY KEY, c text ); + // INSERT INTO t(k, c) VALUES (1, 1) + // DELETE c FROM t WHERE k = 1; + // SELECT * FROM t; + // The last query should return one row (but with c == null). Adding the marker with the insert make sure + // the semantic is correct (while making sure a 'DELETE FROM t WHERE k = 1' does remove the row entirely) + // + // We do not insert the marker for UPDATE however, as this amount to updating the columns in the WHERE + // clause which is inintuitive (#6782) + // + // We never insert markers for Super CF as this would confuse the thrift side. + if (type == StatementType.INSERT && cfm.isCQL3Table() && !prefix.isStatic()) + cf.addColumn(params.makeColumn(cfm.comparator.rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER)); + + List updates = getOperations(); + + if (cfm.comparator.isDense()) + { + if (prefix.isEmpty()) + throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s", cfm.clusteringColumns().get(0))); + + // An empty name for the compact value is what we use to recognize the case where there is not column + // outside the PK, see CreateStatement. + if (!cfm.compactValueColumn().name.bytes.hasRemaining()) + { + // There is no column outside the PK. So no operation could have passed through validation + assert updates.isEmpty(); + new Constants.Setter(cfm.compactValueColumn(), EMPTY).execute(key, cf, prefix, params); + } + else + { + // dense means we don't have a row marker, so don't accept to set only the PK. See CASSANDRA-5648. + if (updates.isEmpty()) + throw new InvalidRequestException(String.format("Column %s is mandatory for this COMPACT STORAGE table", cfm.compactValueColumn().name)); + + for (Operation update : updates) + update.execute(key, cf, prefix, params); + } + } + else + { + for (Operation update : updates) + update.execute(key, cf, prefix, params); + } + + SecondaryIndexManager indexManager = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfId).indexManager; + if (indexManager.hasIndexes()) + { + for (Cell cell : cf) + { + // Indexed values must be validated by any applicable index. See CASSANDRA-3057/4240/8081 for more details + if (!indexManager.validate(cell)) + throw new InvalidRequestException(String.format("Can't index column value of size %d for index %s on %s.%s", + cell.value().remaining(), + cfm.getColumnDefinition(cell.name()).getIndexName(), + cfm.ksName, + cfm.cfName)); + } + } + } + + public static class ParsedInsert extends ModificationStatement.Parsed + { + private final List columnNames; + private final List columnValues; + + /** + * A parsed INSERT statement. + * + * @param name column family being operated on + * @param columnNames list of column names + * @param columnValues list of column values (corresponds to names) + * @param attrs additional attributes for statement (CL, timestamp, timeToLive) + */ + public ParsedInsert(CFName name, + Attributes.Raw attrs, + List columnNames, List columnValues, + boolean ifNotExists) + { + super(name, attrs, null, ifNotExists, false); + this.columnNames = columnNames; + this.columnValues = columnValues; + } + + protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException + { + UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT,boundNames.size(), cfm, attrs); + + // Created from an INSERT + if (stmt.isCounter()) + throw new InvalidRequestException("INSERT statement are not allowed on counter tables, use UPDATE instead"); + if (columnNames.size() != columnValues.size()) + throw new InvalidRequestException("Unmatched column names/values"); + if (columnNames.isEmpty()) + throw new InvalidRequestException("No columns provided to INSERT"); + + for (int i = 0; i < columnNames.size(); i++) + { + ColumnIdentifier id = columnNames.get(i).prepare(cfm); + ColumnDefinition def = cfm.getColumnDefinition(id); + if (def == null) + throw new InvalidRequestException(String.format("Unknown identifier %s", id)); + + for (int j = 0; j < i; j++) + { + ColumnIdentifier otherId = columnNames.get(j).prepare(cfm); + if (id.equals(otherId)) + throw new InvalidRequestException(String.format("Multiple definitions found for column %s", id)); + } + + Term.Raw value = columnValues.get(i); + + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + Term t = value.prepare(keyspace(), def); + t.collectMarkerSpecification(boundNames); + stmt.addKeyValue(def, t); + break; + default: + Operation operation = new Operation.SetValue(value).prepare(keyspace(), def); + operation.collectMarkerSpecification(boundNames); + stmt.addOperation(operation); + break; + } + } + return stmt; + } + } + + public static class ParsedUpdate extends ModificationStatement.Parsed + { + // Provided for an UPDATE + private final List> updates; + private final List whereClause; + + /** + * Creates a new UpdateStatement from a column family name, columns map, consistency + * level, and key term. + * + * @param name column family being operated on + * @param attrs additional attributes for statement (timestamp, timeToLive) + * @param updates a map of column operations to perform + * @param whereClause the where clause + */ + public ParsedUpdate(CFName name, + Attributes.Raw attrs, + List> updates, + List whereClause, + List> conditions) + { + super(name, attrs, conditions, false, false); + this.updates = updates; + this.whereClause = whereClause; + } + + protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException + { + UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.UPDATE, boundNames.size(), cfm, attrs); + + for (Pair entry : updates) + { + ColumnDefinition def = cfm.getColumnDefinition(entry.left.prepare(cfm)); + if (def == null) + throw new InvalidRequestException(String.format("Unknown identifier %s", entry.left)); + + Operation operation = entry.right.prepare(keyspace(), def); + operation.collectMarkerSpecification(boundNames); + + switch (def.kind) + { + case PARTITION_KEY: + case CLUSTERING_COLUMN: + throw new InvalidRequestException(String.format("PRIMARY KEY part %s found in SET part", entry.left)); + default: + stmt.addOperation(operation); + break; + } + } + + stmt.processWhereClause(whereClause, boundNames); + return stmt; + } + } +} diff --git a/cql3/statements/UseStatement.java b/cql3/statements/UseStatement.java new file mode 100644 index 0000000000..efda72dad4 --- /dev/null +++ b/cql3/statements/UseStatement.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; + +public class UseStatement extends ParsedStatement implements CQLStatement +{ + private final String keyspace; + + public UseStatement(String keyspace) + { + this.keyspace = keyspace; + } + + public int getBoundTerms() + { + return 0; + } + + public Prepared prepare() throws InvalidRequestException + { + return new Prepared(this); + } + + public void checkAccess(ClientState state) throws UnauthorizedException + { + state.validateLogin(); + } + + public void validate(ClientState state) throws InvalidRequestException + { + } + + public ResultMessage execute(QueryState state, QueryOptions options) throws InvalidRequestException + { + state.getClientState().setKeyspace(keyspace); + return new ResultMessage.SetKeyspace(keyspace); + } + + public ResultMessage executeInternal(QueryState state, QueryOptions options) + { + // Internal queries are exclusively on the system keyspace and 'use' is thus useless + throw new UnsupportedOperationException(); + } +}