From 81771c3e7d462fb44718ac6b07f91b44c52046a5 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 13:02:52 +0100 Subject: [PATCH] Prevent unwanted product test duplicated runs Verify that a product test has reasonably assigned groups. Incorrectly assigned groups may lead to a test executing too many times, or not executing at all. --- .../io/trino/tests/product/SuiteGroups.java | 52 ++++ .../io/trino/tests/product/TestGroups.java | 3 + .../product/launcher/suite/suites/Suite1.java | 29 +-- testing/trino-product-tests/pom.xml | 24 ++ .../tests/product/hive/TestHiveMerge.java | 51 ++-- .../TestHiveTransactionalTableInsert.java | 3 +- .../product/iceberg/TestCreateDropSchema.java | 9 +- .../test/java/io/trino/server/TestDummy.java | 22 -- .../tests/product/TestProductTestGroups.java | 244 ++++++++++++++++++ 9 files changed, 358 insertions(+), 79 deletions(-) create mode 100644 testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/SuiteGroups.java delete mode 100644 testing/trino-product-tests/src/test/java/io/trino/server/TestDummy.java create mode 100644 testing/trino-product-tests/src/test/java/io/trino/tests/product/TestProductTestGroups.java diff --git a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/SuiteGroups.java b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/SuiteGroups.java new file mode 100644 index 000000000000..eb0007154664 --- /dev/null +++ b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/SuiteGroups.java @@ -0,0 +1,52 @@ +/* + * 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.tests.product; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; + +import static io.trino.tests.product.TestGroups.AZURE; +import static io.trino.tests.product.TestGroups.CLI; +import static io.trino.tests.product.TestGroups.FUNCTIONS; +import static io.trino.tests.product.TestGroups.HIVE_COMPRESSION; +import static io.trino.tests.product.TestGroups.JDBC; +import static io.trino.tests.product.TestGroups.JDBC_KERBEROS_CONSTRAINED_DELEGATION; +import static io.trino.tests.product.TestGroups.LARGE_QUERY; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; +import static io.trino.tests.product.TestGroups.STORAGE_FORMATS_DETAILED; +import static io.trino.tests.product.TestGroups.TPCDS; +import static io.trino.tests.product.TestGroups.TPCH; +import static io.trino.tests.product.TestGroups.TRINO_JDBC; + +public abstract class SuiteGroups +{ + private SuiteGroups() {} + + public static final Set SUITE1_EXCLUSIONS = ImmutableSet.of( + AZURE, + CLI, + JDBC, + TRINO_JDBC, + JDBC_KERBEROS_CONSTRAINED_DELEGATION, + FUNCTIONS, + HIVE_COMPRESSION, + LARGE_QUERY, + PROFILE_SPECIFIC_TESTS, + STORAGE_FORMATS, + STORAGE_FORMATS_DETAILED, + TPCH, + TPCDS); +} diff --git a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java index a5f110672548..e795af938b5b 100644 --- a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java @@ -31,8 +31,11 @@ public final class TestGroups public static final String CONFIGURED_FEATURES = "configured_features"; public static final String TPCH = "tpch"; public static final String TPCDS = "tpcds"; + // TODO some tests from this group are run in Suite1 and Suite3, probably not intentionally public static final String JOIN = "join"; + // TODO some tests from this group are run in Suite1 and Suite3, probably not intentionally public static final String GROUP_BY = "group-by"; + // TODO some tests from this group are run in Suite1 and Suite3, probably not intentionally public static final String SMOKE = "smoke"; public static final String JDBC = "jdbc"; public static final String JDBC_KERBEROS_CONSTRAINED_DELEGATION = "jdbc_kerberos_constrained_delegation"; diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite1.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite1.java index 6b5565c764a9..b9c395d49c8f 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite1.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite1.java @@ -21,19 +21,7 @@ import java.util.List; -import static io.trino.tests.product.TestGroups.AZURE; -import static io.trino.tests.product.TestGroups.CLI; -import static io.trino.tests.product.TestGroups.FUNCTIONS; -import static io.trino.tests.product.TestGroups.HIVE_COMPRESSION; -import static io.trino.tests.product.TestGroups.JDBC; -import static io.trino.tests.product.TestGroups.JDBC_KERBEROS_CONSTRAINED_DELEGATION; -import static io.trino.tests.product.TestGroups.LARGE_QUERY; -import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; -import static io.trino.tests.product.TestGroups.STORAGE_FORMATS_DETAILED; -import static io.trino.tests.product.TestGroups.TPCDS; -import static io.trino.tests.product.TestGroups.TPCH; -import static io.trino.tests.product.TestGroups.TRINO_JDBC; +import static io.trino.tests.product.SuiteGroups.SUITE1_EXCLUSIONS; import static io.trino.tests.product.launcher.suite.SuiteTestRun.testOnEnvironment; public class Suite1 @@ -44,20 +32,7 @@ public List getTestRuns(EnvironmentConfig config) { return ImmutableList.of( testOnEnvironment(EnvMultinode.class) - .withExcludedGroups( - AZURE, - CLI, - JDBC, - TRINO_JDBC, - JDBC_KERBEROS_CONSTRAINED_DELEGATION, - FUNCTIONS, - HIVE_COMPRESSION, - LARGE_QUERY, - PROFILE_SPECIFIC_TESTS, - STORAGE_FORMATS, - STORAGE_FORMATS_DETAILED, - TPCH, - TPCDS) + .withExcludedGroups(SUITE1_EXCLUSIONS.toArray(new String[0])) .build()); } } diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index 3871965c1253..7e88c8b2af70 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -245,6 +245,12 @@ runtime + + com.google.errorprone + error_prone_annotations + runtime + + com.microsoft.sqlserver mssql-jdbc @@ -318,6 +324,24 @@ snappy-java runtime + + + io.airlift + junit-extensions + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java index 52046c538a91..c1c14f2731ed 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java @@ -29,6 +29,7 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V2; import static io.trino.tests.product.hive.BucketingType.NONE; import static io.trino.tests.product.hive.TestHiveTransactionalTable.TEST_TIMEOUT; @@ -42,7 +43,7 @@ public class TestHiveMerge extends HiveProductTest { - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeSimpleSelect() { withTemporaryTable("merge_simple_select_target", false, NONE, targetTable -> { @@ -67,7 +68,7 @@ public void testMergeSimpleSelect() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeSimpleSelectPartitioned() { withTemporaryTable("merge_simple_select_partitioned_target", true, NONE, targetTable -> { @@ -92,7 +93,7 @@ public void testMergeSimpleSelectPartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT, dataProvider = "partitionedAndBucketedProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT, dataProvider = "partitionedAndBucketedProvider") public void testMergeUpdateWithVariousLayouts(boolean partitioned, String bucketing) { BucketingType bucketingType = bucketing.isEmpty() ? NONE : BUCKETED_V2; @@ -129,7 +130,7 @@ public void testMergeUpdateWithVariousLayouts(boolean partitioned, String bucket }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testMergeUnBucketedUnPartitionedFailure() { withTemporaryTable("merge_with_various_formats_failure", false, NONE, targetTable -> { @@ -166,7 +167,7 @@ public Object[][] partitionedAndBucketedProvider() }; } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsUnbucketedUnpartitioned() { withTemporaryTable("merge_multiple", false, NONE, targetTable -> { @@ -175,7 +176,7 @@ public void testMergeMultipleOperationsUnbucketedUnpartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsUnbucketedPartitioned() { withTemporaryTable("merge_multiple", true, NONE, targetTable -> { @@ -184,7 +185,7 @@ public void testMergeMultipleOperationsUnbucketedPartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsBucketedUnpartitioned() { withTemporaryTable("merge_multiple", false, BUCKETED_V2, targetTable -> { @@ -259,7 +260,7 @@ private List getRowsFromQueryResult(QueryResult result) return result.rows().stream().map(QueryAssert.Row::new).collect(toImmutableList()); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeSimpleQuery() { withTemporaryTable("merge_simple_query_target", false, NONE, targetTable -> { @@ -279,7 +280,7 @@ public void testMergeSimpleQuery() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeAllInserts() { withTemporaryTable("merge_all_inserts", false, NONE, targetTable -> { @@ -297,7 +298,7 @@ public void testMergeAllInserts() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeSimpleQueryPartitioned() { withTemporaryTable("merge_simple_query_partitioned_target", true, NONE, targetTable -> { @@ -318,7 +319,7 @@ public void testMergeSimpleQueryPartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeAllColumnsUpdated() { withTemporaryTable("merge_all_columns_updated_target", false, NONE, targetTable -> { @@ -339,7 +340,7 @@ public void testMergeAllColumnsUpdated() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeAllMatchesDeleted() { withTemporaryTable("merge_all_matches_deleted_target", false, NONE, targetTable -> { @@ -360,7 +361,7 @@ public void testMergeAllMatchesDeleted() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000, dataProvider = "partitionedBucketedFailure") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000, dataProvider = "partitionedBucketedFailure") public void testMergeMultipleRowsMatchFails(String createTableSql) { withTemporaryTable("merge_all_matches_deleted_target", true, NONE, targetTable -> { @@ -396,7 +397,7 @@ public Object[][] partitionedBucketedFailure() }; } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeFailingPartitioning() { String testDescription = "failing_merge"; @@ -422,7 +423,7 @@ public void testMergeFailingPartitioning() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeFailureWithDifferentPartitioning() { testMergeWithDifferentPartitioningInternal( @@ -431,7 +432,7 @@ public void testMergeFailureWithDifferentPartitioning() "CREATE TABLE %s (customer STRING, purchases INT) PARTITIONED BY (address STRING) CLUSTERED BY (customer) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')"); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000, dataProvider = "targetAndSourceWithDifferentPartitioning") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000, dataProvider = "targetAndSourceWithDifferentPartitioning") public void testMergeWithDifferentPartitioning(String testDescription, String createTargetTableSql, String createSourceTableSql) { testMergeWithDifferentPartitioningInternal(testDescription, createTargetTableSql, createSourceTableSql); @@ -498,7 +499,7 @@ public Object[][] targetAndSourceWithDifferentPartitioning() }; } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeQueryWithStrangeCapitalization() { withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { @@ -517,7 +518,7 @@ public void testMergeQueryWithStrangeCapitalization() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeWithoutTablesAliases() { withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { @@ -541,7 +542,7 @@ public void testMergeWithoutTablesAliases() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeWithUnpredictablePredicates() { withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { @@ -582,7 +583,7 @@ public void testMergeWithUnpredictablePredicates() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeWithSimplifiedUnpredictablePredicates() { withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { @@ -607,7 +608,7 @@ public void testMergeWithSimplifiedUnpredictablePredicates() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeCasts() { withTemporaryTable("merge_cast_target", false, NONE, targetTable -> { @@ -629,7 +630,7 @@ public void testMergeCasts() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeSubqueries() { withTemporaryTable("merge_nation_target", false, NONE, targetTable -> { @@ -654,7 +655,7 @@ public void testMergeSubqueries() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = 60 * 60 * 1000) public void testMergeOriginalFilesTarget() { withTemporaryTable("region", false, NONE, targetTable -> { @@ -676,7 +677,7 @@ public void testMergeOriginalFilesTarget() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testMergeOverManySplits() { withTemporaryTable("delete_select", false, NONE, targetTable -> { @@ -694,7 +695,7 @@ public void testMergeOverManySplits() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testMergeFalseJoinCondition() { withTemporaryTable("join_false", false, NONE, targetTable -> { diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java index 3d81a206265c..c849148be3f2 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java @@ -18,6 +18,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.utils.QueryExecutors.onHive; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.util.Locale.ENGLISH; @@ -27,7 +28,7 @@ public class TestHiveTransactionalTableInsert extends HiveProductTest { - @Test(dataProvider = "transactionalTableType", groups = HIVE_TRANSACTIONAL) + @Test(dataProvider = "transactionalTableType", groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) public void testInsertIntoTransactionalTable(TransactionalTableType type) { String tableName = "test_insert_into_transactional_table_" + type.name().toLowerCase(ENGLISH); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestCreateDropSchema.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestCreateDropSchema.java index efe3162a00f0..5630e4f7bc7f 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestCreateDropSchema.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestCreateDropSchema.java @@ -23,6 +23,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.ICEBERG; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -44,7 +45,7 @@ public void useIceberg() onTrino().executeQuery("USE iceberg.default"); } - @Test(groups = ICEBERG) + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testDropSchemaFiles() { String schemaName = "schema_without_location_" + randomNameSuffix(); @@ -56,7 +57,7 @@ public void testDropSchemaFiles() assertFileExistence(schemaDir, false, "schema directory exists after dropping schema"); } - @Test(groups = ICEBERG) + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testDropSchemaFilesWithLocation() { String schemaName = "schema_with_empty_location_" + randomNameSuffix(); @@ -68,7 +69,7 @@ public void testDropSchemaFilesWithLocation() assertFileExistence(schemaDir, false, "schema directory exists after dropping schema"); } - @Test(groups = ICEBERG) // specified location, external file in subdir + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) // specified location, external file in subdir public void testDropWithExternalFilesInSubdirectory() { String schemaName = "schema_with_nonempty_location_" + randomNameSuffix(); @@ -89,7 +90,7 @@ public void testDropWithExternalFilesInSubdirectory() hdfsClient.delete(schemaDir); } - @Test(groups = ICEBERG) // default location, external file at top level + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) // default location, external file at top level public void testDropWithExternalFiles() { String schemaName = "schema_with_files_in_default_location_" + randomNameSuffix(); diff --git a/testing/trino-product-tests/src/test/java/io/trino/server/TestDummy.java b/testing/trino-product-tests/src/test/java/io/trino/server/TestDummy.java deleted file mode 100644 index dea00f6fd596..000000000000 --- a/testing/trino-product-tests/src/test/java/io/trino/server/TestDummy.java +++ /dev/null @@ -1,22 +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.server; - -import org.testng.annotations.Test; - -public class TestDummy -{ - @Test - public void buildRequiresTestToExist() {} -} diff --git a/testing/trino-product-tests/src/test/java/io/trino/tests/product/TestProductTestGroups.java b/testing/trino-product-tests/src/test/java/io/trino/tests/product/TestProductTestGroups.java new file mode 100644 index 000000000000..32e07c3e1362 --- /dev/null +++ b/testing/trino-product-tests/src/test/java/io/trino/tests/product/TestProductTestGroups.java @@ -0,0 +1,244 @@ +/* + * 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.tests.product; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.reflect.ClassPath; +import com.google.errorprone.annotations.FormatMethod; +import io.trino.tempto.internal.convention.ConventionBasedTest; +import io.trino.tempto.internal.convention.ConventionBasedTestFactory; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.Sets.difference; +import static com.google.common.collect.Sets.intersection; +import static io.trino.tests.product.SuiteGroups.SUITE1_EXCLUSIONS; +import static io.trino.tests.product.TestGroups.CONFIGURED_FEATURES; +import static io.trino.tests.product.TestGroups.GROUP_BY; +import static io.trino.tests.product.TestGroups.HIVE_FILE_HEADER; +import static io.trino.tests.product.TestGroups.JOIN; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.TestGroups.SMOKE; +import static io.trino.tests.product.TestProductTestGroups.ProductTestInfo.Type.CONVENTION_BASED; +import static io.trino.tests.product.TestProductTestGroups.ProductTestInfo.Type.JAVA; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; + +@Execution(SAME_THREAD) // ConventionBasedTestFactory is not thread-safe +public class TestProductTestGroups +{ + // TODO these are test groups used in convention tests; we should probably remove these groups and also these tests too + @Deprecated + private static final Set IGNORED_CONVENTION_BASED_TEST_GROUPS = ImmutableSet.builder() + .add("varchar") + .add("base_sql") + .add("no_from") + .add("insert") + .add("conditional") + .add("limit") + .add("distinct") + .add("aggregate") + .add("window") + .add("union") + .add("orderby") + .add("set_operation") + .add("with_clause") + .add("qe") + .add("hive") + .add("array_functions") + .add("binary_functions") + .add("conversion_functions") + .add("horology_functions") + .add("json_functions") + .add("map_functions") + .add("math_functions") + .add("ml_functions") + .add("regex_functions") + .add("string_functions") + .add("url_functions") + .add("color") + .add("empty") + .add("system") + .add("jmx") + .add("tpch_connector") // TODO replace with tpch + .build(); + + // TODO these are tests run (not excluded) in Suite1, and also included in other tests + @Deprecated + private static final Set KNOWN_DUPLICATE_GROUPS = ImmutableSet.builder() + .add(SMOKE) + .add(JOIN) + .add(GROUP_BY) + .add(HIVE_FILE_HEADER) + .build(); + + @Test + public void selfTest() + { + Set definedGroups = definedTestGroups(); + + Set conventionBasedGroups = conventionBasedProductTests() + .flatMap(test -> test.groups().stream()) + .collect(toImmutableSet()); + + assertThat(intersection(definedGroups, IGNORED_CONVENTION_BASED_TEST_GROUPS)).as("IGNORED_CONVENTION_TEST_GROUPS must not contain any valid groups") + .isEmpty(); + + assertThat(difference(IGNORED_CONVENTION_BASED_TEST_GROUPS, conventionBasedGroups)).as("IGNORED_CONVENTION_BASED_TEST_GROUPS must not contain any groups test are not in use") + .isEmpty(); + + assertThat(intersection(KNOWN_DUPLICATE_GROUPS, SUITE1_EXCLUSIONS)).as("KNOWN_DUPLICATE_GROUPS must not contain any groups from SUITE1_EXCLUSIONS") + .isEmpty(); + + assertThat(difference(KNOWN_DUPLICATE_GROUPS, definedGroups)).as("KNOWN_DUPLICATE_GROUPS must contain only valid TestGroups") + .isEmpty(); + } + + @Test + public void testGroupsReasonable() + throws Exception + { + Set definedGroups = definedTestGroups(); + List errors = new ArrayList<>(); + + AtomicInteger testMethods = new AtomicInteger(); + productTests().forEach(test -> { + testMethods.incrementAndGet(); + + Set groups = ImmutableSet.copyOf(test.groups()); + assertThat(groups).as("Duplicate test groups in %s", test.name()) + .hasSize(test.groups().size()); + + if (test.type() == CONVENTION_BASED) { + groups = difference(groups, IGNORED_CONVENTION_BASED_TEST_GROUPS); + } + Set invalidGroups = difference(groups, definedGroups); + check(errors::add, invalidGroups.isEmpty(), "Invalid groups in %s: %s", test.name(), invalidGroups); + + // Avoid running as part of Suite1 unintentionally + if (groups.isEmpty()) { + // OK, Suite1 + } + else if (groups.size() == 1 && KNOWN_DUPLICATE_GROUPS.contains(getOnlyElement(groups))) { + // TODO some of these tests are run in Suite1 and some other suites, probably not intentionally + } + else if (groups.equals(Set.of(CONFIGURED_FEATURES)) && test.name().equals("io.trino.tests.product.TestConfiguredFeatures.selectConfiguredConnectors")) { + // This is a special test, self-test of CONFIGURED_FEATURES + } + else { + check( + errors::add, + !intersection(groups, SUITE1_EXCLUSIONS).isEmpty(), + "The test should probably have %s (or any other from %s), besides %s, to avoid being run more times than desired: %s", + PROFILE_SPECIFIC_TESTS, + SUITE1_EXCLUSIONS, + groups, + test.name()); + } + }); + + if (!errors.isEmpty()) { + fail("%s Errors:".formatted(errors.size()) + + errors.stream() + .map(message -> "\n\t- " + message) + .collect(joining())); + } + + // Self-test of this test. If we found too few tests, it might be because our assumptions + // on how product tests look like are no longer valid. + assertThat(testMethods).hasValueGreaterThan(500); + } + + private Stream productTests() + throws Exception + { + return Stream.concat( + javaProductTests(), + conventionBasedProductTests()); + } + + private Stream javaProductTests() + throws Exception + { + return ClassPath.from(getClass().getClassLoader()) + .getTopLevelClassesRecursive("io.trino.tests.product").stream() + .flatMap(classInfo -> Stream.of(classInfo.load().getMethods())) + .filter(method -> method.isAnnotationPresent(org.testng.annotations.Test.class)) + .map(method -> new ProductTestInfo( + JAVA, + "%s.%s".formatted(method.getDeclaringClass().getName(), method.getName()), + ImmutableList.copyOf(method.getAnnotation(org.testng.annotations.Test.class).groups()))); + } + + private Stream conventionBasedProductTests() + { + return Stream.of(new ConventionBasedTestFactory().createTestCases()) + .map(ConventionBasedTest.class::cast) + .map(conventionTest -> new ProductTestInfo( + CONVENTION_BASED, + conventionTest.getTestName(), + ImmutableList.copyOf(conventionTest.getTestGroups()))); + } + + private Set definedTestGroups() + { + return Stream.of(TestGroups.class.getFields()) + .map(field -> { + try { + return (String) field.get(null); + } + catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + }) + .collect(toImmutableSet()); + } + + record ProductTestInfo(Type type, String name, List groups) + { + enum Type + { + JAVA, + CONVENTION_BASED, + } + + ProductTestInfo + { + requireNonNull(type, "type is null"); + requireNonNull(name, "name is null"); + groups = ImmutableList.copyOf(requireNonNull(groups, "groups is null")); + } + } + + @FormatMethod + private static void check(Consumer errorConsumer, boolean condition, String messagePattern, Object... args) + { + if (!condition) { + errorConsumer.accept(messagePattern.formatted(args)); + } + } +}