This is an automated email from the ASF dual-hosted git repository. brandonwilliams pushed a commit to branch cassandra-4.0 in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit 2fcdaa5b76b77106108579cad60226130492e37d Merge: 5143bd81e8 1eccb2bc1f Author: Brandon Williams <[email protected]> AuthorDate: Wed Jun 14 11:34:51 2023 -0500 Merge branch 'cassandra-3.11' into cassandra-4.0 CHANGES.txt | 1 + .../statements/schema/CreateTableStatement.java | 10 +++- .../schema/CreateTableValidationTest.java | 59 ++++++++++++++++++++++ 3 files changed, 68 insertions(+), 2 deletions(-) diff --cc CHANGES.txt index e0ffe9061e,942651a37e..a7c17c7dcb --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,9 -1,7 +1,10 @@@ -3.11.16 +4.0.11 + * Track the amount of read data per row (CASSANDRA-18513) + * Fix Down nodes counter in nodetool describecluster (CASSANDRA-18512) + * Remove unnecessary shuffling of GossipDigests in Gossiper#makeRandomGossipDigest (CASSANDRA-18546) +Merged from 3.11: + * Fix error message handling when trying to use CLUSTERING ORDER with non-clustering column (CASSANDRA-17818 * Add keyspace and table name to exception message during ColumnSubselection deserialization (CASSANDRA-18346) - * Remove unnecessary String.format invocation in QueryProcessor when getting a prepared statement from cache (CASSANDRA-17202) Merged from 3.0: * Suppress CVE-2023-2976 (CASSANDRA-18562) * Remove dh_python use in Debian packaging (CASSANDRA-18558) diff --cc src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 1339ba39f7,0000000000..78b9d6e52e mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@@ -1,529 -1,0 +1,535 @@@ +/* + * 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.schema; + +import java.util.*; ++import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableSet; + +import org.apache.commons.lang3.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.audit.AuditLogContext; +import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.auth.DataResource; +import org.apache.cassandra.auth.IResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.schema.*; +import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; +import org.apache.cassandra.transport.Event.SchemaChange; +import org.apache.cassandra.transport.Event.SchemaChange.Change; +import org.apache.cassandra.transport.Event.SchemaChange.Target; + +import static java.util.Comparator.comparing; + +import static com.google.common.collect.Iterables.concat; + +public final class CreateTableStatement extends AlterSchemaStatement +{ + private static final Logger logger = LoggerFactory.getLogger(CreateTableStatement.class); + private final String tableName; + + private final Map<ColumnIdentifier, CQL3Type.Raw> rawColumns; + private final Set<ColumnIdentifier> staticColumns; + private final List<ColumnIdentifier> partitionKeyColumns; + private final List<ColumnIdentifier> clusteringColumns; + + private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder; + private final TableAttributes attrs; + + private final boolean ifNotExists; + private final boolean useCompactStorage; + + public CreateTableStatement(String keyspaceName, + String tableName, + + Map<ColumnIdentifier, CQL3Type.Raw> rawColumns, + Set<ColumnIdentifier> staticColumns, + List<ColumnIdentifier> partitionKeyColumns, + List<ColumnIdentifier> clusteringColumns, + + LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder, + TableAttributes attrs, + + boolean ifNotExists, + boolean useCompactStorage) + { + super(keyspaceName); + this.tableName = tableName; + + this.rawColumns = rawColumns; + this.staticColumns = staticColumns; + this.partitionKeyColumns = partitionKeyColumns; + this.clusteringColumns = clusteringColumns; + + this.clusteringOrder = clusteringOrder; + this.attrs = attrs; + + this.ifNotExists = ifNotExists; + this.useCompactStorage = useCompactStorage; + } + + public Keyspaces apply(Keyspaces schema) + { + KeyspaceMetadata keyspace = schema.getNullable(keyspaceName); + if (null == keyspace) + throw ire("Keyspace '%s' doesn't exist", keyspaceName); + + if (keyspace.hasTable(tableName)) + { + if (ifNotExists) + return schema; + + throw new AlreadyExistsException(keyspaceName, tableName); + } + + TableMetadata table = builder(keyspace.types).build(); + table.validate(); + + if (keyspace.createReplicationStrategy().hasTransientReplicas() + && table.params.readRepair != ReadRepairStrategy.NONE) + { + throw ire("read_repair must be set to 'NONE' for transiently replicated keyspaces"); + } + + return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.with(table))); + } + + SchemaChange schemaChangeEvent(KeyspacesDiff diff) + { + return new SchemaChange(Change.CREATED, Target.TABLE, keyspaceName, tableName); + } + + public void authorize(ClientState client) + { + client.ensureKeyspacePermission(keyspaceName, Permission.CREATE); + } + + @Override + Set<IResource> createdResources(KeyspacesDiff diff) + { + return ImmutableSet.of(DataResource.table(keyspaceName, tableName)); + } + + @Override + public AuditLogContext getAuditLogContext() + { + return new AuditLogContext(AuditLogEntryType.CREATE_TABLE, keyspaceName, tableName); + } + + public String toString() + { + return String.format("%s (%s, %s)", getClass().getSimpleName(), keyspaceName, tableName); + } + + public TableMetadata.Builder builder(Types types) + { + attrs.validate(); + TableParams params = attrs.asNewTableParams(); + + // use a TreeMap to preserve ordering across JDK versions (see CASSANDRA-9492) - important for stable unit tests + Map<ColumnIdentifier, CQL3Type> columns = new TreeMap<>(comparing(o -> o.bytes)); + rawColumns.forEach((column, type) -> columns.put(column, type.prepare(keyspaceName, types))); + + // check for nested non-frozen UDTs or collections in a non-frozen UDT + columns.forEach((column, type) -> + { + if (type.isUDT() && type.getType().isMultiCell()) + { + ((UserType) type.getType()).fieldTypes().forEach(field -> + { + if (field.isMultiCell()) + throw ire("Non-frozen UDTs with nested non-frozen collections are not supported"); + }); + } + }); + + /* + * Deal with PRIMARY KEY columns + */ + + HashSet<ColumnIdentifier> primaryKeyColumns = new HashSet<>(); + concat(partitionKeyColumns, clusteringColumns).forEach(column -> + { + CQL3Type type = columns.get(column); + if (null == type) + throw ire("Unknown column '%s' referenced in PRIMARY KEY for table '%s'", column, tableName); + + if (!primaryKeyColumns.add(column)) + throw ire("Duplicate column '%s' in PRIMARY KEY clause for table '%s'", column, tableName); + + if (type.getType().isMultiCell()) + { + if (type.isCollection()) + throw ire("Invalid non-frozen collection type %s for PRIMARY KEY column '%s'", type, column); + else + throw ire("Invalid non-frozen user-defined type %s for PRIMARY KEY column '%s'", type, column); + } + + if (type.getType().isCounter()) + throw ire("counter type is not supported for PRIMARY KEY column '%s'", column); + + if (type.getType().referencesDuration()) + throw ire("duration type is not supported for PRIMARY KEY column '%s'", column); + + if (staticColumns.contains(column)) + throw ire("Static column '%s' cannot be part of the PRIMARY KEY", column); + }); + + List<AbstractType<?>> partitionKeyTypes = new ArrayList<>(); + List<AbstractType<?>> clusteringTypes = new ArrayList<>(); + + partitionKeyColumns.forEach(column -> + { + CQL3Type type = columns.remove(column); + partitionKeyTypes.add(type.getType()); + }); + + clusteringColumns.forEach(column -> + { + CQL3Type type = columns.remove(column); + boolean reverse = !clusteringOrder.getOrDefault(column, true); + clusteringTypes.add(reverse ? ReversedType.getInstance(type.getType()) : type.getType()); + }); + - if (clusteringOrder.size() > clusteringColumns.size()) - throw ire("Only clustering columns can be defined in CLUSTERING ORDER directive"); ++ List<ColumnIdentifier> nonClusterColumn = clusteringOrder.keySet().stream() ++ .filter((id) -> !clusteringColumns.contains(id)) ++ .collect(Collectors.toList()); ++ if (!nonClusterColumn.isEmpty()) ++ { ++ throw ire("Only clustering key columns can be defined in CLUSTERING ORDER directive: " + nonClusterColumn + " are not clustering columns"); ++ } + + int n = 0; + for (ColumnIdentifier id : clusteringOrder.keySet()) + { + ColumnIdentifier c = clusteringColumns.get(n); + if (!id.equals(c)) + { + if (clusteringOrder.containsKey(c)) + throw ire("The order of columns in the CLUSTERING ORDER directive must match that of the clustering columns (%s must appear before %s)", c, id); + else + throw ire("Missing CLUSTERING ORDER for column %s", c); + } + ++n; + } + + // For COMPACT STORAGE, we reject any "feature" that we wouldn't be able to translate back to thrift. + if (useCompactStorage) + { + validateCompactTable(clusteringTypes, columns); + } + else + { + // Static columns only make sense if we have at least one clustering column. Otherwise everything is static anyway + if (clusteringColumns.isEmpty() && !staticColumns.isEmpty()) + throw ire("Static columns are only useful (and thus allowed) if the table has at least one clustering column"); + } + + /* + * Counter table validation + */ + + boolean hasCounters = rawColumns.values().stream().anyMatch(CQL3Type.Raw::isCounter); + if (hasCounters) + { + // We've handled anything that is not a PRIMARY KEY so columns only contains NON-PK columns. So + // if it's a counter table, make sure we don't have non-counter types + if (columns.values().stream().anyMatch(t -> !t.getType().isCounter())) + throw ire("Cannot mix counter and non counter columns in the same table"); + + if (params.defaultTimeToLive > 0) + throw ire("Cannot set %s on a table with counters", TableParams.Option.DEFAULT_TIME_TO_LIVE); + } + + /* + * Create the builder + */ + + TableMetadata.Builder builder = TableMetadata.builder(keyspaceName, tableName); + + if (attrs.hasProperty(TableAttributes.ID)) + builder.id(attrs.getId()); + + builder.isCounter(hasCounters) + .params(params); + + for (int i = 0; i < partitionKeyColumns.size(); i++) + builder.addPartitionKeyColumn(partitionKeyColumns.get(i), partitionKeyTypes.get(i)); + + for (int i = 0; i < clusteringColumns.size(); i++) + builder.addClusteringColumn(clusteringColumns.get(i), clusteringTypes.get(i)); + + if (useCompactStorage) + { + fixupCompactTable(clusteringTypes, columns, hasCounters, builder); + } + else + { + columns.forEach((column, type) -> { + if (staticColumns.contains(column)) + builder.addStaticColumn(column, type.getType()); + else + builder.addRegularColumn(column, type.getType()); + }); + } + return builder; + } + + private void validateCompactTable(List<AbstractType<?>> clusteringTypes, + Map<ColumnIdentifier, CQL3Type> columns) + { + boolean isDense = !clusteringTypes.isEmpty(); + + if (columns.values().stream().anyMatch(c -> c.getType().isMultiCell())) + throw ire("Non-frozen collections and UDTs are not supported with COMPACT STORAGE"); + if (!staticColumns.isEmpty()) + throw ire("Static columns are not supported in COMPACT STORAGE tables"); + + if (clusteringTypes.isEmpty()) + { + // It's a thrift "static CF" so there should be some columns definition + if (columns.isEmpty()) + throw ire("No definition found that is not part of the PRIMARY KEY"); + } + + if (isDense) + { + // We can have no columns (only the PK), but we can't have more than one. + if (columns.size() > 1) + throw ire(String.format("COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: %s)", StringUtils.join(columns.keySet(), ", "))); + } + else + { + // we are in the "static" case, so we need at least one column defined. For non-compact however, having + // just the PK is fine. + if (columns.isEmpty()) + throw ire("COMPACT STORAGE with non-composite PRIMARY KEY require one column not part of the PRIMARY KEY, none given"); + } + } + + private void fixupCompactTable(List<AbstractType<?>> clusteringTypes, + Map<ColumnIdentifier, CQL3Type> columns, + boolean hasCounters, + TableMetadata.Builder builder) + { + Set<TableMetadata.Flag> flags = EnumSet.noneOf(TableMetadata.Flag.class); + boolean isDense = !clusteringTypes.isEmpty(); + boolean isCompound = clusteringTypes.size() > 1; + + if (isDense) + flags.add(TableMetadata.Flag.DENSE); + if (isCompound) + flags.add(TableMetadata.Flag.COMPOUND); + if (hasCounters) + flags.add(TableMetadata.Flag.COUNTER); + + boolean isStaticCompact = !isDense && !isCompound; + + builder.flags(flags); + + columns.forEach((name, type) -> { + // Note that for "static" no-clustering compact storage we use static for the defined columns + if (staticColumns.contains(name) || isStaticCompact) + builder.addStaticColumn(name, type.getType()); + else + builder.addRegularColumn(name, type.getType()); + }); + + DefaultNames names = new DefaultNames(builder.columnNames()); + // Compact tables always have a clustering and a single regular value. + if (isStaticCompact) + { + builder.addClusteringColumn(names.defaultClusteringName(), UTF8Type.instance); + builder.addRegularColumn(names.defaultCompactValueName(), hasCounters ? CounterColumnType.instance : BytesType.instance); + } + else if (!builder.hasRegularColumns()) + { + // Even for dense, we might not have our regular column if it wasn't part of the declaration. If + // that's the case, add it but with a specific EmptyType so we can recognize that case later + builder.addRegularColumn(names.defaultCompactValueName(), EmptyType.instance); + } + } + + @Override + public Set<String> clientWarnings(KeyspacesDiff diff) + { + int tableCount = Schema.instance.getNumberOfTables(); + if (tableCount > DatabaseDescriptor.tableCountWarnThreshold()) + { + String msg = String.format("Cluster already contains %d tables in %d keyspaces. Having a large number of tables will significantly slow down schema dependent cluster operations.", + tableCount, + Schema.instance.getKeyspaces().size()); + logger.warn(msg); + return ImmutableSet.of(msg); + } + return ImmutableSet.of(); + } + + private static class DefaultNames + { + private static final String DEFAULT_CLUSTERING_NAME = "column"; + private static final String DEFAULT_COMPACT_VALUE_NAME = "value"; + + private final Set<String> usedNames; + private int clusteringIndex = 1; + private int compactIndex = 0; + + private DefaultNames(Set<String> usedNames) + { + this.usedNames = usedNames; + } + + public String defaultClusteringName() + { + while (true) + { + String candidate = DEFAULT_CLUSTERING_NAME + clusteringIndex; + ++clusteringIndex; + if (usedNames.add(candidate)) + return candidate; + } + } + + public String defaultCompactValueName() + { + while (true) + { + String candidate = compactIndex == 0 ? DEFAULT_COMPACT_VALUE_NAME : DEFAULT_COMPACT_VALUE_NAME + compactIndex; + ++compactIndex; + if (usedNames.add(candidate)) + return candidate; + } + } + } + + public static TableMetadata.Builder parse(String cql, String keyspace) + { + return CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE") + .keyspace(keyspace) + .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock + .builder(Types.none()); + } + + public final static class Raw extends CQLStatement.Raw + { + private final QualifiedName name; + private final boolean ifNotExists; + + private boolean useCompactStorage = false; + private final Map<ColumnIdentifier, CQL3Type.Raw> rawColumns = new HashMap<>(); + private final Set<ColumnIdentifier> staticColumns = new HashSet<>(); + private final List<ColumnIdentifier> clusteringColumns = new ArrayList<>(); + + private List<ColumnIdentifier> partitionKeyColumns; + + private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder = new LinkedHashMap<>(); + public final TableAttributes attrs = new TableAttributes(); + + public Raw(QualifiedName name, boolean ifNotExists) + { + this.name = name; + this.ifNotExists = ifNotExists; + } + + public CreateTableStatement prepare(ClientState state) + { + String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace(); + + if (null == partitionKeyColumns) + throw ire("No PRIMARY KEY specifed for table '%s' (exactly one required)", name); + + return new CreateTableStatement(keyspaceName, + name.getName(), + + rawColumns, + staticColumns, + partitionKeyColumns, + clusteringColumns, + + clusteringOrder, + attrs, + + ifNotExists, + useCompactStorage); + } + + public String keyspace() + { + return name.getKeyspace(); + } + + public Raw keyspace(String keyspace) + { + name.setKeyspace(keyspace, true); + return this; + } + + public String table() + { + return name.getName(); + } + + public void addColumn(ColumnIdentifier column, CQL3Type.Raw type, boolean isStatic) + { + if (null != rawColumns.put(column, type)) + throw ire("Duplicate column '%s' declaration for table '%s'", column, name); + + if (isStatic) + staticColumns.add(column); + } + + public void setCompactStorage() + { + useCompactStorage = true; + } + + public void setPartitionKeyColumn(ColumnIdentifier column) + { + setPartitionKeyColumns(Collections.singletonList(column)); + } + + public void setPartitionKeyColumns(List<ColumnIdentifier> columns) + { + if (null != partitionKeyColumns) + throw ire("Multiple PRIMARY KEY specified for table '%s' (exactly one required)", name); + + partitionKeyColumns = columns; + } + + public void markClusteringColumn(ColumnIdentifier column) + { + clusteringColumns.add(column); + } + + public void extendClusteringOrder(ColumnIdentifier column, boolean ascending) + { + if (null != clusteringOrder.put(column, ascending)) + throw ire("Duplicate column '%s' in CLUSTERING ORDER BY clause for table '%s'", column, name); + } + } +} diff --cc test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java index f2abc7c663,7ff313c425..6662fccb39 --- a/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java +++ b/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java @@@ -18,21 -18,13 +18,23 @@@ */ package org.apache.cassandra.schema; +import java.io.IOException; +import java.util.List; + +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.exceptions.ConfigurationException; + import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.transport.Message; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.transport.SimpleClient; +import org.apache.cassandra.transport.messages.QueryMessage; import org.junit.Test; + import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class CreateTableValidationTest extends CQLTester @@@ -58,49 -52,59 +60,106 @@@ createTable("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 0.1"); } + @Test + public void testCreateKeyspaceTableWarning() throws IOException + { + requireNetwork(); + int tableCountWarn = DatabaseDescriptor.tableCountWarnThreshold(); + int keyspaceCountWarn = DatabaseDescriptor.keyspaceCountWarnThreshold(); + DatabaseDescriptor.setTableCountWarnThreshold(Schema.instance.getNumberOfTables()); + DatabaseDescriptor.setKeyspaceCountWarnThreshold(Schema.instance.getKeyspaces().size()); + + try (SimpleClient client = newSimpleClient(ProtocolVersion.CURRENT).connect(false)) + { + String createKeyspace = "CREATE KEYSPACE createkswarning%d WITH REPLICATION={'class':'org.apache.cassandra.locator.NetworkTopologyStrategy','datacenter1':'2'}"; + QueryMessage query = new QueryMessage(String.format(createKeyspace, 1), QueryOptions.DEFAULT); + Message.Response resp = client.execute(query); + List<String> warns = resp.getWarnings(); + warns.removeIf(w -> w.contains("is higher than the number of nodes")); + assertTrue(warns.size() > 0); + assertTrue(warns.get(0).contains("Having a large number of keyspaces will significantly")); + + DatabaseDescriptor.setKeyspaceCountWarnThreshold(Schema.instance.getKeyspaces().size() + 2); + query = new QueryMessage(String.format(createKeyspace, 2), QueryOptions.DEFAULT); + resp = client.execute(query); + warns = resp.getWarnings(); + if (warns != null) + warns.removeIf(w -> w.contains("is higher than the number of nodes")); + assertTrue(warns == null || warns.isEmpty()); + + query = new QueryMessage(String.format("CREATE TABLE %s.%s (id int primary key, x int)", KEYSPACE, "test1"), QueryOptions.DEFAULT); + resp = client.execute(query); + warns = resp.getWarnings(); + warns.removeIf(w -> w.contains("is higher than the number of nodes")); + assertTrue(warns.size() > 0); + assertTrue(warns.get(0).contains("Having a large number of tables")); + + DatabaseDescriptor.setTableCountWarnThreshold(Schema.instance.getNumberOfTables() + 1); + query = new QueryMessage(String.format("CREATE TABLE %s.%s (id int primary key, x int)", KEYSPACE, "test2"), QueryOptions.DEFAULT); + resp = client.execute(query); + assertTrue(resp.getWarnings() == null || resp.getWarnings().isEmpty()); + } + finally + { + DatabaseDescriptor.setTableCountWarnThreshold(tableCountWarn); + DatabaseDescriptor.setKeyspaceCountWarnThreshold(keyspaceCountWarn); + } + } ++ + @Test + public void testCreateTableOnSelectedClusteringColumn() + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + } + + @Test + public void testCreateTableOnAllClusteringColumns() + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC, ck2 DESC);"); + } ++ + @Test + public void testCreateTableErrorOnNonClusteringKey() + { + String expectedMessage = "Only clustering key columns can be defined in CLUSTERING ORDER directive"; + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC, ck2 DESC, v ASC);", + expectedMessage+": [v]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (v ASC);", + expectedMessage+": [v]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (pk ASC);", + expectedMessage+": [pk]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (pk ASC, ck1 DESC);", + expectedMessage+": [pk]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC, ck2 DESC, pk DESC);", + expectedMessage+": [pk]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (pk DESC, v DESC);", + expectedMessage+": [pk, v]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (pk DESC, v DESC, ck1 DESC);", + expectedMessage+": [pk, v]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC, v ASC);", + expectedMessage+": [v]"); + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (v ASC, ck1 DESC);", + expectedMessage+": [v]"); + } + + @Test + public void testCreateTableInWrongOrder() + { + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck2 ASC, ck1 DESC);", - "The order of columns in the CLUSTERING ORDER directive must be the one of the clustering key"); ++ "The order of columns in the CLUSTERING ORDER directive must match that of the clustering columns"); + } + + @Test + public void testCreateTableWithMissingClusteringColumn() + { + expectedFailure("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck2 ASC);", + "Missing CLUSTERING ORDER for column ck1"); + } + + private void expectedFailure(String statement, String errorMsg) + { ++ + assertThatExceptionOfType(InvalidRequestException.class) + .isThrownBy(() -> createTableMayThrow(statement)) .withMessageContaining(errorMsg); - + } } --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
