Skip to content

Commit

Permalink
Prevent unwanted product test duplicated runs
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
findepi committed Jan 12, 2024
1 parent 0e0dfc8 commit 81771c3
Show file tree
Hide file tree
Showing 9 changed files with 358 additions and 79 deletions.
Original file line number Diff line number Diff line change
@@ -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<String> 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);
}
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -44,20 +32,7 @@ public List<SuiteTestRun> 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());
}
}
24 changes: 24 additions & 0 deletions testing/trino-product-tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,12 @@
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>com.microsoft.sqlserver</groupId>
<artifactId>mssql-jdbc</artifactId>
Expand Down Expand Up @@ -318,6 +324,24 @@
<artifactId>snappy-java</artifactId>
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>junit-extensions</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<repositories>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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;
Expand Down Expand Up @@ -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 -> {
Expand Down Expand Up @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand Down Expand Up @@ -259,7 +260,7 @@ private List<QueryAssert.Row> 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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand Down Expand Up @@ -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";
Expand All @@ -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(
Expand All @@ -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);
Expand Down Expand Up @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand Down Expand Up @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand All @@ -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 -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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);
Expand Down
Loading

0 comments on commit 81771c3

Please sign in to comment.