diff --git a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraMetadata.java b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraMetadata.java index 90f6a0b3442d..f303a6d11a93 100644 --- a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraMetadata.java +++ b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraMetadata.java @@ -381,7 +381,7 @@ public ColumnHandle getDeleteRowIdColumnHandle(ConnectorSession session, Connect @Override public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTableHandle tableHandle) { - throw new TrinoException(NOT_SUPPORTED, "This connector only supports delete with primary key or partition key"); + throw new TrinoException(NOT_SUPPORTED, "Delete without primary key or partition key is not supported"); } @Override @@ -396,7 +396,7 @@ public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle CassandraTableHandle handle = (CassandraTableHandle) deleteHandle; Optional> partitions = handle.getPartitions(); if (partitions.isEmpty()) { - throw new TrinoException(NOT_SUPPORTED, "Deleting without partition key is unsupported"); + throw new TrinoException(NOT_SUPPORTED, "Deleting without partition key is not supported"); } if (partitions.get().isEmpty()) { // there are no records of a given partition key diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraDistributedQueries.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraDistributedQueries.java index 0d60225f3e38..98813fd09c13 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraDistributedQueries.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraDistributedQueries.java @@ -86,7 +86,7 @@ public void testDropColumn() public void testDelete() { assertThatThrownBy(super::testDelete) - .hasStackTraceContaining("This connector only supports delete with primary key or partition key"); + .hasStackTraceContaining("Delete without primary key or partition key is not supported"); } @Override diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraIntegrationSmokeTest.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraIntegrationSmokeTest.java index 4dda60fbeeb5..d0e9e87a85c9 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraIntegrationSmokeTest.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraIntegrationSmokeTest.java @@ -73,7 +73,7 @@ import static org.testng.Assert.assertEquals; public class TestCassandraIntegrationSmokeTest - // TODO extend BaseConnectorTest + // TODO extend BaseConnectorTest and merge with TestCassandraDistributedQueries extends AbstractTestIntegrationSmokeTest { private static final String KEYSPACE = "smoke_test"; diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraDistributedQueriesLatest.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraLatestConnectorSmokeTest.java similarity index 62% rename from plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraDistributedQueriesLatest.java rename to plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraLatestConnectorSmokeTest.java index bf0af82064df..9cde7360b28a 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraDistributedQueriesLatest.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraLatestConnectorSmokeTest.java @@ -14,12 +14,14 @@ package io.trino.plugin.cassandra; import com.google.common.collect.ImmutableMap; +import io.trino.testing.BaseConnectorSmokeTest; import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; import static io.trino.plugin.cassandra.CassandraQueryRunner.createCassandraQueryRunner; -public class TestCassandraDistributedQueriesLatest - extends BaseCassandraDistributedQueries +public class TestCassandraLatestConnectorSmokeTest + extends BaseConnectorSmokeTest { @Override protected QueryRunner createQueryRunner() @@ -28,4 +30,22 @@ protected QueryRunner createQueryRunner() CassandraServer server = closeAfterClass(new CassandraServer("3.11.9")); return createCassandraQueryRunner(server, ImmutableMap.of(), REQUIRED_TPCH_TABLES); } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + switch (connectorBehavior) { + case SUPPORTS_CREATE_SCHEMA: + return false; + + case SUPPORTS_DELETE: + return true; + + case SUPPORTS_ROW_LEVEL_DELETE: + return false; + + default: + return super.hasBehavior(connectorBehavior); + } + } } diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaLatestConnectorSmokeTest.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaLatestConnectorSmokeTest.java new file mode 100644 index 000000000000..a4a442361f26 --- /dev/null +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaLatestConnectorSmokeTest.java @@ -0,0 +1,60 @@ +/* + * Licensed 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 io.trino.plugin.kafka; + +import io.trino.testing.BaseConnectorSmokeTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import io.trino.testing.kafka.TestingKafka; +import org.testng.SkipException; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestKafkaLatestConnectorSmokeTest + extends BaseConnectorSmokeTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + TestingKafka testingKafka = closeAfterClass(TestingKafka.create("6.0.1")); + return KafkaQueryRunner.builder(testingKafka) + .setTables(REQUIRED_TPCH_TABLES) + .build(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + switch (connectorBehavior) { + case SUPPORTS_CREATE_SCHEMA: + return false; + + case SUPPORTS_CREATE_TABLE: + return false; + + default: + return super.hasBehavior(connectorBehavior); + } + } + + @Override + public void testInsert() + { + assertThatThrownBy(super::testInsert) + .hasMessage("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way"); + // TODO implement the test for Kafka + throw new SkipException("TODO"); + } +} diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaDistributedLatest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java similarity index 53% rename from plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaDistributedLatest.java rename to plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java index 7f318cdca72f..01bfba9360c3 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaDistributedLatest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java @@ -11,22 +11,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.kafka; +package io.trino.plugin.mongodb; -import io.trino.testing.AbstractTestQueries; -import io.trino.testing.QueryRunner; -import io.trino.testing.kafka.TestingKafka; +import io.trino.testing.BaseConnectorSmokeTest; +import io.trino.testing.TestingConnectorBehavior; -public class TestKafkaDistributedLatest - extends AbstractTestQueries +public abstract class BaseMongoConnectorSmokeTest + extends BaseConnectorSmokeTest { @Override - protected QueryRunner createQueryRunner() - throws Exception + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) { - TestingKafka testingKafka = closeAfterClass(TestingKafka.create("6.0.1")); - return KafkaQueryRunner.builder(testingKafka) - .setTables(REQUIRED_TPCH_TABLES) - .build(); + switch (connectorBehavior) { + case SUPPORTS_CREATE_SCHEMA: + return false; + default: + return super.hasBehavior(connectorBehavior); + } } } diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3DistributedQueriesLatest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3LatestConnectorSmokeTest.java similarity index 91% rename from plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3DistributedQueriesLatest.java rename to plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3LatestConnectorSmokeTest.java index c6a1d7afdeed..819b740936a6 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3DistributedQueriesLatest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo3LatestConnectorSmokeTest.java @@ -18,8 +18,8 @@ import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoQueryRunner; -public class TestMongo3DistributedQueriesLatest - extends BaseMongoDistributedQueries +public class TestMongo3LatestConnectorSmokeTest + extends BaseMongoConnectorSmokeTest { @Override protected QueryRunner createQueryRunner() diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4DistributedQueriesLatest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4LatestConnectorSmokeTest.java similarity index 91% rename from plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4DistributedQueriesLatest.java rename to plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4LatestConnectorSmokeTest.java index c830a647146e..815e87b7d175 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4DistributedQueriesLatest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongo4LatestConnectorSmokeTest.java @@ -18,8 +18,8 @@ import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoQueryRunner; -public class TestMongo4DistributedQueriesLatest - extends BaseMongoDistributedQueries +public class TestMongo4LatestConnectorSmokeTest + extends BaseMongoConnectorSmokeTest { @Override protected QueryRunner createQueryRunner() diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java index 70764cfc1846..a70434b18c60 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java @@ -23,6 +23,7 @@ import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_SCHEMA; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ROW_LEVEL_DELETE; import static io.trino.testing.sql.TestTable.randomTableSuffix; @@ -105,7 +106,7 @@ public void testJoin() public void testCreateTable() { if (!hasBehavior(SUPPORTS_CREATE_TABLE)) { - assertQueryFails("CREATE TABLE xxxx (a bigint, b double)", "This connector does not support create"); + assertQueryFails("CREATE TABLE xxxx (a bigint, b double)", "This connector does not support creating tables"); return; } @@ -120,7 +121,7 @@ public void testCreateTable() public void testCreateTableAsSelect() { if (!hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)) { - assertQueryFails("CREATE TABLE xxxx (a bigint, b double) AS VALUES (42, -38.5)", "This connector does not support create"); + assertQueryFails("CREATE TABLE xxxx AS SELECT BIGINT '42' a, DOUBLE '-38.5' b", "This connector does not support creating tables with data"); return; } @@ -139,6 +140,10 @@ public void testInsert() return; } + if (!hasBehavior(SUPPORTS_CREATE_TABLE)) { + throw new AssertionError("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way"); + } + String tableName = "test_create_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double)"); assertUpdate("INSERT INTO " + tableName + " (a, b) VALUES (42, -38.5)", 1); @@ -150,11 +155,16 @@ public void testInsert() @Test public void testDelete() { - if (!hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)) { + if (!hasBehavior(SUPPORTS_DELETE)) { assertQueryFails("DELETE FROM region", "This connector does not support deletes"); return; } + if (!hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)) { + assertQueryFails("DELETE FROM region WHERE regionkey = 2", ".*[Dd]elet(e|ing).*(not |un)supported.*"); + return; + } + String tableName = "test_delete_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM region", 5); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java b/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java index 3079b09fe972..9e3552109b4b 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java @@ -36,6 +36,8 @@ public enum TestingConnectorBehavior SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN(SUPPORTS_JOIN_PUSHDOWN), SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM(SUPPORTS_JOIN_PUSHDOWN), + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, SUPPORTS_CREATE_TABLE_WITH_DATA(SUPPORTS_CREATE_TABLE), @@ -44,8 +46,6 @@ public enum TestingConnectorBehavior SUPPORTS_DELETE(false), SUPPORTS_ROW_LEVEL_DELETE(SUPPORTS_DELETE), - SUPPORTS_CREATE_SCHEMA, - /**/; private final Predicate> hasBehaviorByDefault;