diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index eccfc2ebb246..c9276b7ff786 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -815,12 +815,14 @@ tableClusteringOrder[CreateTableStatement.Raw stmt] ; /** - * CREATE TABLE [IF NOT EXISTS] LIKE + * CREATE TABLE [IF NOT EXISTS] LIKE WITH = AND ...; */ copyTableStatement returns [CopyTableStatement.Raw stmt] @init { boolean ifNotExists = false; } - : K_CREATE K_COLUMNFAMILY newCf=columnFamilyName K_LIKE oldCf=columnFamilyName - { $stmt = new CopyTableStatement.Raw(newCf, oldCf); } + : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + newCf=columnFamilyName K_LIKE oldCf=columnFamilyName + { $stmt = new CopyTableStatement.Raw(newCf, oldCf, ifNotExists); } + ( K_WITH property[stmt.attrs] ( K_AND property[stmt.attrs] )*)? ; /** diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java index 43419fb55430..1de74b55a96a 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java @@ -30,6 +30,7 @@ import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.schema.Triggers; import org.apache.cassandra.schema.UserFunctions; import org.apache.cassandra.service.ClientState; @@ -37,23 +38,32 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Event.SchemaChange; +/** + * {@code CREATE TABLE [IF NOT EXISTS] LIKE WITH = } + */ public final class CopyTableStatement extends AlterSchemaStatement { private final String sourceKeyspace; private final String sourceTableName; private final String targetKeyspace; private final String targetTableName; + private final boolean ifNotExists; + private final TableAttributes attrs; public CopyTableStatement(String sourceKeyspace, String targetKeyspace, String sourceTableName, - String targetTableName) + String targetTableName, + boolean ifNotExists, + TableAttributes attrs) { super(targetKeyspace); this.sourceKeyspace = sourceKeyspace; this.targetKeyspace = targetKeyspace; this.sourceTableName = sourceTableName; this.targetTableName = targetTableName; + this.ifNotExists = ifNotExists; + this.attrs = attrs; } @Override @@ -89,21 +99,25 @@ public Keyspaces apply(ClusterMetadata metadata) throw ire("Souce Table '%s'.'%s' doesn't exist", sourceKeyspace, sourceTableName); if (sourceTableMeta.isIndex()) - throw ire("Cannot use CTREATE TABLE LIKE on a index table '%s'.'%s'.", sourceKeyspace, sourceTableName); + throw ire("Cannot use CREATE TABLE LIKE on a index table '%s'.'%s'.", sourceKeyspace, sourceTableName); if (sourceTableMeta.isView()) - throw ire("Cannot use CTREATE TABLE LIKE on a materialized view '%s'.'%s'.", sourceKeyspace, sourceTableName); + throw ire("Cannot use CREATE TABLE LIKE on a materialized view '%s'.'%s'.", sourceKeyspace, sourceTableName); KeyspaceMetadata targetKeyspaceMeta = schema.getNullable(targetKeyspace); if (null == targetKeyspaceMeta) throw ire("Target Keyspace '%s' doesn't exist", targetKeyspace); if (targetKeyspaceMeta.hasTable(targetTableName)) - throw new AlreadyExistsException(targetKeyspace, targetTableName); + { + if(ifNotExists) + return schema; + throw new AlreadyExistsException(targetKeyspace, targetTableName); + } // todo support udt for differenet ks latter if (!sourceKeyspace.equalsIgnoreCase(targetKeyspace) && !sourceKeyspaceMeta.types.isEmpty()) - throw ire("Cannot use CTREATE TABLE LIKE across different keyspace when source table have UDTs."); + throw ire("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDTs."); String sourceCQLString = sourceTableMeta.toCqlString(false, false, true, false); // add all user functions to be able to give a good error message to the user if the alter references @@ -112,7 +126,6 @@ public Keyspaces apply(ClusterMetadata metadata) for (KeyspaceMetadata ksm : schema) ufBuilder.add(ksm.userFunctions); - //todo support table params' setting in the future TableMetadata.Builder targetBuilder = CreateTableStatement.parse(sourceCQLString, targetKeyspace, targetTableName, @@ -121,7 +134,11 @@ public Keyspaces apply(ClusterMetadata metadata) .indexes(Indexes.none()) .triggers(Triggers.none()); - TableMetadata table = targetBuilder.id(TableId.get(metadata)).build(); + TableParams originalParams = targetBuilder.build().params; + TableParams newTableParams = attrs.asAlteredTableParams(originalParams); + TableMetadata table = targetBuilder.params(newTableParams) + .id(TableId.get(metadata)) + .build(); table.validate(); if (targetKeyspaceMeta.replicationStrategy.hasTransientReplicas() @@ -140,11 +157,14 @@ public final static class Raw extends CQLStatement.Raw { private final QualifiedName oldName; private final QualifiedName newName; + private final boolean ifNotExists; + public final TableAttributes attrs = new TableAttributes(); - public Raw(QualifiedName newName, QualifiedName oldName) + public Raw(QualifiedName newName, QualifiedName oldName, boolean ifNotExists) { this.newName = newName; this.oldName = oldName; + this.ifNotExists = ifNotExists; } @Override @@ -152,7 +172,7 @@ public CQLStatement prepare(ClientState state) { String oldKeyspace = oldName.hasKeyspace() ? oldName.getKeyspace() : state.getKeyspace(); String newKeyspace = newName.hasKeyspace() ? newName.getKeyspace() : state.getKeyspace(); - return new CopyTableStatement(oldKeyspace, newKeyspace, oldName.getName(), newName.getName()); + return new CopyTableStatement(oldKeyspace, newKeyspace, oldName.getName(), newName.getName(), ifNotExists, attrs); } } } diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index 819a74164381..cb3c879a94b9 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -313,15 +313,6 @@ public boolean equals(Object o) return equalsWithoutType(cd) && type.equals(cd.type); } - protected boolean equalsWithoutKsTb(ColumnMetadata other) - { - return name.equals(other.name) - && kind == other.kind - && position == other.position - && Objects.equals(mask, other.mask) - && type.equals(other.type); - } - private boolean equalsWithoutType(ColumnMetadata other) { return name.equals(other.name) diff --git a/src/java/org/apache/cassandra/schema/DroppedColumn.java b/src/java/org/apache/cassandra/schema/DroppedColumn.java index 17f4c55d2b45..0cd001fa542a 100644 --- a/src/java/org/apache/cassandra/schema/DroppedColumn.java +++ b/src/java/org/apache/cassandra/schema/DroppedColumn.java @@ -57,11 +57,6 @@ public boolean equals(Object o) return column.equals(dc.column) && droppedTime == dc.droppedTime; } - public boolean equalsWithoutKsTb(DroppedColumn other) - { - return column.equalsWithoutKsTb(other.column) && droppedTime == other.droppedTime; - } - @Override public int hashCode() { diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index c99fe3b8b97c..889264d43f31 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -716,33 +716,6 @@ private boolean equalsWithoutColumns(TableMetadata tm) && triggers.equals(tm.triggers); } - public boolean equalsWithoutTableNameAndDropCns(TableMetadata tm) - { - return partitioner.equals(tm.partitioner) - && kind == tm.kind - && params.equals(tm.params) - && flags.equals(tm.flags) - && indexes.equals(tm.indexes) - && triggers.equals(tm.triggers) - && columnsEqualWitoutKsTb(tm); - } - - // only compare columns - private boolean columnsEqualWitoutKsTb(TableMetadata tm) - { - if (columns == tm.columns) - return true; - - boolean result = true; - for (Map.Entry entry : columns.entrySet()) - { - ColumnMetadata thisColumn = entry.getValue(); - ColumnMetadata thatColumn = tm.columns.get(entry.getKey()); - result &= thatColumn != null && thisColumn.equalsWithoutKsTb(thatColumn); - } - return result; - } - Optional compare(TableMetadata other) { return equalsWithoutColumns(other) diff --git a/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java b/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java index 6ce958f10cb6..0cf551e2e875 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java @@ -85,11 +85,11 @@ public void testCreateLikeTable() throws IOException { try (Cluster cluster = init(Cluster.build(2).start())) { - cluster.schemaChange(withKeyspace("create table %s.sourcetb (k int primary key, v text)")); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.sourcetb (k int primary key, v text)")); TableId node1id = tableId(cluster.get(1), "sourcetb"); TableId node2id = tableId(cluster.get(2), "sourcetb"); assertEquals(node1id, node2id); - cluster.schemaChange("create table " + KEYSPACE + ".targettb like " + KEYSPACE + ".sourcetb"); + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".targettb LIKE " + KEYSPACE + ".sourcetb"); TableId node1id2 = tableId(cluster.get(1), "targettb"); TableId node2id2 = tableId(cluster.get(2), "targettb"); assertNotEquals(node1id, node1id2); diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java index 5ca56996799c..a202c0bacb55 100644 --- a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java +++ b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java @@ -540,7 +540,7 @@ public void testCqlCreateTableLikeAuditing() throws Throwable String sourceTable = currentTable(); - cql = "CREATE TABLE " + KEYSPACE + "." + createTableName() + " like " + KEYSPACE + "." + sourceTable; + cql = "CREATE TABLE " + KEYSPACE + "." + createTableName() + " LIKE " + KEYSPACE + "." + sourceTable; executeAndAssert(cql, AuditLogEntryType.CREATE_TABLE_LIKE); cql = "INSERT INTO " + KEYSPACE + '.' + currentTable() + " (id, v1, v2) VALUES (?, ?, ?)"; diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java index 30dd9a242c00..d7af87dbf5f4 100644 --- a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java +++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java @@ -405,7 +405,7 @@ public void testCreateTableLikeAuthorize() throws Throwable useUser(user, pass); assertUnauthorizedQuery("User user has no SELECT permission on or any of its parents", - "CREATE TABLE ks1.targetTb like ks1.sourcetb"); + "CREATE TABLE ks1.targetTb LIKE ks1.sourcetb"); // has select permission on source table no create permission on target keyspace useSuperUser(); @@ -415,7 +415,7 @@ public void testCreateTableLikeAuthorize() throws Throwable useUser(user, pass); assertUnauthorizedQuery("User user has no CREATE permission on or any of its parents", - "CREATE TABLE ks1.targetTb like ks1.sourcetb"); + "CREATE TABLE ks1.targetTb LIKE ks1.sourcetb"); // different keyspaces // has select permission on source table no create permission on target keyspace @@ -426,7 +426,16 @@ public void testCreateTableLikeAuthorize() throws Throwable useUser(user, pass); assertUnauthorizedQuery("User user has no CREATE permission on or any of its parents", - "CREATE TABLE ks2.targetTb like ks1.sourcetb"); + "CREATE TABLE ks2.targetTb LIKE ks1.sourcetb"); + + // source keyspace and table does not exists + assertUnauthorizedQuery("User user has no SELECT permission on
or any of its parents", + "CREATE TABLE ks2.targetTb LIKE ks1.tbnotexist"); + assertUnauthorizedQuery("User user has no SELECT permission on
or any of its parents", + "CREATE TABLE ks2.targetTb LIKE ksnotexists.sourcetb"); + // target keyspace does not exists + assertUnauthorizedQuery("User user has no CREATE permission on or any of its parents", + "CREATE TABLE ksnotexists.targetTb LIKE ks1.sourcetb"); } } diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index f7b579bfb5c6..ea3a9efc3164 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -152,7 +152,6 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.index.Index; import org.apache.cassandra.index.SecondaryIndexManager; @@ -164,6 +163,7 @@ import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Schema; @@ -205,10 +205,7 @@ import static org.apache.cassandra.cql3.SchemaElement.SchemaElementType.TYPE; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.createMetricsKeyspaceTables; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; -import static org.assertj.core.api.Assertions.assertThatExceptionOfType; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -1082,11 +1079,6 @@ protected String createTable(String keyspace, String query, String tableName) return currentTable; } - protected String createTableLike(String query, String sourceTable) - { - return createTableLike(query, sourceTable, KEYSPACE, null, KEYSPACE); - } - protected String createTableLike(String query, String sourceTable, String sourceKeyspace, String targetKeyspace) { return createTableLike(query, sourceTable, sourceKeyspace, null, targetKeyspace); @@ -1096,11 +1088,11 @@ protected String createTableLike(String query, String sourceTable, String source { if (!tables.contains(sourceTable)) { - throw new IllegalArgumentException("Source table " + sourceTable + " is not exists"); + throw new IllegalArgumentException("Source table " + sourceTable + " does not exist"); } String currentTable = createTableName(targetTable); - String fullQuery = formatQuery(sourceKeyspace, sourceTable, targetKeyspace, query); + String fullQuery = currentTable == null ? query : String.format(query, targetKeyspace + "." + currentTable, sourceKeyspace + "." + sourceTable);; logger.info(fullQuery); schemaChange(fullQuery); return currentTable; @@ -1546,13 +1538,6 @@ protected static void assertNoWarningContains(List warnings, String mess } } - protected void expectedFailure(final Class exceptionType, String statement, String errorMsg) - { - - assertThatExceptionOfType(exceptionType) - .isThrownBy(() -> createTableMayThrow(statement)) .withMessageContaining(errorMsg); - } - protected static ResultMessage schemaChange(String query) { try @@ -1578,11 +1563,6 @@ protected static ResultMessage schemaChange(String query) } } - protected TableMetadata getTableMetadata(String table) - { - return Schema.instance.getTableMetadata(KEYSPACE, table); - } - protected TableMetadata getTableMetadata(String keyspace, String table) { return Schema.instance.getTableMetadata(keyspace, table); @@ -1673,12 +1653,6 @@ protected final String formatQuery(String keyspace, String query) return currentTable == null ? query : String.format(query, keyspace + "." + currentTable); } - protected final String formatQuery(String sourceKeyspace, String sourceTable, String targetKeyspace, String query) - { - String currentTable = currentTable(); - return currentTable == null ? query : String.format(query, targetKeyspace + "." + currentTable, sourceKeyspace + "." + sourceTable); - } - public String formatViewQuery(String query) { return formatViewQuery(KEYSPACE, query); @@ -1984,20 +1958,57 @@ private static boolean isEmptyContainerNull(DataType type, return false; } - protected Pair assertTableMetaEquals(String sourceKeyspace, String targetKeyspace, String sourceTable, String targetTable) - { - TableMetadata sourceTbMeta = getTableMetadata(sourceKeyspace, sourceTable); - TableMetadata targetTbMeta = getTableMetadata(targetKeyspace, targetTable); - assertNotNull(sourceTbMeta); - assertNotNull(targetTbMeta); - assertTrue(sourceTbMeta.equalsWithoutTableNameAndDropCns(targetTbMeta)); - targetTbMeta.columns().stream().forEach(columnMetadata -> { - assertEquals(columnMetadata.ksName, targetKeyspace); - assertEquals(columnMetadata.cfName, targetTable); - }); - assertNotEquals(sourceTbMeta.id, targetTbMeta.id); - assertNotEquals(sourceTbMeta.name, targetTbMeta.name); - return Pair.create(sourceTbMeta, targetTbMeta); + /** + * Determine whether the source and target TableMetadata is equal without compare the table name and dropped columns. + * @param source the source TableMetadata + * @param target the target TableMetadata + * @param compareParams wether compare table params + * @param compareIndexes wether compare table's indexes + * @param compareTrigger wether compare table's triggers + * */ + protected boolean equalsWithoutTableNameAndDropCns(TableMetadata source, TableMetadata target, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + { + return source.partitioner.equals(target.partitioner) + && source.kind == target.kind + && source.flags.equals(target.flags) + && (!compareParams || source.params.equals(target.params)) + && (!compareIndexes || source.indexes.equals(target.indexes)) + && (!compareTrigger || source.triggers.equals(target.triggers)) + && columnsEqualWitoutKsTb(source, target); + } + + // only compare columns + private boolean columnsEqualWitoutKsTb(TableMetadata source, TableMetadata target) + { + if (target.columns() == source.columns()) + return true; + + List left = source.columns().stream().sorted().collect(Collectors.toList()); + List right = target.columns().stream().sorted().collect(Collectors.toList()); + + if (left.size() != right.size()) + return false; + + Iterator leftIterator = left.iterator(); + Iterator rightIterator = right.iterator(); + while (leftIterator.hasNext() && rightIterator.hasNext()) + { + ColumnMetadata leftCn = leftIterator.next(); + ColumnMetadata rightCn = rightIterator.next(); + if (!equalsWithoutKsTb(leftCn, rightCn)) + return false; + } + + return true; + } + + private boolean equalsWithoutKsTb(ColumnMetadata left, ColumnMetadata right) + { + return left.name.equals(right.name) + && left.kind == right.kind + && left.position() == right.position() + && java.util.Objects.equals(left.getMask(), right.getMask()) + && left.type.equals(right.type); } protected void assertRowCountNet(ResultSet r1, int expectedCount) diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 80aea816e92c..8fb930bffa69 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -43,7 +43,6 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.ProtocolVersion; -import org.apache.cassandra.utils.Pair; import static java.lang.String.format; import static org.apache.cassandra.schema.SchemaConstants.AUTH_KEYSPACE_NAME; @@ -54,6 +53,7 @@ import static org.apache.cassandra.schema.SchemaConstants.TRACE_KEYSPACE_NAME; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_SCHEMA; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -838,8 +838,14 @@ public void testDescribeCreateLikeTable() throws Throwable " v1 int, " + " v2 int, " + "PRIMARY KEY ((pk1, pk2), ck1, ck2 ))"); - String targetTable = createTableLike("create table %s like %s", souceTable, KEYSPACE_PER_TEST, KEYSPACE_PER_TEST); - Pair pair = assertTableMetaEquals(KEYSPACE_PER_TEST, KEYSPACE_PER_TEST, souceTable, targetTable); + TableMetadata source = getTableMetadata(KEYSPACE_PER_TEST, currentTable()); + assertNotNull(source); + String targetTable = createTableLike("CREATE TABLE %s LIKE %s", souceTable, KEYSPACE_PER_TEST, KEYSPACE_PER_TEST); + TableMetadata target = getTableMetadata(KEYSPACE_PER_TEST, currentTable()); + assertNotNull(target); + assertTrue(equalsWithoutTableNameAndDropCns(source, target, true, true, true)); + assertNotEquals(source.id, target.id); + assertNotEquals(source.name, target.name); String sourceTableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + souceTable + " (\n" + " pk1 text,\n" + @@ -850,7 +856,7 @@ public void testDescribeCreateLikeTable() throws Throwable " v1 int,\n" + " v2 int,\n" + " PRIMARY KEY ((pk1, pk2), ck1, ck2)\n" + - ") WITH ID = " + pair.left.id + "\n" + + ") WITH ID = " + source.id + "\n" + " AND CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + " AND " + tableParametersCql(); String targetTableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + targetTable + " (\n" + @@ -862,7 +868,7 @@ public void testDescribeCreateLikeTable() throws Throwable " v1 int,\n" + " v2 int,\n" + " PRIMARY KEY ((pk1, pk2), ck1, ck2)\n" + - ") WITH ID = " + pair.right.id + "\n" + + ") WITH ID = " + target.id + "\n" + " AND CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + " AND " + tableParametersCql(); diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java index 4ca7c490b47b..cf5d518bb526 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java @@ -38,7 +38,7 @@ public class CollectionsTest extends CQLTester { @Test - public void testMapBulkRemoval() + public void testMapBulkRemoval() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, m map)"); @@ -87,7 +87,7 @@ public void testInvalidCollectionsMix() throws Throwable } @Test - public void testSets() + public void testSets() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, s set)"); @@ -383,7 +383,7 @@ public void testListWithUnsetValues() throws Throwable } @Test - public void testSetWithUnsetValues() + public void testSetWithUnsetValues() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, s set)"); @@ -476,7 +476,7 @@ public void testMap() throws Throwable * Migrated from cql_tests.py:TestCQL.list_test() */ @Test - public void testList() + public void testList() throws Throwable { createTable("CREATE TABLE %s (fn text, ln text, tags list, PRIMARY KEY (fn, ln))"); @@ -513,7 +513,7 @@ public void testList() * Migrated from cql_tests.py:TestCQL.multi_collection_test() */ @Test - public void testMultiCollections() + public void testMultiCollections() throws Throwable { UUID id = UUID.fromString("b017f48f-ae67-11e1-9096-005056c00008"); @@ -537,7 +537,7 @@ public void testMultiCollections() * Migrated from cql_tests.py:TestCQL.collection_and_regular_test() */ @Test - public void testCollectionAndRegularColumns() + public void testCollectionAndRegularColumns() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, l list, c int)"); @@ -551,7 +551,7 @@ public void testCollectionAndRegularColumns() * Migrated from cql_tests.py:TestCQL.multi_list_set_test() */ @Test - public void testMultipleLists() + public void testMultipleLists() throws Throwable { createTable(" CREATE TABLE %s (k int PRIMARY KEY, l1 list, l2 list)"); @@ -567,7 +567,7 @@ public void testMultipleLists() * migrated from cql_tests.py:TestCQL.alter_with_collections_test() */ @Test - public void testAlterCollections() + public void testAlterCollections() throws Throwable { createTable("CREATE TABLE %s (key int PRIMARY KEY, aset set)"); @@ -628,7 +628,7 @@ public void testInRestrictionWithCollection() throws Throwable * migrated from cql_tests.py:TestCQL.nonpure_function_collection_test() */ @Test - public void testNonPureFunctionCollection() + public void testNonPureFunctionCollection() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, v list)"); // we just want to make sure this doesn't throw @@ -640,7 +640,7 @@ public void testNonPureFunctionCollection() * migrated from cql_tests.py:TestCQL.collection_flush_test() */ @Test - public void testCollectionFlush() + public void testCollectionFlush() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, s set)"); @@ -669,7 +669,7 @@ public void testDropAndReaddCollection() throws Throwable } @Test - public void testDropAndReaddDroppedCollection() + public void testDropAndReaddDroppedCollection() throws Throwable { createTable("create table %s (k int primary key, v set, x int)"); execute("insert into %s (k, v) VALUES (0, {'fffffffff'})"); @@ -679,7 +679,7 @@ public void testDropAndReaddDroppedCollection() } @Test - public void testMapWithLargePartition() + public void testMapWithLargePartition() throws Throwable { Random r = new Random(); long seed = nanoTime(); @@ -705,7 +705,7 @@ public void testMapWithLargePartition() } @Test - public void testMapWithTwoSStables() + public void testMapWithTwoSStables() throws Throwable { createTable("CREATE TABLE %s (userid text PRIMARY KEY, properties map) with compression = {}"); @@ -726,7 +726,7 @@ public void testMapWithTwoSStables() } @Test - public void testSetWithTwoSStables() + public void testSetWithTwoSStables() throws Throwable { createTable("CREATE TABLE %s (userid text PRIMARY KEY, properties set) with compression = {}"); @@ -747,7 +747,7 @@ public void testSetWithTwoSStables() } @Test - public void testUpdateStaticList() + public void testUpdateStaticList() throws Throwable { createTable("CREATE TABLE %s (k1 text, k2 text, s_list list static, PRIMARY KEY (k1, k2))"); @@ -766,7 +766,7 @@ public void testUpdateStaticList() } @Test - public void testListWithElementsBiggerThan64K() + public void testListWithElementsBiggerThan64K() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, l list)"); @@ -816,7 +816,7 @@ public void testListWithElementsBiggerThan64K() } @Test - public void testMapsWithElementsBiggerThan64K() + public void testMapsWithElementsBiggerThan64K() throws Throwable { byte[] bytes = new byte[FBUtilities.MAX_UNSIGNED_SHORT + 10]; Arrays.fill(bytes, (byte) 1); @@ -884,7 +884,7 @@ public void testMapsWithElementsBiggerThan64K() } @Test - public void testSetsWithElementsBiggerThan64K() + public void testSetsWithElementsBiggerThan64K() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY, s set)"); @@ -1030,7 +1030,7 @@ public void testMultipleOperationOnListWithinTheSameQuery() throws Throwable } @Test - public void testMultipleOperationOnMapWithinTheSameQuery() + public void testMultipleOperationOnMapWithinTheSameQuery() throws Throwable { createTable("CREATE TABLE %s (pk int PRIMARY KEY, m map)"); execute("INSERT INTO %s (pk, m) VALUES (1, {0 : 1, 1 : 2, 2 : 3, 3 : 4})"); @@ -1070,7 +1070,7 @@ public void testMultipleOperationOnMapWithinTheSameQuery() } @Test - public void testMultipleOperationOnSetWithinTheSameQuery() + public void testMultipleOperationOnSetWithinTheSameQuery() throws Throwable { createTable("CREATE TABLE %s (pk int PRIMARY KEY, s set)"); execute("INSERT INTO %s (pk, s) VALUES (1, {0, 1, 2})"); @@ -1491,7 +1491,7 @@ public void testMapOperationOnPartKey() throws Throwable } @Test - public void testMapOperationOnClustKey() + public void testMapOperationOnClustKey() throws Throwable { createTable("CREATE TABLE %s (k int, c frozen>, l text, o int, PRIMARY KEY (k, c))"); @@ -1545,7 +1545,7 @@ public void testMapOperationOnClustKey() } @Test - public void testSetOperation() + public void testSetOperation() throws Throwable { createTable("CREATE TABLE %s (k int, c int, l text, " + "s set, " + @@ -1659,7 +1659,7 @@ public void testElementAccessOnList() throws Throwable } @Test - public void testCollectionOperationResultSetMetadata() + public void testCollectionOperationResultSetMetadata() throws Throwable { createTable("CREATE TABLE %s (k int PRIMARY KEY," + "m map," + diff --git a/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java b/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java index a7aeea49cb77..864a085e93f7 100644 --- a/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java +++ b/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java @@ -21,10 +21,13 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.utils.BloomCalculations; import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + public class CreateTableValidationTest extends CQLTester { @Test @@ -88,4 +91,10 @@ public void testCreateTableWithMissingClusteringColumn() "Missing CLUSTERING ORDER for column ck1"); } + protected void expectedFailure(final Class exceptionType, String statement, String errorMsg) + { + assertThatExceptionOfType(exceptionType) + .isThrownBy(() -> createTableMayThrow(statement)).withMessageContaining(errorMsg); + } + } diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java index 732fd997a6d1..d2e0055f429f 100644 --- a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java @@ -30,10 +30,8 @@ public class CreateLikeCqlParseTest extends CQLTester { private static final String[] unSupportCqls = new String[] { - "create table if not exist ta like tb", - "create table ta (a int primary key, b int) like tb", - "create table ta like tb with comment = 'asss'", - "create table ta like tb with compaction = {'class':'UnifiedCompactionStrategy'}" + "CREATE TABLE ta (a int primary key, b int) LIKE tb", + "CREATE TABLE ta (a int primary key, b int MASKED WITH DEFAULT) LIKE tb", }; @Test diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java index f368445494f3..77093a45763a 100644 --- a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java @@ -21,8 +21,10 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Date; import java.util.List; +import java.util.Map; import java.util.UUID; import org.junit.Before; @@ -34,12 +36,23 @@ import org.apache.cassandra.cql3.Duration; import org.apache.cassandra.cql3.validation.operations.CreateTest; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; +import org.apache.cassandra.exceptions.AlreadyExistsException; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.CompressionParams; import org.apache.cassandra.schema.Indexes; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableParams; +import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; import org.apache.cassandra.utils.TimeUUID; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) public class CreateLikeTest extends CQLTester @@ -90,7 +103,7 @@ public void before() public void testTableShemaCopy() { String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c text);"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c) VALUES (?, ?, ?)", 1, duration1, "1"); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c) VALUES (?, ?, ?)", 2, duration2, "2"); @@ -100,7 +113,7 @@ public void testTableShemaCopy() row(2, duration2, "2")); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY);"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (1)"); execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (2)"); @@ -110,7 +123,7 @@ public void testTableShemaCopy() row(2)); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a frozen> PRIMARY KEY);"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (?)", map("k", "v")); execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (?)", map("nk", "nv")); @@ -120,7 +133,7 @@ public void testTableShemaCopy() row(map("nk", "nv"))); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b set>>, c map, d smallint, e duration, f tinyint);"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4); @@ -132,7 +145,7 @@ public void testTableShemaCopy() row(2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short)3, duration2, (byte)5)); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b double, c tinyint, d float, e list, f map, g duration, PRIMARY KEY((a, b, c), d));"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", 1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1); @@ -155,7 +168,7 @@ public void testTableShemaCopy() "j map>>, " + "PRIMARY KEY((a, b), c, d)) " + "WITH CLUSTERING ORDER BY (c DESC, d ASC);"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", 1, "b", 100L, decimal1, set("1", "2"), uuid1, vector1, list(1.1F, 2.2F), timeUuid1, map("k", set(1, 2))); @@ -167,23 +180,35 @@ public void testTableShemaCopy() row(2, "nb", 200L, decimal2, set("3", "4"), uuid2, vector2, list(3.3F, 4.4F), timeUuid2, map("nk", set(3, 4)))); } + @Test + public void testIfNotExists() throws Throwable + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); + String targetTb = createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetTb, targetKs); + assertInvalidThrowMessage("Cannot add already existing table \"" + targetTb + "\" to keyspace \"" + targetKs + "\"", AlreadyExistsException.class, + "CREATE TABLE " + targetKs + "." + targetTb + " LIKE " + sourceKs + "." + sourceTb); + } + @Test public void testCopyAfterAlterTable() { String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP d"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD e uuid"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD f float"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, e, f) VALUES (?, ?, ?, ?, ?)", 1, "1", duration1, uuid1, f1); @@ -194,15 +219,15 @@ public void testCopyAfterAlterTable() row(2, "2", duration2, uuid2, f2)); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP f USING TIMESTAMP 20000"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " RENAME b TO bb "); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16} "); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, bb, c, e) VALUES (?, ?, ?, ?)", 1, "1", duration1, uuid1); @@ -258,29 +283,29 @@ public void testTableOptionsCopy() " AND compaction = {'class':'UnifiedCompactionStrategy'} " + " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 }" + " AND gc_grace_seconds = 100" + - " AND incremental_backups = false" + + " AND incremental_backups = false" + " AND max_index_interval = 1024" + - " AND min_index_interval = 64" + + " AND min_index_interval = 64" + " AND speculative_retry = '95p'" + " AND read_repair = 'NONE'" + " AND memtable_flush_period_in_ms = 360000" + " AND memtable = 'default';" ); - String tbLikeCompressionDefault1 = createTableLike("create table %s like %s", tbCompressionDefault1, sourceKs, targetKs); - String tbLikeCompressionDefault2 = createTableLike("create table %s like %s", tbCompressionDefault2, sourceKs, targetKs); - String tbLikeCompressionSp1 = createTableLike("create table %s like %s", tbCompressionSnappy1, sourceKs, targetKs); - String tbLikeCompressionSp2 = createTableLike("create table %s like %s", tbCompressionSnappy2, sourceKs, targetKs); - String tbLikeCompressionSp3 = createTableLike("create table %s like %s", tbCompressionSnappy3, sourceKs, targetKs); - String tbLikeCompressionSp4 = createTableLike("create table %s like %s", tbCompressionSnappy4, sourceKs, targetKs); - String tbLikeCompressionSp5 = createTableLike("create table %s like %s", tbCompressionSnappy5, sourceKs, targetKs); - String tbLikeMemtableSkipList = createTableLike("create table %s like %s", tableMemtableSkipList, sourceKs, targetKs); - String tbLikeMemtableTrie = createTableLike("create table %s like %s", tableMemtableTrie, sourceKs, targetKs); - String tbLikeMemtableDefault = createTableLike("create table %s like %s", tableMemtableDefault, sourceKs, targetKs); - String tbLikeCompactionStcs = createTableLike("create table %s like %s", tableCompactionStcs, sourceKs, targetKs); - String tbLikeCompactionLcs = createTableLike("create table %s like %s", tableCompactionLcs, sourceKs, targetKs); - String tbLikeCompactionTwcs = createTableLike("create table %s like %s", tableCompactionTwcs, sourceKs, targetKs); - String tbLikeCompactionUcs = createTableLike("create table %s like %s", tableCompactionUcs, sourceKs, targetKs); - String tbLikeCompactionOthers= createTableLike("create table %s like %s", tableOtherOptions, sourceKs, targetKs); + String tbLikeCompressionDefault1 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault1, sourceKs, targetKs); + String tbLikeCompressionDefault2 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault2, sourceKs, targetKs); + String tbLikeCompressionSp1 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy1, sourceKs, targetKs); + String tbLikeCompressionSp2 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy2, sourceKs, targetKs); + String tbLikeCompressionSp3 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy3, sourceKs, targetKs); + String tbLikeCompressionSp4 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy4, sourceKs, targetKs); + String tbLikeCompressionSp5 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy5, sourceKs, targetKs); + String tbLikeMemtableSkipList = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableSkipList, sourceKs, targetKs); + String tbLikeMemtableTrie = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableTrie, sourceKs, targetKs); + String tbLikeMemtableDefault = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableDefault, sourceKs, targetKs); + String tbLikeCompactionStcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionStcs, sourceKs, targetKs); + String tbLikeCompactionLcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionLcs, sourceKs, targetKs); + String tbLikeCompactionTwcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionTwcs, sourceKs, targetKs); + String tbLikeCompactionUcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionUcs, sourceKs, targetKs); + String tbLikeCompactionOthers= createTableLike("CREATE TABLE %s LIKE %s", tableOtherOptions, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tbLikeCompressionDefault1); assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault2, tbLikeCompressionDefault2); @@ -297,6 +322,57 @@ public void testTableOptionsCopy() assertTableMetaEquals(sourceKs, targetKs, tableCompactionTwcs, tbLikeCompactionTwcs); assertTableMetaEquals(sourceKs, targetKs, tableCompactionUcs, tbLikeCompactionUcs); assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tbLikeCompactionOthers); + + // table copy with params setting + String tableCopyAndSetCompression = createTableLike("CREATE TABLE %s LIKE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 };", + tbCompressionSnappy1, sourceKs, targetKs); + String tableCopyAndSetLCSCompaction = createTableLike("CREATE TABLE %s LIKE %s WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16};", + tableCompactionLcs, sourceKs, targetKs); + String tableCopyAndSetAllParams = createTableLike("CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH" + + " bloom_filter_fp_chance = 0.75 " + + " AND caching = {'keys': 'NONE', 'rows_per_partition': '10'}" + + " AND cdc = true " + + " AND comment = 'test for create like and set params'" + + " AND crc_check_chance = 0.8" + + " AND default_time_to_live = 100" + + " AND compaction = {'class':'SizeTieredCompactionStrategy'} " + + " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 }" + + " AND gc_grace_seconds = 1000" + + " AND incremental_backups = true" + + " AND max_index_interval = 128" + + " AND min_index_interval = 16" + + " AND speculative_retry = '96p'" + + " AND read_repair = 'NONE'" + + " AND memtable_flush_period_in_ms = 3600;", + tableOtherOptions, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tableCopyAndSetCompression, false, false, false); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionLcs, tableCopyAndSetLCSCompaction, false, false, false); + assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tableCopyAndSetAllParams, false, false, false); + TableParams paramsSetCompression = getTableMetadata(targetKs, tableCopyAndSetCompression).params; + TableParams paramsSetLCSCompaction = getTableMetadata(targetKs, tableCopyAndSetLCSCompaction).params; + TableParams paramsSetAllParams = getTableMetadata(targetKs, tableCopyAndSetAllParams).params; + + assertEquals(paramsSetCompression, TableParams.builder().compression(CompressionParams.snappy(64 * 1024, 0.0)).build()); + assertEquals(paramsSetLCSCompaction, TableParams.builder().compaction(CompactionParams.create(LeveledCompactionStrategy.class, + Map.of("sstable_size_in_mb", "10", + "fanout_size", "16"))) + .build()); + assertEquals(paramsSetAllParams, TableParams.builder().bloomFilterFpChance(0.75) + .caching(new CachingParams(false, 10)) + .cdc(true) + .comment("test for create like and set params") + .crcCheckChance(0.8) + .defaultTimeToLive(100) + .compaction(CompactionParams.stcs(Collections.emptyMap())) + .compression(CompressionParams.snappy(64 * 1024, 0.0)) + .gcGraceSeconds(1000) + .incrementalBackups(true) + .maxIndexInterval(128) + .minIndexInterval(16) + .speculativeRetry(SpeculativeRetryPolicy.fromString("96PERCENTILE")) + .readRepair(ReadRepairStrategy.NONE) + .memtableFlushPeriodInMs(3600) + .build()); } @Test @@ -304,7 +380,7 @@ public void testStaticColumnCopy() { // create with static column String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b int , c int static, d int, e list, PRIMARY KEY(a, b));", "tb1"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e) VALUES (0, 1, 2, 3, ?)", list("1", "2", "3", "4")); assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), row(0, 1, 2, 3, list("1", "2", "3", "4"))); @@ -312,7 +388,7 @@ public void testStaticColumnCopy() //add static column sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))"); alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD d int static"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); } @@ -322,38 +398,38 @@ public void testColumnMaskTableCopy() DatabaseDescriptor.setDynamicDataMaskingEnabled(true); // Masked partition key String sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int MASKED WITH mask_default() PRIMARY KEY, r int)"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked partition key component sourceTb = createTable(sourceKs, "CREATE TABLE %s (k1 int, k2 text MASKED WITH DEFAULT, r int, PRIMARY KEY(k1, k2))"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked clustering key sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int MASKED WITH mask_default(), r int, PRIMARY KEY (k, c))"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked clustering key with reverse order sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c text MASKED WITH mask_default(), r int, PRIMARY KEY (k, c)) " + "WITH CLUSTERING ORDER BY (c DESC)"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked clustering key component sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c1 int, c2 text MASKED WITH DEFAULT, r int, PRIMARY KEY (k, c1, c2))"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked regular column sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, r1 text MASKED WITH DEFAULT, r2 int)"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Masked static column sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int, r int, s int STATIC MASKED WITH DEFAULT, PRIMARY KEY (k, c))"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); // Multiple masked columns @@ -363,7 +439,7 @@ public void testColumnMaskTableCopy() "r1 int, r2 int MASKED WITH DEFAULT, " + "s1 int static, s2 int static MASKED WITH DEFAULT, " + "PRIMARY KEY((k1, k2), c1, c2))"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, " + @@ -373,12 +449,12 @@ public void testColumnMaskTableCopy() "fs frozen> MASKED WITH DEFAULT, " + "fl frozen> MASKED WITH DEFAULT, " + "fm frozen> MASKED WITH DEFAULT)"); - targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); } @Test - public void testUDTTableCopy() + public void testUDTTableCopy() throws Throwable { //normal udt String udt = createType(sourceKs, "CREATE TYPE %s (a int, b uuid, c text)"); @@ -393,18 +469,18 @@ public void testUDTTableCopy() if (differentKs) { - expectedFailure(InvalidRequestException.class, "create table " + targetKs + ".tbudt like " + sourceKs + "." + sourceTbUdt , - "Cannot use CTREATE TABLE LIKE across different keyspace when source table have UDT"); - expectedFailure(InvalidRequestException.class, "create table " + targetKs + ".tbdtset like " + sourceKs + "." + sourceTbUdt , - "Cannot use CTREATE TABLE LIKE across different keyspace when source table have UDT"); - expectedFailure(InvalidRequestException.class, "create table " + targetKs + ".tbudtfrozen like " + sourceKs + "." + sourceTbUdt , - "Cannot use CTREATE TABLE LIKE across different keyspace when source table have UDT"); + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", + InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudt LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", + InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbdtset LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", InvalidRequestException.class, + "create table " + targetKs + ".tbudtfrozen like " + sourceKs + "." + sourceTbUdt); } else { - String targetTbUdt = createTableLike("create table %s like %s", sourceTbUdt, sourceKs, "tbudt", targetKs); - String targetTbUdtSet = createTableLike("create table %s like %s", sourceTbUdtSet, sourceKs, "tbdtset", targetKs); - String targetTbUdtFrozen = createTableLike("create table %s like %s", sourceTbUdtFrozen, sourceKs, "tbudtfrozen", targetKs); + String targetTbUdt = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdt, sourceKs, "tbudt", targetKs); + String targetTbUdtSet = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtSet, sourceKs, "tbdtset", targetKs); + String targetTbUdtFrozen = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtFrozen, sourceKs, "tbudtfrozen", targetKs); assertTableMetaEquals(sourceKs, targetKs, sourceTbUdt, targetTbUdt); assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtSet, targetTbUdtSet); assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtFrozen, targetTbUdtFrozen); @@ -416,7 +492,7 @@ public void testIndexOpreationOnCopiedTable() { // copied table can do index creation String sourceTb = createTable(sourceKs, "CREATE TABLE %s (id text PRIMARY KEY, val text, num int);"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); String saiIndex = createIndex(targetKs, "CREATE INDEX ON %s(val) USING 'sai'"); execute("INSERT INTO " + targetKs + "." + targetTb + " (id, val, num) VALUES ('1', 'value', 1)"); assertEquals(1, execute("SELECT id FROM " + targetKs + "." + targetTb + " WHERE val = 'value'").size()); @@ -432,25 +508,38 @@ public void testTriggerOperationOnCopiedTable() { String triggerName = "trigger_1"; String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))"); - String targetTb = createTableLike("create table %s like %s", sourceTb, sourceKs, targetKs); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); execute("CREATE TRIGGER " + triggerName + " ON " + targetKs + "." + targetTb + " USING '" + CreateTest.TestTrigger.class.getName() + "'"); assertNotNull(getTableMetadata(targetKs, targetTb).triggers.get(triggerName)); } @Test - public void testUnSupportedSchema() + public void testUnSupportedSchema() throws Throwable { createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text)", "tb"); String index = createIndex( "CREATE INDEX ON " + sourceKs + ".tb (c)"); - expectedFailure(InvalidRequestException.class, - "CREATE TABLE " + sourceKs + ".newtb like " + targetKs + "." + index + ";", - "Souce Table '" + targetKs + "'.'" + index + "' doesn't exist"); - - expectedFailure(InvalidRequestException.class, - "CREATE TABLE system.local_clone like system.local ;", - "System keyspace 'system' is not user-modifiable"); - expectedFailure(InvalidRequestException.class, - "CREATE TABLE system_views.newtb like system_views.snapshots ;", - "System keyspace 'system_views' is not user-modifiable"); + assertInvalidThrowMessage("Souce Table '" + targetKs + "'.'" + index + "' doesn't exist", InvalidRequestException.class, + "CREATE TABLE " + sourceKs + ".newtb LIKE " + targetKs + "." + index + ";"); + + assertInvalidThrowMessage("System keyspace 'system' is not user-modifiable", InvalidRequestException.class, + "CREATE TABLE system.local_clone LIKE system.local ;"); + assertInvalidThrowMessage("System keyspace 'system_views' is not user-modifiable", InvalidRequestException.class, + "CREATE TABLE system_views.newtb LIKE system_views.snapshots ;"); + } + + private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb) + { + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + } + + private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + { + TableMetadata left = getTableMetadata(sourceKs, sourceTb); + TableMetadata right = getTableMetadata(targetKs, targetTb); + assertNotNull(left); + assertNotNull(right); + assertTrue(equalsWithoutTableNameAndDropCns(left, right, compareParams, compareIndexes, compareTrigger)); + assertNotEquals(left.id, right.id); + assertNotEquals(left.name, right.name); } } diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java index 7e5f67194b94..3ce09274793d 100644 --- a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java @@ -51,14 +51,14 @@ public void testCreateLikeWithSession() // use ks1 executeNet("use " + ks1); - executeNet("CREATE TABLE tb3 like " + tb1); - executeNet("CREATE TABLE " + ks1 + ".tb4 like " + tb1); + executeNet("CREATE TABLE tb3 LIKE " + tb1); + executeNet("CREATE TABLE " + ks1 + ".tb4 LIKE " + tb1); executeNet("CREATE TABLE tb5 like " + ks1 + "." + tb1); - executeNet("CREATE TABLE " + ks2 + ".tb6 like " + tb1); + executeNet("CREATE TABLE " + ks2 + ".tb6 LIKE " + tb1); - assertThatExceptionOfType(com.datastax.driver.core.exceptions.InvalidQueryException.class).isThrownBy(() -> executeNet("CREATE TABLE tb7 like " + ks2 + "." + tb1)) + assertThatExceptionOfType(com.datastax.driver.core.exceptions.InvalidQueryException.class).isThrownBy(() -> executeNet("CREATE TABLE tb7 LIKE " + ks2 + "." + tb1)) .withMessage("Souce Table 'ks2'.'tb1' doesn't exist"); assertNotNull(getTableMetadata(ks1, tb1));