diff --git a/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestAccumuloIntegrationSmokeTest.java b/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestAccumuloIntegrationSmokeTest.java index d5c5bb2c8cf3..07d43e647d3a 100644 --- a/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestAccumuloIntegrationSmokeTest.java +++ b/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestAccumuloIntegrationSmokeTest.java @@ -26,6 +26,7 @@ import static org.assertj.core.api.Assertions.assertThat; public class TestAccumuloIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { @Override diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcCachingQueries.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcCachingQueries.java new file mode 100644 index 000000000000..c25bbd6e7e1f --- /dev/null +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcCachingQueries.java @@ -0,0 +1,120 @@ +/* + * 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.jdbc; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.AbstractTestDistributedQueries; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.JdbcSqlExecutor; +import io.trino.testing.sql.TestTable; +import io.trino.tpch.TpchTable; +import org.testng.SkipException; + +import java.util.Map; +import java.util.Optional; +import java.util.Properties; + +import static io.trino.plugin.jdbc.H2QueryRunner.createH2QueryRunner; + +public class TestJdbcCachingQueries + // TODO define shorter tests set that exercises various read and write scenarios (a.k.a. "a smoke test") + extends AbstractTestDistributedQueries +{ + private Map properties; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + properties = ImmutableMap.builder() + .putAll(TestingH2JdbcModule.createProperties()) + .put("metadata.cache-ttl", "10m") + .put("metadata.cache-missing", "true") + .put("allow-drop-table", "true") + .build(); + return createH2QueryRunner(TpchTable.getTables(), properties); + } + + @Override + protected boolean supportsDelete() + { + return false; + } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Override + protected boolean supportsArrays() + { + return false; + } + + @Override + protected boolean supportsCommentOnTable() + { + return false; + } + + @Override + protected boolean supportsCommentOnColumn() + { + return false; + } + + @Override + public void testLargeIn(int valuesCount) + { + throw new SkipException("This test should pass with H2, but takes too long (currently over a mninute) and is not that important"); + } + + @Override + protected TestTable createTableWithDefaultColumns() + { + return new TestTable( + getSqlExecutor(), + "tpch.table", + "(col_required BIGINT NOT NULL," + + "col_nullable BIGINT," + + "col_default BIGINT DEFAULT 43," + + "col_nonnull_default BIGINT NOT NULL DEFAULT 42," + + "col_required2 BIGINT NOT NULL)"); + } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeBaseName = dataMappingTestSetup.getTrinoTypeName().replaceAll("\\([^()]*\\)", ""); + switch (typeBaseName) { + case "boolean": + case "decimal": + case "char": + case "varbinary": + case "time": + case "timestamp": + case "timestamp with time zone": + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + return Optional.of(dataMappingTestSetup); + } + + private JdbcSqlExecutor getSqlExecutor() + { + return new JdbcSqlExecutor(properties.get("connection-url"), new Properties()); + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorTest.java new file mode 100644 index 000000000000..c386213b2e44 --- /dev/null +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorTest.java @@ -0,0 +1,117 @@ +/* + * 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.jdbc; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.JdbcSqlExecutor; +import io.trino.testing.sql.TestTable; +import io.trino.tpch.TpchTable; +import org.testng.SkipException; + +import java.util.Map; +import java.util.Optional; +import java.util.Properties; + +import static io.trino.plugin.jdbc.H2QueryRunner.createH2QueryRunner; + +public class TestJdbcConnectorTest + extends BaseConnectorTest +{ + private Map properties; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + properties = ImmutableMap.builder() + .putAll(TestingH2JdbcModule.createProperties()) + .put("allow-drop-table", "true") + .build(); + return createH2QueryRunner(TpchTable.getTables(), properties); + } + + @Override + protected boolean supportsDelete() + { + return false; + } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Override + protected boolean supportsArrays() + { + return false; + } + + @Override + protected boolean supportsCommentOnTable() + { + return false; + } + + @Override + protected boolean supportsCommentOnColumn() + { + return false; + } + + @Override + public void testLargeIn(int valuesCount) + { + throw new SkipException("This test should pass with H2, but takes too long (currently over a mninute) and is not that important"); + } + + @Override + protected TestTable createTableWithDefaultColumns() + { + return new TestTable( + getSqlExecutor(), + "tpch.table", + "(col_required BIGINT NOT NULL," + + "col_nullable BIGINT," + + "col_default BIGINT DEFAULT 43," + + "col_nonnull_default BIGINT NOT NULL DEFAULT 42," + + "col_required2 BIGINT NOT NULL)"); + } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeBaseName = dataMappingTestSetup.getTrinoTypeName().replaceAll("\\([^()]*\\)", ""); + switch (typeBaseName) { + case "boolean": + case "decimal": + case "char": + case "varbinary": + case "time": + case "timestamp": + case "timestamp with time zone": + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + return Optional.of(dataMappingTestSetup); + } + + private JdbcSqlExecutor getSqlExecutor() + { + return new JdbcSqlExecutor(properties.get("connection-url"), new Properties()); + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDistributedQueries.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDistributedQueries.java deleted file mode 100644 index 2840c97cc0b2..000000000000 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDistributedQueries.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.jdbc; - -import io.trino.testing.AbstractTestQueries; -import io.trino.testing.QueryRunner; -import io.trino.tpch.TpchTable; - -import static io.trino.plugin.jdbc.H2QueryRunner.createH2QueryRunner; - -public class TestJdbcDistributedQueries - extends AbstractTestQueries -{ - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - return createH2QueryRunner(TpchTable.getTables()); - } - - @Override - public void testLargeIn(int valuesCount) - { - } -} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcIntegrationSmokeTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcIntegrationSmokeTest.java index 7a806363b864..e56ba622c985 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcIntegrationSmokeTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcIntegrationSmokeTest.java @@ -32,6 +32,7 @@ import static java.lang.String.format; public class TestJdbcIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private final Map properties = TestingH2JdbcModule.createProperties(); diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryIntegrationSmokeTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryIntegrationSmokeTest.java index a4ed5bb23f20..c62f51646b6d 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryIntegrationSmokeTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryIntegrationSmokeTest.java @@ -29,6 +29,7 @@ @Test public class TestBigQueryIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private BigQuerySqlExecutor bigQuerySqlExecutor; 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 22fe5e9b6486..4dda60fbeeb5 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,6 +73,7 @@ import static org.testng.Assert.assertEquals; public class TestCassandraIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private static final String KEYSPACE = "smoke_test"; diff --git a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidIntegrationSmokeTest.java b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidIntegrationSmokeTest.java index 0a792f4bc0c0..33a7e6f0d37c 100644 --- a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidIntegrationSmokeTest.java +++ b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidIntegrationSmokeTest.java @@ -29,6 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; public abstract class BaseDruidIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { protected static final String SELECT_FROM_ORDERS = "SELECT " + diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchSmokeTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchSmokeTest.java index 577f99deb660..2751bdbd751b 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchSmokeTest.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchSmokeTest.java @@ -47,6 +47,7 @@ import static org.testng.Assert.assertTrue; public abstract class BaseElasticsearchSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private final String image; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveIntegrationSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveIntegrationSmokeTest.java index bcb5d1c37a4e..25cab8ae4e59 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveIntegrationSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveIntegrationSmokeTest.java @@ -163,6 +163,7 @@ import static org.testng.FileAssert.assertFile; public class TestHiveIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergSmoke.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergSmoke.java index c57ca44c84e0..5c442a70084b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergSmoke.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/AbstractTestIcebergSmoke.java @@ -91,6 +91,7 @@ import static org.testng.Assert.assertTrue; public abstract class AbstractTestIcebergSmoke + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private static final Pattern WITH_CLAUSE_EXTRACTER = Pattern.compile(".*(WITH\\s*\\([^)]*\\))\\s*$", Pattern.DOTALL); diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaIntegrationSmokeTest.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaIntegrationSmokeTest.java index c40b60f73292..bb39bd34f1b6 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaIntegrationSmokeTest.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaIntegrationSmokeTest.java @@ -64,6 +64,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; public class TestKafkaIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private TestingKafka testingKafka; diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/AbstractKuduIntegrationSmokeTest.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/AbstractKuduIntegrationSmokeTest.java index 42e070e9866e..11328f8b763a 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/AbstractKuduIntegrationSmokeTest.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/AbstractKuduIntegrationSmokeTest.java @@ -35,6 +35,7 @@ import static org.testng.Assert.assertTrue; public abstract class AbstractKuduIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private TestingKuduServer kuduServer; diff --git a/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlIntegrationSmokeTest.java b/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlIntegrationSmokeTest.java index b5d873901805..18f9e601a63c 100644 --- a/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlIntegrationSmokeTest.java +++ b/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlIntegrationSmokeTest.java @@ -40,6 +40,7 @@ import static org.testng.Assert.assertTrue; public class TestMemSqlIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { protected TestingMemSqlServer memSqlServer; diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoIntegrationSmokeTest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoIntegrationSmokeTest.java index 24f9c42e0360..a4193c64c948 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoIntegrationSmokeTest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoIntegrationSmokeTest.java @@ -45,6 +45,7 @@ @Test(singleThreaded = true) public class TestMongoIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private MongoServer server; diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlIntegrationSmokeTest.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlIntegrationSmokeTest.java index dfbd0c5807dd..0bf6ed0deec0 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlIntegrationSmokeTest.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlIntegrationSmokeTest.java @@ -39,6 +39,7 @@ import static org.testng.Assert.assertTrue; abstract class BaseMySqlIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { protected TestingMySqlServer mysqlServer; diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java index 629b48348a76..5561081d3a20 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java @@ -34,6 +34,7 @@ import static org.testng.Assert.assertTrue; public abstract class BaseOracleIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { @Test diff --git a/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixIntegrationSmokeTest.java b/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixIntegrationSmokeTest.java index a1029738c631..1e4c106ac48e 100644 --- a/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixIntegrationSmokeTest.java +++ b/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixIntegrationSmokeTest.java @@ -34,6 +34,7 @@ import static org.testng.Assert.fail; public class TestPhoenixIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { private TestingPhoenixServer testingPhoenixServer; diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlIntegrationSmokeTest.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java similarity index 96% rename from plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlIntegrationSmokeTest.java rename to plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java index b49b1921d271..3f66119f70c6 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlIntegrationSmokeTest.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java @@ -20,9 +20,11 @@ import io.trino.sql.planner.plan.FilterNode; import io.trino.sql.planner.plan.MarkDistinctNode; import io.trino.sql.planner.plan.ProjectNode; -import io.trino.testing.AbstractTestIntegrationSmokeTest; +import io.trino.testing.BaseConnectorTest; import io.trino.testing.QueryRunner; +import io.trino.testing.sql.JdbcSqlExecutor; import io.trino.testing.sql.TestTable; +import io.trino.tpch.TpchTable; import org.intellij.lang.annotations.Language; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -31,17 +33,12 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; -import java.util.List; import java.util.Map; import java.util.UUID; import static io.trino.SystemSessionProperties.USE_MARK_DISTINCT; import static io.trino.plugin.postgresql.PostgreSqlQueryRunner.createPostgreSqlQueryRunner; import static io.trino.testing.sql.TestTable.randomTableSuffix; -import static io.trino.tpch.TpchTable.CUSTOMER; -import static io.trino.tpch.TpchTable.NATION; -import static io.trino.tpch.TpchTable.ORDERS; -import static io.trino.tpch.TpchTable.REGION; import static java.lang.String.format; import static java.util.stream.Collectors.joining; import static java.util.stream.IntStream.range; @@ -50,8 +47,8 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -public class TestPostgreSqlIntegrationSmokeTest - extends AbstractTestIntegrationSmokeTest +public class TestPostgreSqlConnectorTest + extends BaseConnectorTest { protected TestingPostgreSqlServer postgreSqlServer; @@ -64,7 +61,7 @@ protected QueryRunner createQueryRunner() postgreSqlServer.close(); postgreSqlServer = null; }); - return createPostgreSqlQueryRunner(postgreSqlServer, Map.of(), Map.of(), List.of(CUSTOMER, NATION, ORDERS, REGION)); + return createPostgreSqlQueryRunner(postgreSqlServer, Map.of(), Map.of(), TpchTable.getTables()); } @BeforeClass @@ -74,6 +71,44 @@ public void setExtensions() execute("CREATE EXTENSION file_fdw"); } + @Override + protected boolean supportsDelete() + { + return false; + } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Override + protected boolean supportsArrays() + { + // Arrays are supported conditionally. Check the defaults. + return new PostgreSqlConfig().getArrayMapping() != PostgreSqlConfig.ArrayMapping.DISABLED; + } + + @Override + protected boolean supportsCommentOnTable() + { + return false; + } + + @Override + protected TestTable createTableWithDefaultColumns() + { + return new TestTable( + new JdbcSqlExecutor(postgreSqlServer.getJdbcUrl(), postgreSqlServer.getProperties()), + "tpch.table", + "(col_required BIGINT NOT NULL," + + "col_nullable BIGINT," + + "col_default BIGINT DEFAULT 43," + + "col_nonnull_default BIGINT NOT NULL DEFAULT 42," + + "col_required2 BIGINT NOT NULL)"); + } + @Test public void testDropTable() { @@ -84,16 +119,6 @@ public void testDropTable() assertFalse(getQueryRunner().tableExists(getSession(), "test_drop")); } - @Test - public void testInsert() - throws Exception - { - execute("CREATE TABLE tpch.test_insert (x bigint, y varchar(100))"); - assertUpdate("INSERT INTO test_insert VALUES (123, 'test')", 1); - assertQuery("SELECT * FROM test_insert", "SELECT 123 x, 'test' y"); - assertUpdate("DROP TABLE test_insert"); - } - @Test public void testInsertInPresenceOfNotSupportedColumn() throws Exception diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlDistributedQueries.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlDistributedQueries.java deleted file mode 100644 index 32d6a2d09463..000000000000 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlDistributedQueries.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.postgresql; - -import com.google.common.collect.ImmutableMap; -import io.trino.testing.AbstractTestDistributedQueries; -import io.trino.testing.QueryRunner; -import io.trino.testing.sql.JdbcSqlExecutor; -import io.trino.testing.sql.TestTable; -import io.trino.tpch.TpchTable; - -import static io.trino.plugin.postgresql.PostgreSqlQueryRunner.createPostgreSqlQueryRunner; - -public class TestPostgreSqlDistributedQueries - extends AbstractTestDistributedQueries -{ - private TestingPostgreSqlServer postgreSqlServer; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - postgreSqlServer = new TestingPostgreSqlServer(); - closeAfterClass(() -> { - postgreSqlServer.close(); - postgreSqlServer = null; - }); - return createPostgreSqlQueryRunner( - postgreSqlServer, - ImmutableMap.of(), - ImmutableMap.builder() - // caching here speeds up tests highly, caching is not used in smoke tests - .put("metadata.cache-ttl", "10m") - .put("metadata.cache-missing", "true") - .build(), - TpchTable.getTables()); - } - - @Override - protected boolean supportsDelete() - { - return false; - } - - @Override - protected boolean supportsViews() - { - return false; - } - - @Override - protected boolean supportsArrays() - { - // Arrays are supported conditionally. Check the defaults. - return new PostgreSqlConfig().getArrayMapping() != PostgreSqlConfig.ArrayMapping.DISABLED; - } - - @Override - protected boolean supportsCommentOnTable() - { - return false; - } - - @Override - protected TestTable createTableWithDefaultColumns() - { - return new TestTable( - new JdbcSqlExecutor(postgreSqlServer.getJdbcUrl(), postgreSqlServer.getProperties()), - "tpch.table", - "(col_required BIGINT NOT NULL," + - "col_nullable BIGINT," + - "col_default BIGINT DEFAULT 43," + - "col_nonnull_default BIGINT NOT NULL DEFAULT 42," + - "col_required2 BIGINT NOT NULL)"); - } - - // PostgreSQL specific tests should normally go in TestPostgreSqlIntegrationSmokeTest -} diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java index b5dbbf4a374a..b9ff74747a47 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java @@ -57,6 +57,7 @@ import static org.testng.Assert.assertNotNull; public class TestRaptorIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { @Override diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisDistributed.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java similarity index 51% rename from plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisDistributed.java rename to plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java index cc7e428d8611..af7c98e11c8e 100644 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisDistributed.java +++ b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java @@ -15,15 +15,15 @@ import com.google.common.collect.ImmutableMap; import io.trino.plugin.redis.util.RedisServer; -import io.trino.testing.AbstractTestQueries; +import io.trino.testing.BaseConnectorTest; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; -@Test -public class TestRedisDistributed - extends AbstractTestQueries +import static io.trino.plugin.redis.RedisQueryRunner.createRedisQueryRunner; + +public class TestRedisConnectorTest + extends BaseConnectorTest { private RedisServer redisServer; @@ -32,7 +32,7 @@ protected QueryRunner createQueryRunner() throws Exception { redisServer = new RedisServer(); - return RedisQueryRunner.createRedisQueryRunner(redisServer, ImmutableMap.of(), "string", TpchTable.getTables()); + return createRedisQueryRunner(redisServer, ImmutableMap.of(), "string", TpchTable.getTables()); } @AfterClass(alwaysRun = true) @@ -40,4 +40,52 @@ public void destroy() { redisServer.close(); } + + @Override + protected boolean supportsCreateSchema() + { + return false; + } + + @Override + protected boolean supportsCreateTable() + { + return false; + } + + @Override + protected boolean supportsInsert() + { + return false; + } + + @Override + protected boolean supportsDelete() + { + return false; + } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Override + protected boolean supportsArrays() + { + return false; + } + + @Override + protected boolean supportsCommentOnTable() + { + return false; + } + + @Override + protected boolean supportsCommentOnColumn() + { + return false; + } } diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisIntegrationSmokeTest.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisIntegrationSmokeTest.java deleted file mode 100644 index d11d6cc40c85..000000000000 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisIntegrationSmokeTest.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.redis; - -import io.trino.plugin.redis.util.RedisServer; -import io.trino.testing.AbstractTestIntegrationSmokeTest; -import io.trino.testing.QueryRunner; -import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; - -import static io.trino.plugin.redis.RedisQueryRunner.createRedisQueryRunner; -import static io.trino.tpch.TpchTable.CUSTOMER; -import static io.trino.tpch.TpchTable.NATION; -import static io.trino.tpch.TpchTable.ORDERS; -import static io.trino.tpch.TpchTable.REGION; - -@Test -public class TestRedisIntegrationSmokeTest - extends AbstractTestIntegrationSmokeTest -{ - private RedisServer redisServer; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - redisServer = new RedisServer(); - return createRedisQueryRunner(redisServer, "string", CUSTOMER, NATION, ORDERS, REGION); - } - - @AfterClass(alwaysRun = true) - public void destroy() - { - redisServer.close(); - } -} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerIntegrationSmokeTest.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerIntegrationSmokeTest.java index 05ead4dccfe3..f407cdf19008 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerIntegrationSmokeTest.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerIntegrationSmokeTest.java @@ -45,6 +45,7 @@ import static org.testng.Assert.assertTrue; public class TestSqlServerIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { protected TestingSqlServer sqlServer; diff --git a/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/TestThriftIntegrationSmokeTest.java b/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/TestThriftIntegrationSmokeTest.java index f9a796195800..a4d012304318 100644 --- a/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/TestThriftIntegrationSmokeTest.java +++ b/plugin/trino-thrift/src/test/java/io/trino/plugin/thrift/integration/TestThriftIntegrationSmokeTest.java @@ -24,6 +24,7 @@ import static io.trino.testing.QueryAssertions.assertContains; public class TestThriftIntegrationSmokeTest + // TODO extend BaseConnectorTest extends AbstractTestIntegrationSmokeTest { @Override diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestDistributedQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestDistributedQueries.java index c67cfc644c3a..7fb6e2511830 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestDistributedQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestDistributedQueries.java @@ -66,11 +66,28 @@ /** * Generic test for connectors exercising connector's read and write capabilities. * - * @see AbstractTestIntegrationSmokeTest + * @see BaseConnectorTest + * @deprecated Extend {@link BaseConnectorTest} instead. */ +@Deprecated public abstract class AbstractTestDistributedQueries extends AbstractTestQueries { + protected boolean supportsCreateSchema() + { + return true; + } + + protected boolean supportsCreateTable() + { + return true; + } + + protected boolean supportsInsert() + { + return true; + } + protected boolean supportsDelete() { return true; @@ -159,6 +176,11 @@ public void testResetSession() public void testCreateTable() { String tableName = "test_create_" + randomTableSuffix(); + if (!supportsCreateTable()) { + assertQueryFails("CREATE TABLE " + tableName + " (a bigint, b double, c varchar)", "This connector does not support creating tables"); + return; + } + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double, c varchar)"); assertTrue(getQueryRunner().tableExists(getSession(), tableName)); assertTableColumnNames(tableName, "a", "b", "c"); @@ -204,6 +226,10 @@ public void testCreateTable() public void testCreateTableAsSelect() { String tableName = "test_ctas" + randomTableSuffix(); + if (!supportsCreateTable()) { + assertQueryFails("CREATE TABLE IF NOT EXISTS " + tableName + " AS SELECT name, regionkey FROM nation", "This connector does not support creating tables with data"); + return; + } assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName + " AS SELECT name, regionkey FROM nation", "SELECT count(*) FROM nation"); assertTableColumnNames(tableName, "name", "regionkey"); assertUpdate("DROP TABLE " + tableName); @@ -294,6 +320,8 @@ protected void assertCreateTableAsSelect(Session session, @Language("SQL") Strin @Test public void testRenameTable() { + skipTestUnless(supportsCreateTable()); + String tableName = "test_rename_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x", 1); @@ -324,14 +352,12 @@ public void testRenameTable() @Test public void testCommentTable() { - String tableName = "test_comment_" + randomTableSuffix(); if (!supportsCommentOnTable()) { - assertUpdate("CREATE TABLE " + tableName + "(a integer)"); - assertQueryFails("COMMENT ON TABLE " + tableName + " IS 'new comment'", "This connector does not support setting table comments"); - assertUpdate("DROP TABLE " + tableName); + assertQueryFails("COMMENT ON TABLE nation IS 'new comment'", "This connector does not support setting table comments"); return; } + String tableName = "test_comment_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + "(a integer)"); // comment set @@ -375,14 +401,12 @@ private String getTableComment(String tableName) @Test public void testCommentColumn() { - String tableName = "test_comment_column_" + randomTableSuffix(); if (!supportsCommentOnColumn()) { - assertUpdate("CREATE TABLE " + tableName + "(a integer)"); - assertQueryFails("COMMENT ON COLUMN " + tableName + ".a IS 'new comment'", "This connector does not support setting column comments"); - assertUpdate("DROP TABLE " + tableName); + assertQueryFails("COMMENT ON COLUMN nation.nationkey IS 'new comment'", "This connector does not support setting column comments"); return; } + String tableName = "test_comment_column_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + "(a integer)"); // comment set @@ -425,6 +449,8 @@ private String getColumnComment(String tableName, String columnName) @Test public void testRenameColumn() { + skipTestUnless(supportsCreateTable()); + String tableName = "test_rename_column_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT 'some value' x", 1); @@ -456,6 +482,8 @@ public void testRenameColumn() @Test public void testDropColumn() { + skipTestUnless(supportsCreateTable()); + String tableName = "test_drop_column_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x, 456 y, 111 a", 1); @@ -478,6 +506,8 @@ public void testDropColumn() @Test public void testAddColumn() { + skipTestUnless(supportsCreateTable()); + String tableName = "test_add_column_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT CAST('first' AS varchar) x", 1); @@ -514,6 +544,11 @@ public void testAddColumn() @Test public void testInsert() { + if (!supportsInsert()) { + assertQueryFails("INSERT INTO nation(nationkey) VALUES (42)", "This connector does not support inserts"); + return; + } + String query = "SELECT phone, custkey, acctbal FROM customer"; String tableName = "test_insert_" + randomTableSuffix(); @@ -554,6 +589,8 @@ public void testInsert() @Test public void testInsertUnicode() { + skipTestUnless(supportsInsert()); + String tableName = "test_insert_unicode_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + "(test varchar)"); @@ -584,6 +621,8 @@ public void testInsertUnicode() @Test public void testInsertArray() { + skipTestUnless(supportsInsert()); + String tableName = "test_insert_array_" + randomTableSuffix(); if (!supportsArrays()) { assertThatThrownBy(() -> query("CREATE TABLE " + tableName + " (a array(bigint))")) @@ -604,18 +643,15 @@ public void testInsertArray() @Test public void testDelete() { - // delete half the table, then delete the rest - String tableName = "test_delete_" + randomTableSuffix(); - if (!supportsDelete()) { - assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM orders WITH NO DATA", 0); - assertQueryFails("DELETE FROM " + tableName, "This connector does not support deletes"); - assertUpdate("DROP TABLE " + tableName); + assertQueryFails("DELETE FROM nation", "This connector does not support deletes"); return; } + String tableName = "test_delete_" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM orders", "SELECT count(*) FROM orders"); + // delete half the table, then delete the rest assertUpdate("DELETE FROM " + tableName + " WHERE orderkey % 2 = 0", "SELECT count(*) FROM orders WHERE orderkey % 2 = 0"); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM orders WHERE orderkey % 2 <> 0"); @@ -956,6 +992,8 @@ public void testShowCreateView() @Test public void testQueryLoggingCount() { + skipTestUnless(supportsCreateTable()); + QueryManager queryManager = getDistributedQueryRunner().getCoordinator().getQueryManager(); executeExclusively(() -> { assertEventually( @@ -1011,9 +1049,12 @@ public void testShowSchemasFromOther() assertTrue(result.getOnlyColumnAsSet().containsAll(ImmutableSet.of(INFORMATION_SCHEMA, "tiny", "sf1"))); } + // TODO move to to engine-only @Test public void testSymbolAliasing() { + skipTestUnless(supportsCreateTable()); + String tableName = "test_symbol_aliasing" + randomTableSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT 1 foo_1, 2 foo_2_4", 1); assertQuery("SELECT foo_1, foo_2_4 FROM " + tableName, "SELECT 1, 2"); @@ -1024,6 +1065,9 @@ public void testSymbolAliasing() @Flaky(issue = "https://github.com/trinodb/trino/issues/5172", match = "AssertionError: expected \\[.*\\] but found \\[.*\\]") public void testWrittenStats() { + skipTestUnless(supportsCreateTable()); + skipTestUnless(supportsInsert()); + String tableName = "test_written_stats_" + randomTableSuffix(); String sql = "CREATE TABLE " + tableName + " AS SELECT * FROM nation"; ResultWithQueryId resultResultWithQueryId = getDistributedQueryRunner().executeWithQueryId(getSession(), sql); @@ -1048,6 +1092,10 @@ public void testWrittenStats() public void testCreateSchema() { String schemaName = "test_schema_create_" + randomTableSuffix(); + if (!supportsCreateSchema()) { + assertQueryFails("CREATE SCHEMA " + schemaName, "This connector does not support creating schemas"); + return; + } assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).doesNotContain(schemaName); assertUpdate("CREATE SCHEMA " + schemaName); assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).contains(schemaName); @@ -1059,6 +1107,8 @@ public void testCreateSchema() @Test public void testInsertForDefaultColumn() { + skipTestUnless(supportsInsert()); + try (TestTable testTable = createTableWithDefaultColumns()) { assertUpdate(format("INSERT INTO %s (col_required, col_required2) VALUES (1, 10)", testTable.getName()), 1); assertUpdate(format("INSERT INTO %s VALUES (2, 3, 4, 5, 6)", testTable.getName()), 1); @@ -1069,11 +1119,16 @@ public void testInsertForDefaultColumn() } } - protected abstract TestTable createTableWithDefaultColumns(); + protected TestTable createTableWithDefaultColumns() + { + throw new UnsupportedOperationException(); + } @Test(dataProvider = "testColumnNameDataProvider") public void testColumnName(String columnName) { + skipTestUnless(supportsCreateTable()); + if (!requiresDelimiting(columnName)) { testColumnName(columnName, false); } @@ -1173,6 +1228,8 @@ protected String dataMappingTableName(String trinoTypeName) @Test(dataProvider = "testDataMappingSmokeTestDataProvider") public void testDataMappingSmokeTest(DataMappingTestSetup dataMappingTestSetup) { + skipTestUnless(supportsCreateTable()); + String trinoTypeName = dataMappingTestSetup.getTrinoTypeName(); String sampleValueLiteral = dataMappingTestSetup.getSampleValueLiteral(); String highValueLiteral = dataMappingTestSetup.getHighValueLiteral(); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestIntegrationSmokeTest.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestIntegrationSmokeTest.java index db514efde477..535221b2aa1f 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestIntegrationSmokeTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestIntegrationSmokeTest.java @@ -32,12 +32,9 @@ import static org.testng.Assert.assertTrue; /** - * Generic test for connectors exercising connector's read capabilities. - * This is also the base class for connector-specific tests (not generic), - * regardless whether they exercise read-only or read-write capabilities. - * - * @see AbstractTestDistributedQueries + * @deprecated Use {@link BaseConnectorTest} instead. */ +@Deprecated public abstract class AbstractTestIntegrationSmokeTest extends AbstractTestQueryFramework { diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java index 5272c421c52b..9c165821f1a5 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java @@ -902,7 +902,7 @@ public void testScalarSubquery() } @Test - public void testPredicatePushdown() + public void testPredicate() { assertQuery("" + "SELECT *\n" + diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java new file mode 100644 index 000000000000..003fa8043071 --- /dev/null +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -0,0 +1,412 @@ +/* + * 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.testing; + +import io.trino.Session; +import io.trino.sql.analyzer.FeaturesConfig.JoinDistributionType; +import org.intellij.lang.annotations.Language; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.stream.Stream; + +import static io.trino.SystemSessionProperties.IGNORE_STATS_CALCULATOR_FAILURES; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.DataProviders.toDataProvider; +import static io.trino.testing.QueryAssertions.assertContains; +import static io.trino.testing.assertions.Assert.assertEquals; +import static java.lang.String.join; +import static java.util.Collections.nCopies; +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertTrue; + +/** + * Generic test for connectors. + * + * @see AbstractTestDistributedQueries + */ +public abstract class BaseConnectorTest + extends AbstractTestDistributedQueries +{ + @Test + public void testColumnsInReverseOrder() + { + assertQuery("SELECT shippriority, clerk, totalprice FROM orders"); + } + + @Test + public void testAggregation() + { + assertQuery("SELECT sum(orderkey) FROM orders"); + assertQuery("SELECT sum(totalprice) FROM orders"); + assertQuery("SELECT max(comment) FROM nation"); + + assertQuery("SELECT count(*) FROM orders"); + assertQuery("SELECT count(*) FROM orders WHERE orderkey > 10"); + assertQuery("SELECT count(*) FROM (SELECT * FROM orders LIMIT 10)"); + assertQuery("SELECT count(*) FROM (SELECT * FROM orders WHERE orderkey > 10 LIMIT 10)"); + + assertQuery("SELECT DISTINCT regionkey FROM nation"); + assertQuery("SELECT regionkey FROM nation GROUP BY regionkey"); + + // TODO support aggregation pushdown with GROUPING SETS + assertQuery( + "SELECT regionkey, nationkey FROM nation GROUP BY GROUPING SETS ((regionkey), (nationkey))", + "SELECT NULL, nationkey FROM nation " + + "UNION ALL SELECT DISTINCT regionkey, NULL FROM nation"); + assertQuery( + "SELECT regionkey, nationkey, count(*) FROM nation GROUP BY GROUPING SETS ((), (regionkey), (nationkey), (regionkey, nationkey))", + "SELECT NULL, NULL, count(*) FROM nation " + + "UNION ALL SELECT NULL, nationkey, 1 FROM nation " + + "UNION ALL SELECT regionkey, NULL, count(*) FROM nation GROUP BY regionkey " + + "UNION ALL SELECT regionkey, nationkey, 1 FROM nation"); + + assertQuery("SELECT count(regionkey) FROM nation"); + assertQuery("SELECT count(DISTINCT regionkey) FROM nation"); + assertQuery("SELECT regionkey, count(*) FROM nation GROUP BY regionkey"); + + assertQuery("SELECT min(regionkey), max(regionkey) FROM nation"); + assertQuery("SELECT min(DISTINCT regionkey), max(DISTINCT regionkey) FROM nation"); + assertQuery("SELECT regionkey, min(regionkey), min(name), max(regionkey), max(name) FROM nation GROUP BY regionkey"); + + assertQuery("SELECT sum(regionkey) FROM nation"); + assertQuery("SELECT sum(DISTINCT regionkey) FROM nation"); + assertQuery("SELECT regionkey, sum(regionkey) FROM nation GROUP BY regionkey"); + + assertQuery( + "SELECT avg(nationkey) FROM nation", + "SELECT avg(CAST(nationkey AS double)) FROM nation"); + assertQuery( + "SELECT avg(DISTINCT nationkey) FROM nation", + "SELECT avg(DISTINCT CAST(nationkey AS double)) FROM nation"); + assertQuery( + "SELECT regionkey, avg(nationkey) FROM nation GROUP BY regionkey", + "SELECT regionkey, avg(CAST(nationkey AS double)) FROM nation GROUP BY regionkey"); + } + + @Test + public void testExactPredicate() + { + assertQueryReturnsEmptyResult("SELECT * FROM orders WHERE orderkey = 10"); + + // filtered column is selected + assertQuery("SELECT custkey, orderkey FROM orders WHERE orderkey = 32", "VALUES (1301, 32)"); + + // filtered column is not selected + assertQuery("SELECT custkey FROM orders WHERE orderkey = 32", "VALUES (1301)"); + } + + @Test + public void testInListPredicate() + { + assertQueryReturnsEmptyResult("SELECT * FROM orders WHERE orderkey IN (10, 11, 20, 21)"); + + // filtered column is selected + assertQuery("SELECT custkey, orderkey FROM orders WHERE orderkey IN (7, 10, 32, 33)", "VALUES (392, 7), (1301, 32), (670, 33)"); + + // filtered column is not selected + assertQuery("SELECT custkey FROM orders WHERE orderkey IN (7, 10, 32, 33)", "VALUES (392), (1301), (670)"); + } + + @Test + public void testIsNullPredicate() + { + assertQueryReturnsEmptyResult("SELECT * FROM orders WHERE orderkey IS NULL"); + assertQueryReturnsEmptyResult("SELECT * FROM orders WHERE orderkey = 10 OR orderkey IS NULL"); + + // filtered column is selected + assertQuery("SELECT custkey, orderkey FROM orders WHERE orderkey = 32 OR orderkey IS NULL", "VALUES (1301, 32)"); + + // filtered column is not selected + assertQuery("SELECT custkey FROM orders WHERE orderkey = 32 OR orderkey IS NULL", "VALUES (1301)"); + } + + @Test + public void testLikePredicate() + { + // filtered column is not selected + assertQuery("SELECT orderkey FROM orders WHERE orderpriority LIKE '5-L%'"); + + // filtered column is selected + assertQuery("SELECT orderkey, orderpriority FROM orders WHERE orderpriority LIKE '5-L%'"); + + // filtered column is not selected + assertQuery("SELECT orderkey FROM orders WHERE orderpriority LIKE '5-L__'"); + + // filtered column is selected + assertQuery("SELECT orderkey, orderpriority FROM orders WHERE orderpriority LIKE '5-L__'"); + } + + @Test + public void testMultipleRangesPredicate() + { + // List columns explicitly. Some connectors do not maintain column ordering. + assertQuery("" + + "SELECT orderkey, custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment " + + "FROM orders " + + "WHERE orderkey BETWEEN 10 AND 50"); + } + + @Test + public void testRangePredicate() + { + // List columns explicitly. Some connectors do not maintain column ordering. + assertQuery("" + + "SELECT orderkey, custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment " + + "FROM orders " + + "WHERE orderkey BETWEEN 10 AND 50"); + } + + @Test + public void testConcurrentScans() + { + String unionMultipleTimes = join(" UNION ALL ", nCopies(25, "SELECT * FROM orders")); + assertQuery("SELECT sum(if(rand() >= 0, orderkey)) FROM (" + unionMultipleTimes + ")", "VALUES 11246812500"); + } + + @Test + public void testSelectAll() + { + assertQuery("SELECT * FROM orders"); + } + + /** + * Test interactions between optimizer (including CBO), scheduling and connector metadata APIs. + */ + @Test(timeOut = 300_000, dataProvider = "joinDistributionTypes") + public void testJoinWithEmptySides(JoinDistributionType joinDistributionType) + { + Session session = noJoinReordering(joinDistributionType); + // empty build side + assertQuery(session, "SELECT count(*) FROM nation JOIN region ON nation.regionkey = region.regionkey AND region.name = ''", "VALUES 0"); + assertQuery(session, "SELECT count(*) FROM nation JOIN region ON nation.regionkey = region.regionkey AND region.regionkey < 0", "VALUES 0"); + // empty probe side + assertQuery(session, "SELECT count(*) FROM region JOIN nation ON nation.regionkey = region.regionkey AND region.name = ''", "VALUES 0"); + assertQuery(session, "SELECT count(*) FROM nation JOIN region ON nation.regionkey = region.regionkey AND region.regionkey < 0", "VALUES 0"); + } + + @DataProvider + public Object[][] joinDistributionTypes() + { + return Stream.of(JoinDistributionType.values()) + .collect(toDataProvider()); + } + + /** + * Test interactions between optimizer (including CBO) and connector metadata APIs. + */ + @Test + public void testJoin() + { + Session session = Session.builder(getSession()) + .setSystemProperty(IGNORE_STATS_CALCULATOR_FAILURES, "false") + .build(); + + // 2 inner joins, eligible for join reodering + assertQuery( + session, + "SELECT c.name, n.name, r.name " + + "FROM nation n " + + "JOIN customer c ON c.nationkey = n.nationkey " + + "JOIN region r ON n.regionkey = r.regionkey"); + + // 2 inner joins, eligible for join reodering, where one table has a filter + assertQuery( + session, + "SELECT c.name, n.name, r.name " + + "FROM nation n " + + "JOIN customer c ON c.nationkey = n.nationkey " + + "JOIN region r ON n.regionkey = r.regionkey " + + "WHERE n.name = 'ARGENTINA'"); + + // 2 inner joins, eligible for join reodering, on top of aggregation + assertQuery( + session, + "SELECT c.name, n.name, n.count, r.name " + + "FROM (SELECT name, regionkey, nationkey, count(*) count FROM nation GROUP BY name, regionkey, nationkey) n " + + "JOIN customer c ON c.nationkey = n.nationkey " + + "JOIN region r ON n.regionkey = r.regionkey"); + } + + @Test + public void testDescribeTable() + { + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar(1)", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "date", "", "") + .row("orderpriority", "varchar(15)", "", "") + .row("clerk", "varchar(15)", "", "") + .row("shippriority", "integer", "", "") + .row("comment", "varchar(79)", "", "") + .build(); + MaterializedResult actualColumns = computeActual("DESCRIBE orders"); + assertEquals(actualColumns, expectedColumns); + } + + @Test + public void testExplainAnalyze() + { + assertExplainAnalyze("EXPLAIN ANALYZE SELECT * FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE SELECT count(*), clerk FROM orders GROUP BY clerk"); + assertExplainAnalyze( + "EXPLAIN ANALYZE SELECT x + y FROM (" + + " SELECT orderdate, COUNT(*) x FROM orders GROUP BY orderdate) a JOIN (" + + " SELECT orderdate, COUNT(*) y FROM orders GROUP BY orderdate) b ON a.orderdate = b.orderdate"); + assertExplainAnalyze("EXPLAIN ANALYZE SELECT count(*), clerk FROM orders GROUP BY clerk UNION ALL SELECT sum(orderkey), clerk FROM orders GROUP BY clerk"); + + assertExplainAnalyze("EXPLAIN ANALYZE SHOW COLUMNS FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE EXPLAIN SELECT count(*) FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE EXPLAIN ANALYZE SELECT count(*) FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE SHOW FUNCTIONS"); + assertExplainAnalyze("EXPLAIN ANALYZE SHOW TABLES"); + assertExplainAnalyze("EXPLAIN ANALYZE SHOW SCHEMAS"); + assertExplainAnalyze("EXPLAIN ANALYZE SHOW CATALOGS"); + assertExplainAnalyze("EXPLAIN ANALYZE SHOW SESSION"); + } + + @Test + public void testExplainAnalyzeVerbose() + { + assertExplainAnalyze("EXPLAIN ANALYZE VERBOSE SELECT * FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE VERBOSE SELECT rank() OVER (PARTITION BY orderkey ORDER BY clerk DESC) FROM orders"); + assertExplainAnalyze("EXPLAIN ANALYZE VERBOSE SELECT rank() OVER (PARTITION BY orderkey ORDER BY clerk DESC) FROM orders WHERE orderkey < 0"); + } + + @Test + public void testTableSampleSystem() + { + MaterializedResult fullSample = computeActual("SELECT orderkey FROM orders TABLESAMPLE SYSTEM (100)"); + MaterializedResult emptySample = computeActual("SELECT orderkey FROM orders TABLESAMPLE SYSTEM (0)"); + MaterializedResult randomSample = computeActual("SELECT orderkey FROM orders TABLESAMPLE SYSTEM (50)"); + MaterializedResult all = computeActual("SELECT orderkey FROM orders"); + + assertContains(all, fullSample); + assertEquals(emptySample.getMaterializedRows().size(), 0); + assertTrue(all.getMaterializedRows().size() >= randomSample.getMaterializedRows().size()); + } + + @Test + public void testTableSampleWithFiltering() + { + MaterializedResult emptySample = computeActual("SELECT DISTINCT orderkey, orderdate FROM orders TABLESAMPLE SYSTEM (99) WHERE orderkey BETWEEN 0 AND 0"); + MaterializedResult halfSample = computeActual("SELECT DISTINCT orderkey, orderdate FROM orders TABLESAMPLE SYSTEM (50) WHERE orderkey BETWEEN 0 AND 9999999999"); + MaterializedResult all = computeActual("SELECT orderkey, orderdate FROM orders"); + + assertEquals(emptySample.getMaterializedRows().size(), 0); + // Assertions need to be loose here because SYSTEM sampling random selects data on split boundaries. In this case either all the data will be selected, or + // none of it. Sampling with a 100% ratio is ignored, so that also cannot be used to guarantee results. + assertTrue(all.getMaterializedRows().size() >= halfSample.getMaterializedRows().size()); + } + + @Test + public void testShowCreateTable() + { + assertThat((String) computeActual("SHOW CREATE TABLE orders").getOnlyValue()) + // If the connector reports additional column properties, the expected value needs to be adjusted in the test subclass + .matches("CREATE TABLE \\w+\\.\\w+\\.orders \\Q(\n" + + " orderkey bigint,\n" + + " custkey bigint,\n" + + " orderstatus varchar(1),\n" + + " totalprice double,\n" + + " orderdate date,\n" + + " orderpriority varchar(15),\n" + + " clerk varchar(15),\n" + + " shippriority integer,\n" + + " comment varchar(79)\n" + + ")"); + } + + @Test + public void testSelectInformationSchemaTables() + { + String catalog = getSession().getCatalog().get(); + String schema = getSession().getSchema().get(); + String schemaPattern = schema.replaceAll("^.", "_"); + + assertQuery("SELECT table_name FROM information_schema.tables WHERE table_schema = '" + schema + "' AND table_name = 'orders'", "VALUES 'orders'"); + assertQuery("SELECT table_name FROM information_schema.tables WHERE table_schema LIKE '" + schema + "' AND table_name LIKE '%rders'", "VALUES 'orders'"); + assertQuery("SELECT table_name FROM information_schema.tables WHERE table_schema LIKE '" + schemaPattern + "' AND table_name LIKE '%rders'", "VALUES 'orders'"); + assertQuery( + "SELECT table_name FROM information_schema.tables " + + "WHERE table_catalog = '" + catalog + "' AND table_schema LIKE '" + schema + "' AND table_name LIKE '%orders'", + "VALUES 'orders'"); + assertQuery("SELECT table_name FROM information_schema.tables WHERE table_catalog = 'something_else'", "SELECT '' WHERE false"); + + assertQuery( + "SELECT DISTINCT table_name FROM information_schema.tables WHERE table_schema = 'information_schema' OR rand() = 42 ORDER BY 1", + "VALUES " + + "('applicable_roles'), " + + "('columns'), " + + "('enabled_roles'), " + + "('role_authorization_descriptors'), " + + "('roles'), " + + "('schemata'), " + + "('table_privileges'), " + + "('tables'), " + + "('views')"); + } + + @Test + public void testSelectInformationSchemaColumns() + { + String catalog = getSession().getCatalog().get(); + String schema = getSession().getSchema().get(); + String schemaPattern = schema.replaceAll(".$", "_"); + + @Language("SQL") String ordersTableWithColumns = "VALUES " + + "('orders', 'orderkey'), " + + "('orders', 'custkey'), " + + "('orders', 'orderstatus'), " + + "('orders', 'totalprice'), " + + "('orders', 'orderdate'), " + + "('orders', 'orderpriority'), " + + "('orders', 'clerk'), " + + "('orders', 'shippriority'), " + + "('orders', 'comment')"; + + assertQuery("SELECT table_schema FROM information_schema.columns WHERE table_schema = '" + schema + "' GROUP BY table_schema", "VALUES '" + schema + "'"); + assertQuery("SELECT table_name FROM information_schema.columns WHERE table_name = 'orders' GROUP BY table_name", "VALUES 'orders'"); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema = '" + schema + "' AND table_name = 'orders'", ordersTableWithColumns); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema = '" + schema + "' AND table_name LIKE '%rders'", ordersTableWithColumns); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema LIKE '" + schemaPattern + "' AND table_name LIKE '_rder_'", ordersTableWithColumns); + assertQuery( + "SELECT table_name, column_name FROM information_schema.columns " + + "WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "' AND table_name LIKE '%orders%'", + ordersTableWithColumns); + + assertQuerySucceeds("SELECT * FROM information_schema.columns"); + assertQuery("SELECT DISTINCT table_name, column_name FROM information_schema.columns WHERE table_name LIKE '_rders'", ordersTableWithColumns); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "'"); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "'"); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "' AND table_name LIKE '_rders'", ordersTableWithColumns); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_name LIKE '%'"); + assertQuery("SELECT column_name FROM information_schema.columns WHERE table_catalog = 'something_else'", "SELECT '' WHERE false"); + + assertQuery( + "SELECT DISTINCT table_name FROM information_schema.columns WHERE table_schema = 'information_schema' OR rand() = 42 ORDER BY 1", + "VALUES " + + "('applicable_roles'), " + + "('columns'), " + + "('enabled_roles'), " + + "('role_authorization_descriptors'), " + + "('roles'), " + + "('schemata'), " + + "('table_privileges'), " + + "('tables'), " + + "('views')"); + } +}