From cf0bae8f14a5ead9de2d32d1c31b125c1babff8f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 14 Nov 2022 18:07:04 +0100 Subject: [PATCH] Eliminate duplicate random suffix generation code --- .../plugin/jdbc/TestCachingJdbcClient.java | 17 +--- plugin/trino-bigquery/pom.xml | 6 ++ plugin/trino-cassandra/pom.xml | 6 ++ .../plugin/cassandra/TestCassandraTable.java | 17 +--- plugin/trino-clickhouse/pom.xml | 6 ++ plugin/trino-druid/pom.xml | 6 ++ plugin/trino-elasticsearch/pom.xml | 13 +++ plugin/trino-hive-hadoop2/pom.xml | 6 ++ plugin/trino-mongodb/pom.xml | 6 ++ plugin/trino-oracle/pom.xml | 6 ++ .../trino/plugin/password/ldap/LdapUtil.java | 13 --- .../password/ldap/TestingOpenLdapServer.java | 6 +- .../launcher/env/EnvironmentContainers.java | 16 +--- .../trino/tests/product/TestCreateTable.java | 10 +-- .../TestDeltaLakeAlterTableCompatibility.java | 26 +++--- .../TestDeltaLakeColumnMappingMode.java | 20 ++--- ...akeDatabricksCheckpointsCompatibility.java | 32 ++++---- ...ricksCreateTableAsSelectCompatibility.java | 14 ++-- ...akeDatabricksCreateTableCompatibility.java | 16 ++-- .../TestDeltaLakeDatabricksDelete.java | 4 +- ...eltaLakeDatabricksInsertCompatibility.java | 30 +++---- ...keDatabricksPartitioningCompatibility.java | 16 ++-- .../TestDeltaLakeDatabricksUpdates.java | 6 +- .../TestDeltaLakeDropTableCompatibility.java | 4 +- .../product/deltalake/TestDeltaLakeGcs.java | 6 +- .../TestDeltaLakeTransactionLogCache.java | 4 +- ...DeltaLakeWriteDatabricksCompatibility.java | 20 ++--- .../TestHiveAndDeltaLakeCompatibility.java | 6 +- .../TestHiveAndDeltaLakeRedirect.java | 82 +++++++++---------- .../product/hive/AbstractTestHiveViews.java | 8 +- .../hive/TestAbfsSyncPartitionMetadata.java | 4 +- .../product/hive/TestCreateDropSchema.java | 16 ++-- .../product/hive/TestHiveBucketedTables.java | 4 +- .../TestHivePartitionSchemaEvolution.java | 4 +- .../hive/TestHiveRedirectionToIceberg.java | 62 +++++++------- .../hive/TestHiveSparkCompatibility.java | 20 ++--- .../product/hive/TestHiveStorageFormats.java | 6 +- .../hive/TestHiveTransactionalTable.java | 10 +-- .../product/hive/util/TemporaryHiveTable.java | 14 ---- .../product/iceberg/TestCreateDropSchema.java | 10 +-- .../iceberg/TestIcebergCreateTable.java | 6 +- ...TestIcebergFormatVersionCompatibility.java | 4 +- .../TestIcebergHiveTablesCompatibility.java | 18 ++-- .../product/iceberg/TestIcebergInsert.java | 4 +- .../product/iceberg/TestIcebergOptimize.java | 4 +- .../iceberg/TestIcebergProcedureCalls.java | 4 +- .../iceberg/TestIcebergRedirectionToHive.java | 56 ++++++------- .../iceberg/TestIcebergRenameTable.java | 6 +- .../TestIcebergSparkCompatibility.java | 70 ++++++++-------- ...estIcebergSparkDropTableCompatibility.java | 4 +- .../java/io/trino/testing/TestingNames.java | 0 51 files changed, 372 insertions(+), 382 deletions(-) rename testing/{trino-testing => trino-testing-services}/src/main/java/io/trino/testing/TestingNames.java (100%) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java index 1a2f8094a758..a280a4d2229c 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java @@ -35,7 +35,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -61,9 +60,7 @@ import static io.trino.spi.testing.InterfaceTestUtils.assertAllMethodsOverridden; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.testing.TestingConnectorSession.builder; -import static java.lang.Character.MAX_RADIX; -import static java.lang.Math.abs; -import static java.lang.Math.min; +import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.util.Collections.emptyList; import static java.util.Objects.requireNonNull; @@ -78,10 +75,6 @@ @Test(singleThreaded = true) public class TestCachingJdbcClient { - private static final SecureRandom random = new SecureRandom(); - // The suffix needs to be long enough to "prevent" collisions in practice. The length of 5 was proven not to be long enough - private static final int RANDOM_SUFFIX_LENGTH = 10; - private static final Duration FOREVER = Duration.succinctDuration(1, DAYS); private static final Duration ZERO = Duration.succinctDuration(0, MILLISECONDS); @@ -758,7 +751,7 @@ public void testConcurrentSchemaCreateAndDrop() List> futures = new ArrayList<>(); for (int i = 0; i < 5; i++) { futures.add(executor.submit(() -> { - String schemaName = "schema_" + randomSuffix(); + String schemaName = "schema_" + randomNameSuffix(); assertThat(cachingJdbcClient.getSchemaNames(session)).doesNotContain(schemaName); cachingJdbcClient.createSchema(session, schemaName); assertThat(cachingJdbcClient.getSchemaNames(session)).contains(schemaName); @@ -895,12 +888,6 @@ public void testEverythingImplemented() assertAllMethodsOverridden(JdbcClient.class, CachingJdbcClient.class); } - private static String randomSuffix() - { - String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX); - return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length())); - } - private static SingleJdbcCacheStatsAssertions assertTableNamesCache(CachingJdbcClient client) { return assertCacheStats(client, CachingJdbcCache.TABLE_NAMES_CACHE); diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 79da731187e4..6318e8266f2b 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -322,6 +322,12 @@ + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index 7735ba3e078a..4fc299d85815 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -171,6 +171,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTable.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTable.java index 30b5d374f937..9eae507e0435 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTable.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTable.java @@ -16,14 +16,11 @@ import com.google.common.collect.ImmutableList; import io.trino.testing.sql.SqlExecutor; -import java.security.SecureRandom; import java.util.List; import java.util.function.Function; import static com.google.common.base.Verify.verify; -import static java.lang.Character.MAX_RADIX; -import static java.lang.Math.abs; -import static java.lang.Math.min; +import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.lang.String.join; import static java.util.stream.Collectors.joining; @@ -31,10 +28,6 @@ public class TestCassandraTable implements AutoCloseable { - private static final SecureRandom random = new SecureRandom(); - // The suffix needs to be long enough to "prevent" collisions in practice. The length of 5 was proven not to be long enough - private static final int RANDOM_SUFFIX_LENGTH = 10; - private final SqlExecutor sqlExecutor; private final String keyspace; private final String tableName; @@ -57,7 +50,7 @@ public TestCassandraTable( { this.sqlExecutor = sqlExecutor; this.keyspace = keyspace; - this.tableName = namePrefix + randomTableSuffix(); + this.tableName = namePrefix + randomNameSuffix(); sqlExecutor.execute(format("CREATE TABLE %s.%s %s", keyspace, tableName, tableDefinition(columnDefinitions))); String columns = columnDefinitions.stream() .map(columnDefinition -> columnDefinition.name) @@ -116,12 +109,6 @@ public void close() sqlExecutor.execute("DROP TABLE " + getTableName()); } - private static String randomTableSuffix() - { - String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX); - return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length())); - } - public static ColumnDefinition partitionColumn(String name, String type) { return new ColumnDefinition(name, type, PrimaryKeyType.PARTITION); diff --git a/plugin/trino-clickhouse/pom.xml b/plugin/trino-clickhouse/pom.xml index 0d2e56369598..d6aa3c61e31f 100644 --- a/plugin/trino-clickhouse/pom.xml +++ b/plugin/trino-clickhouse/pom.xml @@ -129,6 +129,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-druid/pom.xml b/plugin/trino-druid/pom.xml index c61f9f6fb78e..838813bd1514 100644 --- a/plugin/trino-druid/pom.xml +++ b/plugin/trino-druid/pom.xml @@ -122,6 +122,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index 02b40131c34d..b6a85855e477 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -297,6 +297,19 @@ test + + io.trino + trino-testing-services + test + + + + org.openjdk.jmh + jmh-core + + + + io.trino trino-tpch diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index eb595050fc1b..9e6099fe9588 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -164,6 +164,12 @@ test + + io.trino + trino-testing-services + test + + io.airlift testing diff --git a/plugin/trino-mongodb/pom.xml b/plugin/trino-mongodb/pom.xml index d93a2adeab7a..b429a432761e 100644 --- a/plugin/trino-mongodb/pom.xml +++ b/plugin/trino-mongodb/pom.xml @@ -154,6 +154,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-oracle/pom.xml b/plugin/trino-oracle/pom.xml index 3d3b8d297419..b2ebc79975b5 100644 --- a/plugin/trino-oracle/pom.xml +++ b/plugin/trino-oracle/pom.xml @@ -148,6 +148,12 @@ test + + io.trino + trino-testing-services + test + + io.trino trino-tpch diff --git a/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/LdapUtil.java b/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/LdapUtil.java index 314bc0f2c02a..68bf253b0c07 100644 --- a/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/LdapUtil.java +++ b/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/LdapUtil.java @@ -24,22 +24,15 @@ import javax.naming.directory.DirContext; import javax.naming.directory.ModificationItem; -import java.security.SecureRandom; import java.util.Arrays; import java.util.List; import java.util.Map; -import static java.lang.Character.MAX_RADIX; -import static java.lang.Math.abs; -import static java.lang.Math.min; import static java.lang.String.format; import static java.util.Objects.requireNonNull; final class LdapUtil { - private static final SecureRandom random = new SecureRandom(); - private static final int RANDOM_SUFFIX_LENGTH = 10; - public static final String MEMBER = "member"; private LdapUtil() @@ -120,10 +113,4 @@ public static LdapObjectDefinition buildLdapUserObject(String organizationName, .setObjectClasses(Arrays.asList("person", "inetOrgPerson")) .build(); } - - public static String randomSuffix() - { - String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX); - return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length())); - } } diff --git a/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/TestingOpenLdapServer.java b/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/TestingOpenLdapServer.java index f71b20c43f86..bb94d7496ca5 100644 --- a/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/TestingOpenLdapServer.java +++ b/plugin/trino-password-authenticators/src/test/java/io/trino/plugin/password/ldap/TestingOpenLdapServer.java @@ -38,7 +38,7 @@ import static io.trino.plugin.password.ldap.LdapUtil.buildLdapGroupObject; import static io.trino.plugin.password.ldap.LdapUtil.buildLdapOrganizationObject; import static io.trino.plugin.password.ldap.LdapUtil.buildLdapUserObject; -import static io.trino.plugin.password.ldap.LdapUtil.randomSuffix; +import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -82,13 +82,13 @@ public String getLdapUrl() public DisposableSubContext createOrganization() throws NamingException { - return createDisposableSubContext(buildLdapOrganizationObject("organization_" + randomSuffix(), BASE_DISTINGUISED_NAME)); + return createDisposableSubContext(buildLdapOrganizationObject("organization_" + randomNameSuffix(), BASE_DISTINGUISED_NAME)); } public DisposableSubContext createGroup(DisposableSubContext organization) throws Exception { - return createDisposableSubContext(buildLdapGroupObject(organization.getDistinguishedName(), "group_" + randomSuffix())); + return createDisposableSubContext(buildLdapGroupObject(organization.getDistinguishedName(), "group_" + randomNameSuffix())); } public DisposableSubContext createUser(DisposableSubContext organization, String userName, String password) diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentContainers.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentContainers.java index 3eccff16e5a8..97d796f4aa6f 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentContainers.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentContainers.java @@ -16,19 +16,13 @@ import io.trino.tests.product.launcher.docker.DockerFiles.ResourceProvider; import java.nio.file.Path; -import java.security.SecureRandom; import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.Character.MAX_RADIX; -import static java.lang.Math.abs; -import static java.lang.Math.min; +import static io.trino.testing.TestingNames.randomNameSuffix; import static org.testcontainers.utility.MountableFile.forHostPath; public final class EnvironmentContainers { - private static final SecureRandom random = new SecureRandom(); - private static final int RANDOM_SUFFIX_LENGTH = 5; - public static final String TRINO = "presto"; public static final String COORDINATOR = TRINO + "-master"; public static final String WORKER = TRINO + "-worker"; @@ -56,7 +50,7 @@ public static void configureTempto(Environment.Builder builder, ResourceProvider { builder.configureContainer(TESTS, dockerContainer -> { Path path = configDir.getPath("tempto-configuration.yaml"); - String suffix = getParentDirectoryName(path) + "-" + randomSuffix(); + String suffix = getParentDirectoryName(path) + "-" + randomNameSuffix(); String temptoConfig = "/docker/presto-product-tests/conf/tempto/tempto-configuration-for-" + suffix + ".yaml"; dockerContainer .withCopyFileToContainer(forHostPath(path), temptoConfig) @@ -72,10 +66,4 @@ private static String getParentDirectoryName(Path path) checkArgument(path.getNameCount() >= 2, "Cannot determine parent directory of: %s", path); return path.getName(path.getNameCount() - 2).toString(); } - - private static String randomSuffix() - { - String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX); - return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length())); - } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestCreateTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestCreateTable.java index 0a2320f9bcca..5cb49fd8615a 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestCreateTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestCreateTable.java @@ -20,8 +20,8 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.CREATE_TABLE; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -53,8 +53,8 @@ public void shouldCreateTableAsEmptySelect() @Test(groups = CREATE_TABLE) public void shouldNotCreateTableInNonExistentSchema() { - String schemaName = "test_schema_" + randomTableSuffix(); - String table = schemaName + ".test_create_no_schema_" + randomTableSuffix(); + String schemaName = "test_schema_" + randomNameSuffix(); + String table = schemaName + ".test_create_no_schema_" + randomNameSuffix(); assertQueryFailure(() -> onTrino().executeQuery("CREATE TABLE " + table + " (a bigint)")) .hasMessageMatching("\\QQuery failed (#\\E\\S+\\Q): Schema " + schemaName + " not found"); @@ -66,8 +66,8 @@ public void shouldNotCreateTableInNonExistentSchema() @Test(groups = CREATE_TABLE) public void shouldNotCreateExternalTableInNonExistentSchema() { - String schemaName = "test_schema_" + randomTableSuffix(); - String table = schemaName + ".test_create_no_schema_" + randomTableSuffix(); + String schemaName = "test_schema_" + randomNameSuffix(); + String table = schemaName + ".test_create_no_schema_" + randomNameSuffix(); assertQueryFailure(() -> onTrino().executeQuery("CREATE TABLE " + table + " (a bigint) WITH (external_location = '/tmp')")) .hasMessageMatching("\\QQuery failed (#\\E\\S+\\Q): Schema " + schemaName + " not found"); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java index bed1ed5daeff..340926a3c5b0 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeAlterTableCompatibility.java @@ -23,6 +23,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_91; @@ -35,7 +36,6 @@ import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getColumnCommentOnTrino; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getDatabricksRuntimeVersion; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getTableCommentOnDelta; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -49,7 +49,7 @@ public class TestDeltaLakeAlterTableCompatibility @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testAddColumnWithCommentOnTrino() { - String tableName = "test_dl_add_column_with_comment_" + randomTableSuffix(); + String tableName = "test_dl_add_column_with_comment_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT) WITH (location = 's3://%s/%s')", @@ -71,7 +71,7 @@ public void testAddColumnWithCommentOnTrino() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testAddColumnUnsupportedWriterVersion() { - String tableName = "test_dl_add_column_unsupported_writer_" + randomTableSuffix(); + String tableName = "test_dl_add_column_unsupported_writer_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -95,7 +95,7 @@ public void testAddColumnUnsupportedWriterVersion() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testRenameColumn() { - String tableName = "test_dl_rename_column_" + randomTableSuffix(); + String tableName = "test_dl_rename_column_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -128,7 +128,7 @@ public void testRenameColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testRenamePartitionedColumn() { - String tableName = "test_dl_rename_partitioned_column_" + randomTableSuffix(); + String tableName = "test_dl_rename_partitioned_column_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -162,7 +162,7 @@ public void testRenamePartitionedColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCommentOnTable() { - String tableName = "test_dl_comment_table_" + randomTableSuffix(); + String tableName = "test_dl_comment_table_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT) WITH (location = 's3://%s/%s')", @@ -186,7 +186,7 @@ public void testCommentOnTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCommentOnTableUnsupportedWriterVersion() { - String tableName = "test_dl_comment_table_unsupported_writer_" + randomTableSuffix(); + String tableName = "test_dl_comment_table_unsupported_writer_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -210,7 +210,7 @@ public void testCommentOnTableUnsupportedWriterVersion() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCommentOnColumn() { - String tableName = "test_dl_comment_column_" + randomTableSuffix(); + String tableName = "test_dl_comment_column_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT) WITH (location = 's3://%s/%s')", @@ -232,7 +232,7 @@ public void testCommentOnColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCommentOnColumnUnsupportedWriterVersion() { - String tableName = "test_dl_comment_column_unsupported_writer_" + randomTableSuffix(); + String tableName = "test_dl_comment_column_unsupported_writer_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -256,7 +256,7 @@ public void testCommentOnColumnUnsupportedWriterVersion() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoAlterTablePreservesTableMetadata() { - String tableName = "test_trino_alter_table_preserves_table_metadata_" + randomTableSuffix(); + String tableName = "test_trino_alter_table_preserves_table_metadata_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -283,7 +283,7 @@ public void testTrinoAlterTablePreservesTableMetadata() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoAlterTablePreservesChangeDataFeed() { - String tableName = "test_trino_alter_table_preserves_cdf_" + randomTableSuffix(); + String tableName = "test_trino_alter_table_preserves_cdf_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format(""" @@ -313,7 +313,7 @@ public void testTrinoPreservesReaderAndWriterVersions() { // Databricks 7.3 doesn't support 'delta.minReaderVersion' and 'delta.minWriterVersion' table properties // Writing those properties to protocol entry in COMMENT and ADD COLUMN statements is fine - String tableName = "test_trino_preserves_versions_" + randomTableSuffix(); + String tableName = "test_trino_preserves_versions_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("" + @@ -347,7 +347,7 @@ public void testTrinoPreservesReaderAndWriterVersions() public void testTrinoAlterTablePreservesGeneratedColumn() { // Databricks 7.3 doesn't support generated columns - String tableName = "test_trino_alter_table_preserves_generated_column_" + randomTableSuffix(); + String tableName = "test_trino_alter_table_preserves_generated_column_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format(""" diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java index 1349cb76bc7c..712e8e9313ce 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeColumnMappingMode.java @@ -23,6 +23,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_91; @@ -30,7 +31,6 @@ import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; @@ -41,7 +41,7 @@ public class TestDeltaLakeColumnMappingMode @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testColumnMappingModeNone() { - String tableName = "test_dl_column_mapping_mode_none" + randomTableSuffix(); + String tableName = "test_dl_column_mapping_mode_none" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -82,7 +82,7 @@ public void testColumnMappingModeName() private void testColumnMappingMode(String mode) { - String tableName = "test_dl_column_mapping_mode_name_" + randomTableSuffix(); + String tableName = "test_dl_column_mapping_mode_name_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -153,7 +153,7 @@ public void testColumnMappingModeNameWithNonLowerCaseColumnName() private void testColumnMappingModeNameWithNonLowerCaseColumn(String mode) { - String tableName = "test_dl_column_mapping_mode_name_non_loewr_case_" + randomTableSuffix(); + String tableName = "test_dl_column_mapping_mode_name_non_loewr_case_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -201,7 +201,7 @@ public void testColumnMappingModeNameAddColumn() private void testColumnMappingModeAddColumn(String mode) { - String tableName = "test_dl_column_mapping_mode_add_column_" + randomTableSuffix(); + String tableName = "test_dl_column_mapping_mode_add_column_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -272,7 +272,7 @@ public void testShowStatsFromJsonForColumnMappingModeName() private void testShowStatsFromJsonForColumnMappingMode(String mode) { - String tableName = "test_dl_show_stats_json_for_column_mapping_mode_" + randomTableSuffix(); + String tableName = "test_dl_show_stats_json_for_column_mapping_mode_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -317,7 +317,7 @@ public void testShowStatsFromParquetForColumnMappingModeName() private void testShowStatsFromParquetForColumnMappingMode(String mode) { - String tableName = "test_dl_show_parquet_stats_parquet_for_column_mapping_mode_" + randomTableSuffix(); + String tableName = "test_dl_show_parquet_stats_parquet_for_column_mapping_mode_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -365,7 +365,7 @@ public void testShowStatsOnPartitionedForColumnMappingModeName() private void testShowStatsOnPartitionedForColumnMappingMode(String mode) { - String tableName = "test_dl_show_stats_partitioned_for_column_mapping_mode_" + randomTableSuffix(); + String tableName = "test_dl_show_stats_partitioned_for_column_mapping_mode_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -410,7 +410,7 @@ public void testUnsupportedOperationsColumnMappingModeName() private void testUnsupportedOperationsColumnMappingModeName(String mode) { - String tableName = "test_dl_unsupported_column_mapping_mode_" + randomTableSuffix(); + String tableName = "test_dl_unsupported_column_mapping_mode_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -459,7 +459,7 @@ public void testSpecialCharacterColumnNamesWithColumnMappingModeName() private void testSpecialCharacterColumnNamesWithColumnMappingMode(String mode) { - String tableName = "test_dl_special_character_column_mapping_mode_" + randomTableSuffix(); + String tableName = "test_dl_special_character_column_mapping_mode_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCheckpointsCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCheckpointsCompatibility.java index 0880e323aa53..06b220d36bbd 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCheckpointsCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCheckpointsCompatibility.java @@ -33,6 +33,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; @@ -43,7 +44,6 @@ import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getDatabricksRuntimeVersion; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -72,7 +72,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCanReadTrinoCheckpoint() { - String tableName = "test_dl_checkpoints_compat_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; // using mixed case column names for extend test coverage @@ -133,7 +133,7 @@ public void testDatabricksCanReadTrinoCheckpoint() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoUsesCheckpointInterval() { - String tableName = "test_dl_checkpoints_compat_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format( @@ -193,7 +193,7 @@ public void testTrinoUsesCheckpointInterval() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksUsesCheckpointInterval() { - String tableName = "test_dl_checkpoints_compat_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (a_number bigint, a_string varchar) " + @@ -265,7 +265,7 @@ public void testDatabricksUsesCheckpointInterval() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoCheckpointMinMaxStatisticsForRowType() { - String tableName = "test_dl_checkpoints_row_compat_min_max_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_row_compat_min_max_trino_" + randomNameSuffix(); testCheckpointMinMaxStatisticsForRowType(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName); } @@ -273,7 +273,7 @@ public void testTrinoCheckpointMinMaxStatisticsForRowType() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCheckpointMinMaxStatisticsForRowType() { - String tableName = "test_dl_checkpoints_row_compat_min_max_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_row_compat_min_max_databricks_" + randomNameSuffix(); testCheckpointMinMaxStatisticsForRowType(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName); } @@ -332,7 +332,7 @@ private void testCheckpointMinMaxStatisticsForRowType(Consumer sqlExecut @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoCheckpointNullStatisticsForRowType() { - String tableName = "test_dl_checkpoints_row_compat_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_row_compat_trino_" + randomNameSuffix(); testCheckpointNullStatisticsForRowType(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName); } @@ -340,7 +340,7 @@ public void testTrinoCheckpointNullStatisticsForRowType() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCheckpointNullStatisticsForRowType() { - String tableName = "test_dl_checkpoints_row_compat_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_row_compat_databricks_" + randomNameSuffix(); testCheckpointNullStatisticsForRowType(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName); } @@ -397,7 +397,7 @@ private void testCheckpointNullStatisticsForRowType(Consumer sqlExecutor @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoWriteStatsAsJsonDisabled() { - String tableName = "test_dl_checkpoints_write_stats_as_json_disabled_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_json_disabled_trino_" + randomNameSuffix(); testWriteStatsAsJsonDisabled(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName); } @@ -405,7 +405,7 @@ public void testTrinoWriteStatsAsJsonDisabled() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksWriteStatsAsJsonDisabled() { - String tableName = "test_dl_checkpoints_write_stats_as_json_disabled_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_json_disabled_databricks_" + randomNameSuffix(); testWriteStatsAsJsonDisabled(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName); } @@ -440,7 +440,7 @@ private void testWriteStatsAsJsonDisabled(Consumer sqlExecutor, String t @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoWriteStatsAsStructDisabled() { - String tableName = "test_dl_checkpoints_write_stats_as_struct_disabled_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_struct_disabled_trino_" + randomNameSuffix(); testWriteStatsAsStructDisabled(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName); } @@ -448,7 +448,7 @@ public void testTrinoWriteStatsAsStructDisabled() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksWriteStatsAsStructDisabled() { - String tableName = "test_dl_checkpoints_write_stats_as_struct_disabled_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_struct_disabled_databricks_" + randomNameSuffix(); testWriteStatsAsStructDisabled(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName); } @@ -484,7 +484,7 @@ private void testWriteStatsAsStructDisabled(Consumer sqlExecutor, String @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoWriteStatsAsJsonEnabled(String type, String inputValue, Double nullsFraction, Object statsValue) { - String tableName = "test_dl_checkpoints_write_stats_as_json_enabled_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_json_enabled_trino_" + randomNameSuffix(); testWriteStatsAsJsonEnabled(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName, type, inputValue, nullsFraction, statsValue); } @@ -492,7 +492,7 @@ public void testTrinoWriteStatsAsJsonEnabled(String type, String inputValue, Dou @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksWriteStatsAsJsonEnabled(String type, String inputValue, Double nullsFraction, Object statsValue) { - String tableName = "test_dl_checkpoints_write_stats_as_json_enabled_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_json_enabled_databricks_" + randomNameSuffix(); testWriteStatsAsJsonEnabled(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName, type, inputValue, nullsFraction, statsValue); } @@ -588,7 +588,7 @@ public Object[][] testDeltaCheckpointWriteStatsAsJson() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoWriteStatsAsStructEnabled() { - String tableName = "test_dl_checkpoints_write_stats_as_struct_enabled_trino_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_struct_enabled_trino_" + randomNameSuffix(); testWriteStatsAsStructEnabled(sql -> onTrino().executeQuery(sql), tableName, "delta.default." + tableName); } @@ -596,7 +596,7 @@ public void testTrinoWriteStatsAsStructEnabled() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksWriteStatsAsStructEnabled() { - String tableName = "test_dl_checkpoints_write_stats_as_struct_enabled_databricks_" + randomTableSuffix(); + String tableName = "test_dl_checkpoints_write_stats_as_struct_enabled_databricks_" + randomNameSuffix(); testWriteStatsAsStructEnabled(sql -> onDelta().executeQuery(sql), tableName, "default." + tableName); } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableAsSelectCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableAsSelectCompatibility.java index 24ed53f9776b..8fe6dad1cb3f 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableAsSelectCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableAsSelectCompatibility.java @@ -35,13 +35,13 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.TransactionLogAssertions.assertLastEntryIsCheckpointed; import static io.trino.tests.product.deltalake.TransactionLogAssertions.assertTransactionLogVersion; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -68,7 +68,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testPrestoTypesWithDatabricks() { - String tableName = "test_dl_ctas_" + randomTableSuffix(); + String tableName = "test_dl_ctas_" + randomNameSuffix(); try { assertThat(onTrino().executeQuery("CREATE TABLE delta.default.\"" + tableName + "\" " + @@ -98,7 +98,7 @@ public void testPrestoTypesWithDatabricks() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testPrestoTimestampsWithDatabricks() { - String tableName = "test_dl_ctas_timestamps_" + randomTableSuffix(); + String tableName = "test_dl_ctas_timestamps_" + randomNameSuffix(); try { assertThat(onTrino().executeQuery("CREATE TABLE delta.default.\"" + tableName + "\" " + @@ -126,7 +126,7 @@ public void testPrestoTimestampsWithDatabricks() public void testPrestoCacheInvalidatedOnCreateTable() throws URISyntaxException, IOException { - String tableName = "test_dl_ctas_caching_" + randomTableSuffix(); + String tableName = "test_dl_ctas_caching_" + randomNameSuffix(); try { assertThat(onTrino().executeQuery("CREATE TABLE delta.default.\"" + tableName + "\" " + @@ -175,7 +175,7 @@ public void testPrestoCacheInvalidatedOnCreateTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCreateFromTrinoWithDefaultPartitionValues() { - String tableName = "test_create_partitioned_table_default_as_" + randomTableSuffix(); + String tableName = "test_create_partitioned_table_default_as_" + randomNameSuffix(); try { assertThat(onTrino().executeQuery( @@ -205,7 +205,7 @@ public void testCreateFromTrinoWithDefaultPartitionValues() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testReplaceTableWithSchemaChange() { - String tableName = "test_replace_table_with_schema_change_" + randomTableSuffix(); + String tableName = "test_replace_table_with_schema_change_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE delta.default." + tableName + " (ts VARCHAR) " + "with (location = 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "', checkpoint_interval = 10)"); @@ -230,7 +230,7 @@ public void testReplaceTableWithSchemaChange() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testReplaceTableWithSchemaChangeOnCheckpoint() { - String tableName = "test_replace_table_with_schema_change_" + randomTableSuffix(); + String tableName = "test_replace_table_with_schema_change_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE delta.default." + tableName + " (ts VARCHAR) " + "with (location = 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "', checkpoint_interval = 10)"); try { diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableCompatibility.java index de29e208f66f..89ae8944fbfb 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCreateTableCompatibility.java @@ -23,6 +23,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_104_RUNTIME_VERSION; @@ -32,7 +33,6 @@ import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getColumnCommentOnTrino; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getDatabricksRuntimeVersion; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getTableCommentOnDelta; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -54,7 +54,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCanReadInitialCreateTable() { - String tableName = "test_dl_create_table_compat_" + randomTableSuffix(); + String tableName = "test_dl_create_table_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (integer int, string varchar, timetz timestamp with time zone) with (location = 's3://%s/%s')", @@ -94,7 +94,7 @@ public void testDatabricksCanReadInitialCreateTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCanReadInitialCreatePartitionedTable() { - String tableName = "test_dl_create_table_compat_" + randomTableSuffix(); + String tableName = "test_dl_create_table_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery( @@ -137,7 +137,7 @@ public void testDatabricksCanReadInitialCreatePartitionedTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCanReadInitialCreateTableAs() { - String tableName = "test_dl_create_table_as_compat_" + randomTableSuffix(); + String tableName = "test_dl_create_table_as_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (integer, string, timetz) with (location = 's3://%s/%s') AS " + @@ -182,7 +182,7 @@ public void testDatabricksCanReadInitialCreateTableAs() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksCanReadInitialCreatePartitionedTableAs() { - String tableName = "test_dl_create_table_compat_" + randomTableSuffix(); + String tableName = "test_dl_create_table_compat_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (integer, string, timetz) with (location = 's3://%s/%s', partitioned_by = ARRAY['string']) AS " + @@ -245,7 +245,7 @@ private void testInsert(String tableName, List existingRows) @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCreateTableWithTableComment() { - String tableName = "test_dl_create_table_comment_" + randomTableSuffix(); + String tableName = "test_dl_create_table_comment_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT) COMMENT 'test comment' WITH (location = 's3://%s/%s')", @@ -268,7 +268,7 @@ public void testCreateTableWithTableComment() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCreateTableWithColumnCommentOnTrino() { - String tableName = "test_dl_create_column_comment_" + randomTableSuffix(); + String tableName = "test_dl_create_column_comment_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT COMMENT 'test comment') WITH (location = 's3://%s/%s')", @@ -294,7 +294,7 @@ public void testCreateTableWithColumnCommentOnTrino() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCreateTableWithColumnCommentOnDelta() { - String tableName = "test_dl_create_column_comment_" + randomTableSuffix(); + String tableName = "test_dl_create_column_comment_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onDelta().executeQuery(format("CREATE TABLE default.%s (col INT COMMENT 'test comment') USING DELTA LOCATION 's3://%s/%s'", diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksDelete.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksDelete.java index 70b640c07af2..5340833d1fc1 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksDelete.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksDelete.java @@ -19,12 +19,12 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; @@ -35,7 +35,7 @@ public class TestDeltaLakeDatabricksDelete @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeleteOnAppendOnlyTableFails() { - String tableName = "test_delete_on_append_only_table_fails_" + randomTableSuffix(); + String tableName = "test_delete_on_append_only_table_fails_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + " (a INT, b INT)" + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksInsertCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksInsertCompatibility.java index ba32b5adeae0..94032a23ea2f 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksInsertCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksInsertCompatibility.java @@ -28,6 +28,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; @@ -36,7 +37,6 @@ import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.getDatabricksRuntimeVersion; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.util.Arrays.asList; @@ -57,7 +57,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testInsertCompatibility() { - String tableName = "test_dl_insert_" + randomTableSuffix(); + String tableName = "test_dl_insert_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -93,7 +93,7 @@ public void testInsertCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testPartitionedInsertCompatibility() { - String tableName = "test_dl_partitioned_insert_" + randomTableSuffix(); + String tableName = "test_dl_partitioned_insert_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -131,7 +131,7 @@ public void testPartitionedInsertCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoPartitionedDifferentOrderInsertCompatibility() { - String tableName = "test_dl_trino_partitioned_different_order_insert_" + randomTableSuffix(); + String tableName = "test_dl_trino_partitioned_different_order_insert_" + randomNameSuffix(); onTrino().executeQuery("" + "CREATE TABLE delta.default." + tableName + @@ -158,7 +158,7 @@ public void testTrinoPartitionedDifferentOrderInsertCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaPartitionedDifferentOrderInsertCompatibility() { - String tableName = "test_dl_delta_partitioned_different_order_insert_" + randomTableSuffix(); + String tableName = "test_dl_delta_partitioned_different_order_insert_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -185,7 +185,7 @@ public void testDeltaPartitionedDifferentOrderInsertCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testInsertNonLowercaseColumnsCompatibility() { - String tableName = "test_dl_insert_nonlowercase_columns_" + randomTableSuffix(); + String tableName = "test_dl_insert_nonlowercase_columns_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -223,7 +223,7 @@ public void testInsertNonLowercaseColumnsCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testInsertNestedNonLowercaseColumnsCompatibility() { - String tableName = "test_dl_insert_nested_nonlowercase_columns_" + randomTableSuffix(); + String tableName = "test_dl_insert_nested_nonlowercase_columns_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -263,7 +263,7 @@ public void testInsertNestedNonLowercaseColumnsCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testPartitionedInsertNonLowercaseColumnsCompatibility() { - String tableName = "test_dl_partitioned_insert_nonlowercase_columns" + randomTableSuffix(); + String tableName = "test_dl_partitioned_insert_nonlowercase_columns" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + @@ -302,7 +302,7 @@ public void testPartitionedInsertNonLowercaseColumnsCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeleteCompatibility() { - String tableName = "test_delete_compatibility_" + randomTableSuffix(); + String tableName = "test_delete_compatibility_" + randomNameSuffix(); onDelta().executeQuery("CREATE TABLE default." + tableName + " (a int, b int)" + " USING DELTA LOCATION 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "'"); @@ -331,7 +331,7 @@ public void testDeleteCompatibility() public void testCheckConstraintsCompatibility() { // CHECK constraint is not supported by Trino - String tableName = "test_check_constraint_not_supported_" + randomTableSuffix(); + String tableName = "test_check_constraint_not_supported_" + randomNameSuffix(); onDelta().executeQuery("CREATE TABLE default." + tableName + "(id INT, a_number INT) " + @@ -387,7 +387,7 @@ private void testCompression(boolean optimizedWriter, String compressionCodec) String tableName = "test_compression" + (optimizedWriter ? "_optimized" : "") + "_" + compressionCodec + - "_" + randomTableSuffix(); + "_" + randomNameSuffix(); String trinoTableName = "delta.default." + tableName; String location = "s3://" + bucketName + "/databricks-compatibility-test-" + tableName; @@ -458,7 +458,7 @@ public Object[][] compressionCodecs() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testWritesToTableWithCheckConstraintFails() { - String tableName = "test_writes_into_table_with_check_constraint_" + randomTableSuffix(); + String tableName = "test_writes_into_table_with_check_constraint_" + randomNameSuffix(); try { onDelta().executeQuery("CREATE TABLE default." + tableName + " (a INT, b INT) " + "USING DELTA " + @@ -484,7 +484,7 @@ public void testWritesToTableWithCheckConstraintFails() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testMetadataOperationsRetainCheckConstraints() { - String tableName = "test_metadata_operations_retain_check_constraints_" + randomTableSuffix(); + String tableName = "test_metadata_operations_retain_check_constraints_" + randomNameSuffix(); try { onDelta().executeQuery("CREATE TABLE default." + tableName + " (a INT, b INT) " + "USING DELTA " + @@ -507,7 +507,7 @@ public void testMetadataOperationsRetainCheckConstraints() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testWritesToTableWithGeneratedColumnFails() { - String tableName = "test_writes_into_table_with_generated_column_" + randomTableSuffix(); + String tableName = "test_writes_into_table_with_generated_column_" + randomNameSuffix(); try { onDelta().executeQuery("CREATE TABLE default." + tableName + " (a INT, b BOOLEAN GENERATED ALWAYS AS (CAST(true AS BOOLEAN))) " + "USING DELTA " + @@ -538,7 +538,7 @@ public void testWritesToTableWithGeneratedColumnFails() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testWritesToTableWithCDFFails() { - String tableName = "test_writes_into_table_with_CDF_" + randomTableSuffix(); + String tableName = "test_writes_into_table_with_CDF_" + randomNameSuffix(); try { onDelta().executeQuery("CREATE TABLE default." + tableName + " (a INT, b INT) " + "USING DELTA " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksPartitioningCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksPartitioningCompatibility.java index 2915761cb315..6462d7cba82e 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksPartitioningCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksPartitioningCompatibility.java @@ -20,11 +20,11 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -42,7 +42,7 @@ public void testDatabricksCanReadFromCtasTableCreatedByTrinoWithSpecialCharacter private void testDatabricksCanReadFromCtasTableCreatedByTrinoWithSpecialCharactersInPartitioningColumnWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -93,7 +93,7 @@ public void testTrinoCanReadFromCtasTableCreatedByDatabricksWithSpecialCharacter private void testTrinoCanReadFromCtasTableCreatedByDatabricksWithSpecialCharactersInPartitioningColumnWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -147,7 +147,7 @@ public void testDatabricksCanReadTableCreatedByTrinoWithSpecialCharactersInParti private void testDatabricksCanReadTableCreatedByTrinoWithSpecialCharactersInPartitioningColumnWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -200,7 +200,7 @@ public void testTrinoCanReadTableCreatedByDatabricksWithSpecialCharactersInParti private void testTrinoCanReadTableCreatedByDatabricksWithSpecialCharactersInPartitioningColumnWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -256,7 +256,7 @@ public void testDatabricksCanReadFromTableUpdatedByTrino() private void testDatabricksCanReadFromTableUpdatedByTrinoWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -309,7 +309,7 @@ public void testTrinoCanReadFromTableUpdatedByDatabricks() private void testTrinoCanReadFromTableUpdatedByDatabricksWithCpIntervalSet(int interval) { - String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomTableSuffix()); + String tableName = format("test_dl_create_table_partition_by_special_char_with_%d_partitions_%s", interval, randomNameSuffix()); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of( @@ -359,7 +359,7 @@ private void testTrinoCanReadFromTableUpdatedByDatabricksWithCpIntervalSet(int i @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoCanReadFromTablePartitionChangedByDatabricks() { - String tableName = "test_dl_create_table_partition_changed_by_databricks_" + randomTableSuffix(); + String tableName = "test_dl_create_table_partition_changed_by_databricks_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; ImmutableList expected = ImmutableList.of(row(1, "part")); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksUpdates.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksUpdates.java index 168af5d72e03..0e1b0e19797c 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksUpdates.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksUpdates.java @@ -24,12 +24,12 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -41,7 +41,7 @@ public class TestDeltaLakeDatabricksUpdates @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testUpdateOnAppendOnlyTableFails() { - String tableName = "test_update_on_append_only_table_fails_" + randomTableSuffix(); + String tableName = "test_update_on_append_only_table_fails_" + randomNameSuffix(); onDelta().executeQuery("" + "CREATE TABLE default." + tableName + " (a INT, b INT)" + @@ -64,7 +64,7 @@ public void testUpdateOnAppendOnlyTableFails() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testUpdatesFromDatabricks() { - String tableName = "test_updates_" + randomTableSuffix(); + String tableName = "test_updates_" + randomNameSuffix(); assertThat(onTrino().executeQuery("CREATE TABLE delta.default.\"" + tableName + "\" " + "(id, value) " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDropTableCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDropTableCompatibility.java index 23243b07f2e3..26bbd40404d8 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDropTableCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDropTableCompatibility.java @@ -25,6 +25,7 @@ import java.util.Optional; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; @@ -32,7 +33,6 @@ import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; import static io.trino.tests.product.hive.Engine.DELTA; import static io.trino.tests.product.hive.Engine.TRINO; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -85,7 +85,7 @@ public void testCreateManagedTableInDeltaDropTableInTrino() private void testDropTableAccuracy(Engine creator, Engine dropper, boolean explicitLocation) { - String schemaName = "test_schema_with_location_" + randomTableSuffix(); + String schemaName = "test_schema_with_location_" + randomNameSuffix(); String schemaLocation = format("s3://%s/databricks-compatibility-test-%s", bucketName, schemaName); String tableName = explicitLocation ? "test_external_table" : "test_managed_table"; Optional tableLocation = explicitLocation diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeGcs.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeGcs.java index a902c4a7f4f2..53e4ac1a27ae 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeGcs.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeGcs.java @@ -20,9 +20,9 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_GCS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -36,7 +36,7 @@ public class TestDeltaLakeGcs @Test(groups = {DELTA_LAKE_GCS, PROFILE_SPECIFIC_TESTS}) public void testCreateAndSelectNationTable() { - String tableName = "nation_" + randomTableSuffix(); + String tableName = "nation_" + randomNameSuffix(); onTrino().executeQuery(format( "CREATE TABLE delta.default.%1$s WITH (location = '%2$s/%1$s') AS SELECT * FROM tpch.tiny.nation", tableName, @@ -49,7 +49,7 @@ public void testCreateAndSelectNationTable() @Test(groups = {DELTA_LAKE_GCS, PROFILE_SPECIFIC_TESTS}) public void testBasicWriteOperations() { - String tableName = "table_write_operations_" + randomTableSuffix(); + String tableName = "table_write_operations_" + randomNameSuffix(); onTrino().executeQuery(format( "CREATE TABLE delta.default.%1$s (a_bigint bigint, a_varchar varchar) WITH (location = '%2$s/%1$s')", tableName, diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeTransactionLogCache.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeTransactionLogCache.java index 8a439bc5731c..e8afaf5ceb20 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeTransactionLogCache.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeTransactionLogCache.java @@ -30,12 +30,12 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -60,7 +60,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testAllDataFilesAreLoadedWhenTransactionLogFileAfterTheCachedTableVersionIsMissing() { - String tableName = "test_dl_cached_table_files_accuracy_" + randomTableSuffix(); + String tableName = "test_dl_cached_table_files_accuracy_" + randomNameSuffix(); String tableDirectory = "databricks-compatibility-test-" + tableName; onTrino().executeQuery(format("CREATE TABLE delta.default.%s (col INT) WITH (location = 's3://%s/%s')", diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeWriteDatabricksCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeWriteDatabricksCompatibility.java index 5c8432ec4784..f5e2e3ec7b79 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeWriteDatabricksCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeWriteDatabricksCompatibility.java @@ -31,12 +31,12 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_EXCLUDE_73; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -64,7 +64,7 @@ public void setup() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testUpdateCompatibility() { - String tableName = "test_update_compatibility_" + randomTableSuffix(); + String tableName = "test_update_compatibility_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (a int, b int, c int) USING DELTA LOCATION '%2$s%1$s'", @@ -96,7 +96,7 @@ public void testUpdateCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeleteCompatibility() { - String tableName = "test_delete_compatibility_" + randomTableSuffix(); + String tableName = "test_delete_compatibility_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (a int, b int) USING DELTA LOCATION '%2$s%1$s'", @@ -126,7 +126,7 @@ public void testDeleteCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeleteOnPartitionedTableCompatibility() { - String tableName = "test_delete_on_partitioned_table_compatibility_" + randomTableSuffix(); + String tableName = "test_delete_on_partitioned_table_compatibility_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (a int, b int) USING DELTA LOCATION '%2$s%1$s' PARTITIONED BY (b)", @@ -156,7 +156,7 @@ public void testDeleteOnPartitionedTableCompatibility() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeleteOnPartitionKeyCompatibility() { - String tableName = "test_delete_on_partitioned_table_compatibility_" + randomTableSuffix(); + String tableName = "test_delete_on_partitioned_table_compatibility_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (a int, b int) USING DELTA LOCATION '%2$s%1$s' PARTITIONED BY (b)", @@ -239,7 +239,7 @@ public void testCaseDeleteEntirePartition(String partitionColumn) @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testTrinoRespectsDatabricksSettingNonNullableColumn() { - String tableName = "test_databricks_table_with_nonnullable_columns_" + randomTableSuffix(); + String tableName = "test_databricks_table_with_nonnullable_columns_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (non_nullable_col INT NOT NULL, nullable_col INT) USING DELTA LOCATION '%2$s%1$s'", @@ -267,7 +267,7 @@ public void testTrinoRespectsDatabricksSettingNonNullableColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDatabricksRespectsTrinoSettingNonNullableColumn() { - String tableName = "test_trino_table_with_nonnullable_columns_" + randomTableSuffix(); + String tableName = "test_trino_table_with_nonnullable_columns_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE delta.default.\"" + tableName + "\" " + "(non_nullable_col INT NOT NULL, nullable_col INT) " + @@ -293,7 +293,7 @@ public void testDatabricksRespectsTrinoSettingNonNullableColumn() @Test(groups = {DELTA_LAKE_EXCLUDE_73, PROFILE_SPECIFIC_TESTS}) public void testInsertingIntoDatabricksTableWithAddedNotNullConstraint() { - String tableName = "test_databricks_table_altered_after_initial_write_" + randomTableSuffix(); + String tableName = "test_databricks_table_altered_after_initial_write_" + randomNameSuffix(); onDelta().executeQuery(format( "CREATE TABLE default.%1$s (non_nullable_col INT, nullable_col INT) USING DELTA LOCATION '%2$s%1$s'", @@ -340,7 +340,7 @@ public void testDatabricksVacuumRemoveChangeDataFeedFiles() private void testVacuumRemoveChangeDataFeedFiles(Consumer vacuumExecutor) { - String tableName = "test_vacuum_ignore_cdf_" + randomTableSuffix(); + String tableName = "test_vacuum_ignore_cdf_" + randomNameSuffix(); String directoryName = "databricks-compatibility-test-" + tableName; String changeDataPrefix = directoryName + "/_change_data"; @@ -431,7 +431,7 @@ private class CaseTestTable CaseTestTable(String namePrefix, String partitionColumnName, Collection rows) { - this.name = namePrefix + "_" + randomTableSuffix(); + this.name = namePrefix + "_" + randomNameSuffix(); this.columns = List.of("lower", "UPPER", partitionColumnName); this.rows = List.copyOf(rows); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeCompatibility.java index f4a34679b4ba..673f3a2051f8 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeCompatibility.java @@ -18,12 +18,12 @@ import org.testng.annotations.Test; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -35,10 +35,10 @@ public class TestHiveAndDeltaLakeCompatibility public void testInformationSchemaColumnsOnPresenceOfHiveView() { // use dedicated schema so we control the number and shape of tables - String schemaName = "test_redirect_to_delta_information_schema_columns_schema_" + randomTableSuffix(); + String schemaName = "test_redirect_to_delta_information_schema_columns_schema_" + randomNameSuffix(); onTrino().executeQuery("CREATE SCHEMA IF NOT EXISTS hive." + schemaName); - String hiveViewName = "delta_schema_columns_hive_view_" + randomTableSuffix(); + String hiveViewName = "delta_schema_columns_hive_view_" + randomNameSuffix(); String hiveViewQualifiedName = format("hive.%s.%s", schemaName, hiveViewName); onTrino().executeQuery("CREATE VIEW " + hiveViewQualifiedName + " AS SELECT 1 AS col_one"); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeRedirect.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeRedirect.java index 71a89365c7ee..8985c3fe4be4 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeRedirect.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestHiveAndDeltaLakeRedirect.java @@ -35,12 +35,12 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tempto.query.QueryExecutor.param; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.DELTA_LAKE_OSS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_MATCH; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onDelta; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -53,7 +53,7 @@ public class TestHiveAndDeltaLakeRedirect @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaRedirect() { - String tableName = "test_redirect_to_delta_" + randomTableSuffix(); + String tableName = "test_redirect_to_delta_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, false)); @@ -73,8 +73,8 @@ public void testHiveToDeltaRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaNonDefaultSchemaRedirect() { - String schemaName = "test_extraordinary_" + randomTableSuffix(); - String tableName = "test_redirect_to_delta_non_default_schema_" + randomTableSuffix(); + String schemaName = "test_extraordinary_" + randomNameSuffix(); + String tableName = "test_redirect_to_delta_non_default_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); onDelta().executeQuery(format("CREATE SCHEMA IF NOT EXISTS %s LOCATION \"%s\"", schemaName, schemaLocation)); @@ -96,7 +96,7 @@ public void testHiveToDeltaNonDefaultSchemaRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToNonexistentDeltaCatalogRedirectFailure() { - String tableName = "test_redirect_to_nonexistent_delta_" + randomTableSuffix(); + String tableName = "test_redirect_to_nonexistent_delta_" + randomNameSuffix(); try { onDelta().executeQuery(createTableOnDelta(tableName, false)); @@ -116,7 +116,7 @@ public void testHiveToNonexistentDeltaCatalogRedirectFailure() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaRedirectWithDefaultSchemaInSession() { - String tableName = "test_redirect_to_delta_with_use_" + randomTableSuffix(); + String tableName = "test_redirect_to_delta_with_use_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, false)); @@ -138,7 +138,7 @@ public void testHiveToDeltaRedirectWithDefaultSchemaInSession() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToUnpartitionedDeltaPartitionsRedirectFailure() { - String tableName = "test_delta_lake_unpartitioned_table_" + randomTableSuffix(); + String tableName = "test_delta_lake_unpartitioned_table_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, false)); @@ -156,7 +156,7 @@ public void testHiveToUnpartitionedDeltaPartitionsRedirectFailure() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToPartitionedDeltaPartitionsRedirectFailure() { - String tableName = "test_delta_lake_partitioned_table_" + randomTableSuffix(); + String tableName = "test_delta_lake_partitioned_table_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -174,7 +174,7 @@ public void testHiveToPartitionedDeltaPartitionsRedirectFailure() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveRedirect() { - String tableName = "test_redirect_to_hive_" + randomTableSuffix(); + String tableName = "test_redirect_to_hive_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, false)); @@ -199,9 +199,9 @@ public void testDeltaToHiveRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveNonDefaultSchemaRedirect() { - String schemaName = "test_extraordinary" + randomTableSuffix(); + String schemaName = "test_extraordinary" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); - String tableName = "test_redirect_to_hive_non_default_schema_" + randomTableSuffix(); + String tableName = "test_redirect_to_hive_non_default_schema_" + randomNameSuffix(); onTrino().executeQuery(format("CREATE SCHEMA IF NOT EXISTS hive.%s WITH (location='%s')", schemaName, schemaLocation)); @@ -229,7 +229,7 @@ public void testDeltaToHiveNonDefaultSchemaRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToNonexistentHiveCatalogRedirectFailure() { - String tableName = "test_redirect_to_nonexistent_hive_" + randomTableSuffix(); + String tableName = "test_redirect_to_nonexistent_hive_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, false)); @@ -249,7 +249,7 @@ public void testDeltaToNonexistentHiveCatalogRedirectFailure() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveRedirectWithDefaultSchemaInSession() { - String tableName = "test_redirect_to_hive_with_use_" + randomTableSuffix(); + String tableName = "test_redirect_to_hive_with_use_" + randomNameSuffix(); onTrino().executeQuery("USE hive.default"); @@ -276,7 +276,7 @@ public void testDeltaToHiveRedirectWithDefaultSchemaInSession() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToPartitionedHivePartitionsRedirect() { - String tableName = "test_hive_partitioned_table_" + randomTableSuffix(); + String tableName = "test_hive_partitioned_table_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); @@ -301,7 +301,7 @@ public void testDeltaToPartitionedHivePartitionsRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToUnpartitionedHivePartitionsRedirectFailure() { - String tableName = "test_hive_unpartitioned_table_" + randomTableSuffix(); + String tableName = "test_hive_unpartitioned_table_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, false)); @@ -318,7 +318,7 @@ public void testDeltaToUnpartitionedHivePartitionsRedirectFailure() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveInsert() { - String tableName = "test_hive_insert_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_insert_by_delta_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); @@ -348,7 +348,7 @@ public void testDeltaToHiveInsert() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaInsert() { - String tableName = "test_delta_insert_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_insert_by_hive_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -367,7 +367,7 @@ public void testHiveToDeltaInsert() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveDescribe() { - String tableName = "test_hive_describe_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_describe_by_delta_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); @@ -388,7 +388,7 @@ public void testDeltaToHiveDescribe() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaDescribe() { - String tableName = "test_delta_describe_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_describe_by_hive_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -410,7 +410,7 @@ public void testHiveToDeltaDescribe() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveShowCreateTable() { - String tableName = "test_hive_show_create_table_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_show_create_table_by_delta_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); @@ -427,7 +427,7 @@ public void testDeltaToHiveShowCreateTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaShowCreateTable() { - String tableName = "test_delta_show_create_table_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_show_create_table_by_hive_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -444,7 +444,7 @@ public void testHiveToDeltaShowCreateTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveAlterTable() { - String tableName = "test_hive_alter_table_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_alter_table_by_delta_" + randomNameSuffix(); // TODO set the partitioning for the table to `true` after the fix of https://github.com/trinodb/trino/issues/11826 onTrino().executeQuery(createTableInHiveConnector("default", tableName, false)); String newTableName = tableName + "_new"; @@ -470,7 +470,7 @@ public void testDeltaToHiveAlterTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaAlterTable() { - String tableName = "test_delta_alter_table_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_alter_table_by_hive_" + randomNameSuffix(); String newTableName = tableName + "_new"; onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -489,7 +489,7 @@ public void testHiveToDeltaAlterTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveCommentTable() { - String tableName = "test_hive_comment_table_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_comment_table_by_delta_" + randomNameSuffix(); onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); try { @@ -510,7 +510,7 @@ public void testDeltaToHiveCommentTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaCommentTable() { - String tableName = "test_delta_comment_table_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_comment_table_by_hive_" + randomNameSuffix(); onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -532,7 +532,7 @@ public void testHiveToDeltaCommentTable() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testDeltaToHiveCommentColumn() { - String tableName = "test_hive_comment_column_by_delta_" + randomTableSuffix(); + String tableName = "test_hive_comment_column_by_delta_" + randomNameSuffix(); String columnName = "id"; onTrino().executeQuery(createTableInHiveConnector("default", tableName, true)); @@ -555,7 +555,7 @@ public void testDeltaToHiveCommentColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testHiveToDeltaCommentColumn() { - String tableName = "test_delta_comment_column_by_hive_" + randomTableSuffix(); + String tableName = "test_delta_comment_column_by_hive_" + randomNameSuffix(); String columnName = "nationkey"; onDelta().executeQuery(createTableOnDelta(tableName, true)); @@ -579,8 +579,8 @@ public void testHiveToDeltaCommentColumn() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testInsertIntoDeltaTableFromHiveNonDefaultSchemaRedirect() { - String destSchema = "test_extraordinary_" + randomTableSuffix(); - String destTableName = "test_create_delta_table_from_hive_non_default_schema_" + randomTableSuffix(); + String destSchema = "test_extraordinary_" + randomNameSuffix(); + String destTableName = "test_create_delta_table_from_hive_non_default_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, destSchema); onDelta().executeQuery(format("CREATE SCHEMA IF NOT EXISTS %s LOCATION \"%s\"", destSchema, schemaLocation)); @@ -616,11 +616,11 @@ public void testInsertIntoDeltaTableFromHiveNonDefaultSchemaRedirect() public void testInformationSchemaColumnsHiveToDeltaRedirect() { // use dedicated schema so we control the number and shape of tables - String schemaName = "test_redirect_to_delta_information_schema_columns_schema_" + randomTableSuffix(); + String schemaName = "test_redirect_to_delta_information_schema_columns_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); onTrino().executeQuery(format("CREATE SCHEMA IF NOT EXISTS hive.%s WITH (location = '%s')", schemaName, schemaLocation)); - String tableName = "redirect_to_delta_information_schema_columns_table_" + randomTableSuffix(); + String tableName = "redirect_to_delta_information_schema_columns_table_" + randomNameSuffix(); try { onDelta().executeQuery(createTableOnDelta(schemaName, tableName, false)); @@ -662,11 +662,11 @@ public void testInformationSchemaColumnsHiveToDeltaRedirect() public void testInformationSchemaColumnsDeltaToHiveRedirect() { // use dedicated schema so we control the number and shape of tables - String schemaName = "test_redirect_to_hive_information_schema_columns_schema_" + randomTableSuffix(); + String schemaName = "test_redirect_to_hive_information_schema_columns_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); onTrino().executeQuery(format("CREATE SCHEMA IF NOT EXISTS hive.%s WITH (location='%s')", schemaName, schemaLocation)); - String tableName = "test_redirect_to_hive_information_schema_columns_table_" + randomTableSuffix(); + String tableName = "test_redirect_to_hive_information_schema_columns_table_" + randomNameSuffix(); try { onTrino().executeQuery(createTableInHiveConnector(schemaName, tableName, false)); @@ -705,11 +705,11 @@ public void testInformationSchemaColumnsDeltaToHiveRedirect() public void testSystemJdbcColumnsHiveToDeltaRedirect() { // use dedicated schema so we control the number and shape of tables - String schemaName = "test_redirect_to_delta_system_jdbc_columns_schema_" + randomTableSuffix(); + String schemaName = "test_redirect_to_delta_system_jdbc_columns_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); onTrino().executeQuery(format("CREATE SCHEMA IF NOT EXISTS hive.%s WITH (location='%s')", schemaName, schemaLocation)); - String tableName = "test_redirect_to_delta_system_jdbc_columns_table_" + randomTableSuffix(); + String tableName = "test_redirect_to_delta_system_jdbc_columns_table_" + randomNameSuffix(); try { onDelta().executeQuery(createTableOnDelta(schemaName, tableName, false)); @@ -752,11 +752,11 @@ public void testSystemJdbcColumnsHiveToDeltaRedirect() public void testSystemJdbcColumnsDeltaToHiveRedirect() { // use dedicated schema so we control the number and shape of tables - String schemaName = "test_redirect_to_hive_system_jdbc_columns_schema_" + randomTableSuffix(); + String schemaName = "test_redirect_to_hive_system_jdbc_columns_schema_" + randomNameSuffix(); String schemaLocation = format("s3://%s/delta-redirect-test-%s", bucketName, schemaName); onTrino().executeQuery(format("CREATE SCHEMA IF NOT EXISTS hive.%s WITH (location='%s')", schemaName, schemaLocation)); - String tableName = "test_redirect_to_hive_system_jdbc_columns_table_" + randomTableSuffix(); + String tableName = "test_redirect_to_hive_system_jdbc_columns_table_" + randomNameSuffix(); try { onTrino().executeQuery(createTableInHiveConnector(schemaName, tableName, false)); @@ -794,10 +794,10 @@ public void testSystemJdbcColumnsDeltaToHiveRedirect() @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testViewReferencingHiveAndDeltaTable(boolean legacyHiveViewTranslation) { - String hiveTableName = "test_view_hive_table_" + randomTableSuffix(); - String deltaTableName = "test_view_delta_table_" + randomTableSuffix(); - String viewName = "test_view_view_" + randomTableSuffix(); - String deltaRegionTableName = "test_view_delta_region_table_" + randomTableSuffix(); + String hiveTableName = "test_view_hive_table_" + randomNameSuffix(); + String deltaTableName = "test_view_delta_table_" + randomNameSuffix(); + String viewName = "test_view_view_" + randomNameSuffix(); + String deltaRegionTableName = "test_view_delta_region_table_" + randomNameSuffix(); @Language("SQL") String deltaTableData = "SELECT " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java index 7f2ff1f5f868..a12567073294 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java @@ -40,10 +40,10 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_ICEBERG_REDIRECTIONS; import static io.trino.tests.product.TestGroups.HIVE_VIEWS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onHive; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -170,8 +170,8 @@ public void testSelectOnViewFromDifferentSchema() @Test(groups = HIVE_VIEWS) public void testViewReferencingTableInDifferentSchema() { - String schemaX = "test_view_table_in_different_schema_x" + randomTableSuffix(); - String schemaY = "test_view_table_in_different_schema_y" + randomTableSuffix(); + String schemaX = "test_view_table_in_different_schema_x" + randomNameSuffix(); + String schemaY = "test_view_table_in_different_schema_y" + randomNameSuffix(); String tableName = "test_table"; String viewName = "test_view"; @@ -190,7 +190,7 @@ public void testViewReferencingTableInDifferentSchema() @Test(groups = HIVE_VIEWS) public void testViewReferencingTableInTheSameSchemaWithoutQualifier() { - String schemaX = "test_view_table_same_schema_without_qualifier_schema" + randomTableSuffix(); + String schemaX = "test_view_table_same_schema_without_qualifier_schema" + randomNameSuffix(); String tableName = "test_table"; String viewName = "test_view"; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAbfsSyncPartitionMetadata.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAbfsSyncPartitionMetadata.java index 6a0ead438aaf..dda69929ddc8 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAbfsSyncPartitionMetadata.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAbfsSyncPartitionMetadata.java @@ -19,8 +19,8 @@ import org.testng.annotations.Test; import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.AZURE; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_ISSUE; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_MATCH; import static io.trino.tests.product.utils.QueryExecutors.onHive; @@ -32,7 +32,7 @@ public class TestAbfsSyncPartitionMetadata extends BaseTestSyncPartitionMetadata { - private final String schema = "test_" + randomTableSuffix(); + private final String schema = "test_" + randomNameSuffix(); @BeforeTestWithContext public void setUp() diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestCreateDropSchema.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestCreateDropSchema.java index 837f65b902be..e1a13fe38f04 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestCreateDropSchema.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestCreateDropSchema.java @@ -20,7 +20,7 @@ import org.testng.annotations.Test; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -38,7 +38,7 @@ public class TestCreateDropSchema @Test public void testCreateDropSchema() { - String schemaName = "test_drop_schema_" + randomTableSuffix(); + String schemaName = "test_drop_schema_" + randomNameSuffix(); String schemaDir = format("%s/%s.db", warehouseDirectory, schemaName); onTrino().executeQuery("CREATE SCHEMA " + schemaName); @@ -56,7 +56,7 @@ public void testCreateDropSchema() @Test public void testDropSchemaFiles() { - String schemaName = "schema_without_location_" + randomTableSuffix(); + String schemaName = "schema_without_location_" + randomNameSuffix(); String schemaDir = format("%s/%s.db/", warehouseDirectory, schemaName); onTrino().executeQuery(format("CREATE SCHEMA %s", schemaName)); @@ -68,7 +68,7 @@ public void testDropSchemaFiles() @Test public void testDropSchemaFilesWithLocation() { - String schemaName = "schema_with_empty_location_" + randomTableSuffix(); + String schemaName = "schema_with_empty_location_" + randomNameSuffix(); String schemaDir = warehouseDirectory + "/schema-with-empty-location/"; onTrino().executeQuery(format("CREATE SCHEMA %s WITH (location = '%s')", schemaName, schemaDir)); @@ -80,7 +80,7 @@ public void testDropSchemaFilesWithLocation() @Test // specified location, external file in subdir public void testDropWithExternalFilesInSubdirectory() { - String schemaName = "schema_with_nonempty_location_" + randomTableSuffix(); + String schemaName = "schema_with_nonempty_location_" + randomNameSuffix(); String schemaDir = warehouseDirectory + "/schema-with-nonempty-location/"; // Use subdirectory to make sure file check is recursive String subDir = schemaDir + "subdir/"; @@ -101,7 +101,7 @@ public void testDropWithExternalFilesInSubdirectory() @Test // default location, empty external subdir public void testDropSchemaFilesWithEmptyExternalSubdir() { - String schemaName = "schema_with_empty_subdirectory_" + randomTableSuffix(); + String schemaName = "schema_with_empty_subdirectory_" + randomNameSuffix(); String schemaDir = format("%s/%s.db/", warehouseDirectory, schemaName); String externalSubdir = schemaDir + "external-subdir/"; @@ -118,7 +118,7 @@ public void testDropSchemaFilesWithEmptyExternalSubdir() @Test // default location, transactions without external files public void testDropSchemaFilesTransactions() { - String schemaName = "schema_directory_transactions_" + randomTableSuffix(); + String schemaName = "schema_directory_transactions_" + randomNameSuffix(); String schemaDir = format("%s/%s.db/", warehouseDirectory, schemaName); onTrino().executeQuery(format("CREATE SCHEMA %s", schemaName)); @@ -142,7 +142,7 @@ public void testDropSchemaFilesTransactions() @Test // specified location, transaction with top-level external file public void testDropTransactionsWithExternalFiles() { - String schemaName = "schema_transactions_with_external_files_" + randomTableSuffix(); + String schemaName = "schema_transactions_with_external_files_" + randomNameSuffix(); String schemaDir = warehouseDirectory + "/schema-transactions-with-external-files/"; // Create file in schema directory before creating schema diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java index 55b0d961c1fc..cbbca6dbc0de 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java @@ -43,12 +43,12 @@ import static io.trino.tempto.fulfillment.table.TableRequirements.immutableTable; import static io.trino.tempto.fulfillment.table.hive.tpch.TpchTableDefinitions.NATION; import static io.trino.tempto.query.QueryExecutor.param; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.LARGE_QUERY; import static io.trino.tests.product.TpchTableResults.PRESTO_NATION_RESULT; import static io.trino.tests.product.hive.BucketingType.BUCKETED_DEFAULT; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V1; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V2; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.hive.util.TemporaryHiveTable.temporaryHiveTable; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_ISSUE; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_MATCH; @@ -317,7 +317,7 @@ public void testBucketingVersion() @Flaky(issue = ERROR_COMMITTING_WRITE_TO_HIVE_ISSUE, match = ERROR_COMMITTING_WRITE_TO_HIVE_MATCH) public void testBucketingWithUnsupportedDataTypes(BucketingType bucketingType, String columnToBeBucketed) { - try (TemporaryHiveTable table = temporaryHiveTable("table_with_unsupported_bucketing_types_" + randomTableSuffix())) { + try (TemporaryHiveTable table = temporaryHiveTable("table_with_unsupported_bucketing_types_" + randomNameSuffix())) { String tableName = table.getName(); onHive().executeQuery(format("CREATE TABLE %s (" + "n_integer INT," + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHivePartitionSchemaEvolution.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHivePartitionSchemaEvolution.java index bd9d376d88bc..bcad0ec538b4 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHivePartitionSchemaEvolution.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHivePartitionSchemaEvolution.java @@ -28,7 +28,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.hive.util.TemporaryHiveTable.temporaryHiveTable; import static io.trino.tests.product.utils.JdbcDriverUtils.setSessionProperty; import static io.trino.tests.product.utils.QueryExecutors.onHive; @@ -133,7 +133,7 @@ private static boolean tryExecuteOnHive(String sql) private TemporaryHiveTable createTable(String format) { - String tableName = "schema_evolution_" + randomTableSuffix(); + String tableName = "schema_evolution_" + randomNameSuffix(); tryExecuteOnHive(format( "CREATE TABLE %s (" + " int_column int," + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java index df145082e536..24144f50d5e8 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRedirectionToIceberg.java @@ -27,9 +27,9 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tempto.query.QueryExecutor.param; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_ICEBERG_REDIRECTIONS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static java.sql.JDBCType.VARCHAR; @@ -46,7 +46,7 @@ public void createAdditionalSchema() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirect() { - String tableName = "redirect_" + randomTableSuffix(); + String tableName = "redirect_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -62,7 +62,7 @@ public void testRedirect() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectWithNonDefaultSchema() { - String tableName = "redirect_non_default_schema_" + randomTableSuffix(); + String tableName = "redirect_non_default_schema_" + randomNameSuffix(); String hiveTableName = "hive.nondefaultschema." + tableName; String icebergTableName = "iceberg.nondefaultschema." + tableName; @@ -78,7 +78,7 @@ public void testRedirectWithNonDefaultSchema() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectToNonexistentCatalog() { - String tableName = "redirect_to_nonexistent_iceberg_" + randomTableSuffix(); + String tableName = "redirect_to_nonexistent_iceberg_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -101,7 +101,7 @@ public void testRedirectToNonexistentCatalog() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectWithDefaultSchemaInSession() { - String tableName = "redirect_with_use_" + randomTableSuffix(); + String tableName = "redirect_with_use_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -123,7 +123,7 @@ public void testRedirectWithDefaultSchemaInSession() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectPartitionsToUnpartitioned() { - String tableName = "iceberg_unpartitioned_table_" + randomTableSuffix(); + String tableName = "iceberg_unpartitioned_table_" + randomNameSuffix(); String icebergTableName = "iceberg.default." + tableName; createIcebergTable(icebergTableName, false); @@ -139,7 +139,7 @@ public void testRedirectPartitionsToUnpartitioned() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectPartitionsToPartitioned() { - String tableName = "iceberg_partitioned_table_" + randomTableSuffix(); + String tableName = "iceberg_partitioned_table_" + randomNameSuffix(); String icebergTableName = "iceberg.default." + tableName; createIcebergTable(icebergTableName, true); @@ -160,7 +160,7 @@ public void testRedirectPartitionsToPartitioned() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}, dataProvider = "schemaAndPartitioning") public void testInsert(String schema, boolean partitioned) { - String tableName = "iceberg_insert_" + randomTableSuffix(); + String tableName = "iceberg_insert_" + randomNameSuffix(); String hiveTableName = "hive." + schema + "." + tableName; String icebergTableName = "iceberg." + schema + "." + tableName; @@ -191,7 +191,7 @@ public static Object[][] schemaAndPartitioning() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDelete() { - String tableName = "iceberg_insert_" + randomTableSuffix(); + String tableName = "iceberg_insert_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -209,7 +209,7 @@ public void testDelete() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testUpdate() { - String tableName = "iceberg_insert_" + randomTableSuffix(); + String tableName = "iceberg_insert_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -226,8 +226,8 @@ public void testUpdate() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testMerge() { - String sourceTableName = "iceberg_merge_source_" + randomTableSuffix(); - String targetTableName = "iceberg_merge_target_" + randomTableSuffix(); + String sourceTableName = "iceberg_merge_source_" + randomNameSuffix(); + String targetTableName = "iceberg_merge_target_" + randomNameSuffix(); String hiveSourceTableName = "hive.default." + sourceTableName; String hiveTargetTableName = "hive.default." + targetTableName; String icebergSourceTableName = "iceberg.default." + sourceTableName; @@ -253,7 +253,7 @@ public void testMerge() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDropTable() { - String tableName = "hive_drop_iceberg_" + randomTableSuffix(); + String tableName = "hive_drop_iceberg_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -266,7 +266,7 @@ public void testDropTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDescribe() { - String tableName = "iceberg_describe_" + randomTableSuffix(); + String tableName = "iceberg_describe_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -282,7 +282,7 @@ public void testDescribe() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowCreateTable() { - String tableName = "iceberg_show_create_table_" + randomTableSuffix(); + String tableName = "iceberg_show_create_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -308,7 +308,7 @@ public void testShowCreateTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowStats() { - String tableName = "iceberg_show_create_table_" + randomTableSuffix(); + String tableName = "iceberg_show_create_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -328,7 +328,7 @@ public void testShowStats() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableRename() { - String tableName = "iceberg_rename_table_" + randomTableSuffix(); + String tableName = "iceberg_rename_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -354,7 +354,7 @@ public void testAlterTableRename() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableAddColumn() { - String tableName = "iceberg_alter_table_" + randomTableSuffix(); + String tableName = "iceberg_alter_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -374,7 +374,7 @@ public void testAlterTableAddColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableDropColumn() { - String tableName = "iceberg_alter_table_drop_column_" + randomTableSuffix(); + String tableName = "iceberg_alter_table_drop_column_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -394,7 +394,7 @@ public void testAlterTableDropColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableRenameColumn() { - String tableName = "iceberg_alter_table_" + randomTableSuffix(); + String tableName = "iceberg_alter_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -414,7 +414,7 @@ public void testAlterTableRenameColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testCommentTable() { - String tableName = "iceberg_comment_table_" + randomTableSuffix(); + String tableName = "iceberg_comment_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -435,7 +435,7 @@ public void testCommentTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testCommentColumn() { - String tableName = "iceberg_comment_column_" + randomTableSuffix(); + String tableName = "iceberg_comment_column_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; String columnName = "nationkey"; @@ -456,7 +456,7 @@ public void testCommentColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowGrants() { - String tableName = "iceberg_show_grants_" + randomTableSuffix(); + String tableName = "iceberg_show_grants_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; createIcebergTable(icebergTableName, false); @@ -471,10 +471,10 @@ public void testShowGrants() public void testInformationSchemaColumns() { // use dedicated schema so that we control the number and shape of tables - String schemaName = "redirect_information_schema_" + randomTableSuffix(); + String schemaName = "redirect_information_schema_" + randomNameSuffix(); onTrino().executeQuery("CREATE SCHEMA IF NOT EXISTS hive." + schemaName); - String tableName = "redirect_information_schema_table_" + randomTableSuffix(); + String tableName = "redirect_information_schema_table_" + randomNameSuffix(); String icebergTableName = "iceberg." + schemaName + "." + tableName; createIcebergTable(icebergTableName, false); @@ -514,10 +514,10 @@ public void testInformationSchemaColumns() public void testSystemJdbcColumns() { // use dedicated schema so that we control the number and shape of tables - String schemaName = "redirect_system_jdbc_columns_" + randomTableSuffix(); + String schemaName = "redirect_system_jdbc_columns_" + randomNameSuffix(); onTrino().executeQuery("CREATE SCHEMA IF NOT EXISTS hive." + schemaName); - String tableName = "redirect_system_jdbc_columns_table_" + randomTableSuffix(); + String tableName = "redirect_system_jdbc_columns_table_" + randomNameSuffix(); String icebergTableName = "iceberg." + schemaName + "." + tableName; createIcebergTable(icebergTableName, false); @@ -557,7 +557,7 @@ public void testSystemJdbcColumns() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testGrant() { - String tableName = "iceberg_grant_" + randomTableSuffix(); + String tableName = "iceberg_grant_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -572,7 +572,7 @@ public void testGrant() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRevoke() { - String tableName = "iceberg_revoke_" + randomTableSuffix(); + String tableName = "iceberg_revoke_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -587,7 +587,7 @@ public void testRevoke() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testSetTableAuthorization() { - String tableName = "iceberg_set_table_authorization_" + randomTableSuffix(); + String tableName = "iceberg_set_table_authorization_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -602,7 +602,7 @@ public void testSetTableAuthorization() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDeny() { - String tableName = "iceberg_deny_" + randomTableSuffix(); + String tableName = "iceberg_deny_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveSparkCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveSparkCompatibility.java index 705df7f766ca..fed7efcd4614 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveSparkCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveSparkCompatibility.java @@ -26,9 +26,9 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_SPARK; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onHive; import static io.trino.tests.product.utils.QueryExecutors.onSpark; import static io.trino.tests.product.utils.QueryExecutors.onTrino; @@ -46,7 +46,7 @@ public class TestHiveSparkCompatibility @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}, dataProvider = "testReadSparkCreatedTableDataProvider") public void testReadSparkCreatedTable(String sparkTableFormat, String expectedTrinoTableFormat) { - String sparkTableName = "spark_created_table_" + sparkTableFormat.replaceAll("[^a-zA-Z]", "").toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String sparkTableName = "spark_created_table_" + sparkTableFormat.replaceAll("[^a-zA-Z]", "").toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); onSpark().executeQuery( @@ -191,7 +191,7 @@ public void testReadSparkCreatedTable(String sparkTableFormat, String expectedTr @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}, dataProvider = "sparkParquetTimestampFormats") public void testSparkParquetTimestampCompatibility(String sparkTimestampFormat, String sparkTimestamp, String[] expectedValues) { - String sparkTableName = "test_spark_parquet_timestamp_compatibility_" + sparkTimestampFormat.toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String sparkTableName = "test_spark_parquet_timestamp_compatibility_" + sparkTimestampFormat.toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); onSpark().executeQuery("SET spark.sql.parquet.outputTimestampType = " + sparkTimestampFormat); @@ -217,7 +217,7 @@ public void testSparkParquetTimestampCompatibility(String sparkTimestampFormat, @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}) public void testInsertFailsOnBucketedTableCreatedBySpark() { - String hiveTableName = "spark_insert_bucketed_table_" + randomTableSuffix(); + String hiveTableName = "spark_insert_bucketed_table_" + randomNameSuffix(); onSpark().executeQuery( "CREATE TABLE default." + hiveTableName + "(a_key integer, a_value integer) " + @@ -233,7 +233,7 @@ public void testInsertFailsOnBucketedTableCreatedBySpark() @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}) public void testUpdateFailsOnBucketedTableCreatedBySpark() { - String hiveTableName = "spark_update_bucketed_table_" + randomTableSuffix(); + String hiveTableName = "spark_update_bucketed_table_" + randomNameSuffix(); onSpark().executeQuery( "CREATE TABLE default." + hiveTableName + "(a_key integer, a_value integer) " + @@ -249,7 +249,7 @@ public void testUpdateFailsOnBucketedTableCreatedBySpark() @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}) public void testDeleteFailsOnBucketedTableCreatedBySpark() { - String hiveTableName = "spark_delete_bucketed_table_" + randomTableSuffix(); + String hiveTableName = "spark_delete_bucketed_table_" + randomNameSuffix(); onSpark().executeQuery( "CREATE TABLE default." + hiveTableName + "(a_key integer, a_value integer) " + @@ -311,7 +311,7 @@ public void testReadTrinoCreatedParquetTableWithNativeWriter() private void testReadTrinoCreatedTable(String tableName, String tableFormat) { - String sparkTableName = "trino_created_table_" + tableName + "_" + randomTableSuffix(); + String sparkTableName = "trino_created_table_" + tableName + "_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); // Spark timestamps are in microsecond precision @@ -427,7 +427,7 @@ private void testReadTrinoCreatedTable(String tableName, String tableFormat) @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}) public void testReadSparkdDateAndTimePartitionName() { - String sparkTableName = "test_trino_reading_spark_date_and_time_type_partitioned_" + randomTableSuffix(); + String sparkTableName = "test_trino_reading_spark_date_and_time_type_partitioned_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); onSpark().executeQuery(format("CREATE TABLE default.%s (value integer) PARTITIONED BY (dt date)", sparkTableName)); @@ -486,7 +486,7 @@ public void testReadSparkdDateAndTimePartitionName() @Test(groups = {HIVE_SPARK, PROFILE_SPECIFIC_TESTS}, dataProvider = "unsupportedPartitionDates") public void testReadSparkInvalidDatePartitionName(String inputDate, java.sql.Date outputDate) { - String sparkTableName = "test_trino_reading_spark_invalid_date_type_partitioned_" + randomTableSuffix(); + String sparkTableName = "test_trino_reading_spark_invalid_date_type_partitioned_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); onSpark().executeQuery(format("CREATE TABLE default.%s (value integer) PARTITIONED BY (dt date)", sparkTableName)); @@ -525,7 +525,7 @@ public void testReadSparkBucketedTable() { // Spark tables can be created using native Spark code or by going through Hive code // This tests the native Spark path. - String sparkTableName = "test_trino_reading_spark_native_buckets_" + randomTableSuffix(); + String sparkTableName = "test_trino_reading_spark_native_buckets_" + randomNameSuffix(); String trinoTableName = format("%s.default.%s", TRINO_CATALOG, sparkTableName); onSpark().executeQuery(format( diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java index dbbc00e2286f..a97c2ddd1056 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java @@ -62,10 +62,10 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tempto.query.QueryExecutor.param; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HMS_ONLY; 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.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_ISSUE; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_MATCH; import static io.trino.tests.product.utils.JdbcDriverUtils.setSessionProperty; @@ -804,7 +804,7 @@ public void testLargeOrcInsert() private void runLargeInsert(StorageFormat storageFormat) { - String tableName = "test_large_insert_" + storageFormat.getName() + randomTableSuffix(); + String tableName = "test_large_insert_" + storageFormat.getName() + randomNameSuffix(); setSessionProperties(storageFormat); onTrino().executeQuery("CREATE TABLE " + tableName + " WITH (" + storageFormat.getStoragePropertiesAsSql() + ") AS SELECT * FROM tpch.sf1.lineitem WHERE false"); onTrino().executeQuery("INSERT INTO " + tableName + " SELECT * FROM tpch.sf1.lineitem"); @@ -935,7 +935,7 @@ private String createTestTable(String tableNamePrefix, StorageFormat format, Str setSessionProperties(onTrino().getConnection(), format); String formatName = format.getName().toLowerCase(ENGLISH); - String tableName = format("%s_%s_%s", tableNamePrefix, formatName, randomTableSuffix()); + String tableName = format("%s_%s_%s", tableNamePrefix, formatName, randomNameSuffix()); onTrino().executeQuery( format("CREATE TABLE %s %s WITH (%s)", tableName, sql, format.getStoragePropertiesAsSql())); return tableName; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java index 3ae365b08feb..087f262ac61e 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java @@ -55,6 +55,7 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V2; @@ -64,7 +65,6 @@ import static io.trino.tests.product.hive.TransactionalTableType.ACID; import static io.trino.tests.product.hive.TransactionalTableType.INSERT_ONLY; import static io.trino.tests.product.hive.util.TableLocationUtils.getTablePath; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_ISSUE; import static io.trino.tests.product.utils.HadoopTestUtils.ERROR_COMMITTING_WRITE_TO_HIVE_MATCH; import static io.trino.tests.product.utils.QueryExecutors.onHive; @@ -471,7 +471,7 @@ public void testFailAcidBeforeHive3() throw new SkipException("This tests behavior of ACID table before Hive 3 "); } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable("test_fail_acid_before_hive3_" + randomTableSuffix())) { + try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable("test_fail_acid_before_hive3_" + randomNameSuffix())) { String tableName = table.getName(); onHive().executeQuery("" + "CREATE TABLE " + tableName + "(a bigint) " + @@ -511,7 +511,7 @@ public void testCtasAcidTable(boolean isPartitioned, BucketingType bucketingType throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(format("ctas_transactional_%s", randomTableSuffix()))) { + try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(format("ctas_transactional_%s", randomNameSuffix()))) { String tableName = table.getName(); onTrino().executeQuery("CREATE TABLE " + tableName + " " + trinoTableProperties(ACID, isPartitioned, bucketingType) + @@ -1806,7 +1806,7 @@ void withTemporaryTable(String rootName, boolean transactional, boolean isPartit if (transactional) { ensureTransactionalHive(); } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName(rootName, isPartitioned, bucketingType) + randomTableSuffix())) { + try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName(rootName, isPartitioned, bucketingType) + randomNameSuffix())) { testRunner.accept(table.getName()); } } @@ -2180,7 +2180,7 @@ private static Stream> mapRows(QueryResult result) public static String tableName(String testName, boolean isPartitioned, BucketingType bucketingType) { - return format("test_%s_%b_%s_%s", testName, isPartitioned, bucketingType.name(), randomTableSuffix()); + return format("test_%s_%b_%s_%s", testName, isPartitioned, bucketingType.name(), randomNameSuffix()); } private static boolean isCompactionForTable(CompactionMode compactMode, String tableName, Map row) diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/util/TemporaryHiveTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/util/TemporaryHiveTable.java index b70098ff71ec..6c9a8f90d6a8 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/util/TemporaryHiveTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/util/TemporaryHiveTable.java @@ -13,31 +13,17 @@ */ package io.trino.tests.product.hive.util; -import java.security.SecureRandom; - import static io.trino.tests.product.utils.QueryExecutors.onHive; -import static java.lang.Character.MAX_RADIX; -import static java.lang.Math.abs; -import static java.lang.Math.min; import static java.util.Objects.requireNonNull; public class TemporaryHiveTable implements AutoCloseable { - private static final SecureRandom random = new SecureRandom(); - private static final int RANDOM_SUFFIX_LENGTH = 12; - public static TemporaryHiveTable temporaryHiveTable(String tableName) { return new TemporaryHiveTable(tableName); } - public static String randomTableSuffix() - { - String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX); - return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length())); - } - private final String name; private TemporaryHiveTable(String name) 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 d373554e8c65..8ec58b37fdd1 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 @@ -22,8 +22,8 @@ import javax.inject.Inject; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.ICEBERG; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -48,7 +48,7 @@ public void useIceberg() @Test(groups = ICEBERG) public void testDropSchemaFiles() { - String schemaName = "schema_without_location_" + randomTableSuffix(); + String schemaName = "schema_without_location_" + randomNameSuffix(); String schemaDir = format("%s/%s.db/", warehouseDirectory, schemaName); onTrino().executeQuery(format("CREATE SCHEMA %s", schemaName)); @@ -60,7 +60,7 @@ public void testDropSchemaFiles() @Test(groups = ICEBERG) public void testDropSchemaFilesWithLocation() { - String schemaName = "schema_with_empty_location_" + randomTableSuffix(); + String schemaName = "schema_with_empty_location_" + randomNameSuffix(); String schemaDir = warehouseDirectory + "/schema-with-empty-location/"; onTrino().executeQuery(format("CREATE SCHEMA %s WITH (location = '%s')", schemaName, schemaDir)); @@ -72,7 +72,7 @@ public void testDropSchemaFilesWithLocation() @Test(groups = ICEBERG) // specified location, external file in subdir public void testDropWithExternalFilesInSubdirectory() { - String schemaName = "schema_with_nonempty_location_" + randomTableSuffix(); + String schemaName = "schema_with_nonempty_location_" + randomNameSuffix(); String schemaDir = warehouseDirectory + "/schema-with-nonempty-location/"; // Use subdirectory to make sure file check is recursive String subDir = schemaDir + "subdir/"; @@ -93,7 +93,7 @@ public void testDropWithExternalFilesInSubdirectory() @Test(groups = ICEBERG) // default location, external file at top level public void testDropWithExternalFiles() { - String schemaName = "schema_with_files_in_default_location_" + randomTableSuffix(); + String schemaName = "schema_with_files_in_default_location_" + randomNameSuffix(); String schemaDir = format("%s/%s.db/", warehouseDirectory, schemaName); // Create file in schema directory before creating schema diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergCreateTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergCreateTable.java index 8fb889c3edd8..cc6f0a33f6c9 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergCreateTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergCreateTable.java @@ -22,10 +22,10 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HMS_ONLY; import static io.trino.tests.product.TestGroups.ICEBERG; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -51,7 +51,7 @@ public void cleanUp() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testCreateTable() { - String tableName = "iceberg.iceberg.test_create_table_" + randomTableSuffix(); + String tableName = "iceberg.iceberg.test_create_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE " + tableName + "(a bigint, b varchar)"); try { onTrino().executeQuery("INSERT INTO " + tableName + "(a, b) VALUES " + @@ -72,7 +72,7 @@ public void testCreateTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testCreateTableAsSelect() { - String tableName = "iceberg.iceberg.test_create_table_as_select_" + randomTableSuffix(); + String tableName = "iceberg.iceberg.test_create_table_as_select_" + randomNameSuffix(); onTrino().executeQuery("" + "CREATE TABLE " + tableName + " AS " + "SELECT * FROM (VALUES " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergFormatVersionCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergFormatVersionCompatibility.java index cdb67bf190fa..770610ab3cd7 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergFormatVersionCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergFormatVersionCompatibility.java @@ -22,9 +22,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.ICEBERG_FORMAT_VERSION_COMPATIBILITY; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onCompatibilityTestServer; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -36,7 +36,7 @@ public class TestIcebergFormatVersionCompatibility @Test(groups = {ICEBERG_FORMAT_VERSION_COMPATIBILITY, PROFILE_SPECIFIC_TESTS}) public void testTrinoTimeTravelReadTableCreatedByEarlyVersionTrino() { - String baseTableName = "test_trino_time_travel_read_table_created_by_early_version_trino_" + randomTableSuffix(); + String baseTableName = "test_trino_time_travel_read_table_created_by_early_version_trino_" + randomNameSuffix(); String tableName = format("iceberg.default.%s", baseTableName); String snapshotsTableName = format("iceberg.default.\"%s$snapshots\"", baseTableName); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveTablesCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveTablesCompatibility.java index 05c16d9a6e54..70966b1cc3c3 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveTablesCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergHiveTablesCompatibility.java @@ -19,10 +19,10 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HMS_ONLY; import static io.trino.tests.product.TestGroups.ICEBERG; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -38,7 +38,7 @@ public class TestIcebergHiveTablesCompatibility @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testIcebergSelectFromHiveTable() { - String tableName = "test_iceberg_select_from_hive_" + randomTableSuffix(); + String tableName = "test_iceberg_select_from_hive_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE hive.default." + tableName + "(a bigint)"); assertQueryFailure(() -> onTrino().executeQuery("SELECT * FROM iceberg.default." + tableName)) @@ -56,7 +56,7 @@ public void testIcebergSelectFromHiveTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testHiveSelectFromIcebergTable() { - String tableName = "test_hive_select_from_iceberg_" + randomTableSuffix(); + String tableName = "test_hive_select_from_iceberg_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE iceberg.default." + tableName + "(a bigint)"); assertQueryFailure(() -> onTrino().executeQuery("SELECT * FROM hive.default." + tableName)) @@ -74,7 +74,7 @@ public void testHiveSelectFromIcebergTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testIcebergCannotCreateTableNamesakeToHiveTable() { - String tableName = "test_iceberg_create_namesake_hive_table_" + randomTableSuffix(); + String tableName = "test_iceberg_create_namesake_hive_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE hive.default." + tableName + "(a bigint)"); assertQueryFailure(() -> onTrino().executeQuery("CREATE TABLE iceberg.default." + tableName + "(a bigint)")) @@ -86,7 +86,7 @@ public void testIcebergCannotCreateTableNamesakeToHiveTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testHiveCannotCreateTableNamesakeToIcebergTable() { - String tableName = "test_hive_create_namesake_iceberg_table_" + randomTableSuffix(); + String tableName = "test_hive_create_namesake_iceberg_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE iceberg.default." + tableName + "(a bigint)"); assertQueryFailure(() -> onTrino().executeQuery("CREATE TABLE hive.default." + tableName + "(a bigint)")) @@ -98,10 +98,10 @@ public void testHiveCannotCreateTableNamesakeToIcebergTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testHiveSelectTableColumns() { - String hiveTableName = "test_hive_table_columns_table_" + randomTableSuffix(); + String hiveTableName = "test_hive_table_columns_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE hive.default." + hiveTableName + "(a bigint)"); - String icebergTableName = "test_iceberg_table_columns_table_" + randomTableSuffix(); + String icebergTableName = "test_iceberg_table_columns_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE iceberg.default." + icebergTableName + "(a bigint)"); assertThat(onTrino().executeQuery( @@ -119,7 +119,7 @@ public void testHiveSelectTableColumns() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testHiveListsIcebergTable() { - String tableName = "test_hive_lists_iceberg_table_" + randomTableSuffix(); + String tableName = "test_hive_lists_iceberg_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE iceberg.default." + tableName + "(a bigint)"); assertThat(onTrino().executeQuery("SHOW TABLES FROM hive.default")).contains(row(tableName)); onTrino().executeQuery("DROP TABLE iceberg.default." + tableName); @@ -128,7 +128,7 @@ public void testHiveListsIcebergTable() @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testIcebergListsHiveTable() { - String tableName = "test_iceberg_lists_hive_table_" + randomTableSuffix(); + String tableName = "test_iceberg_lists_hive_table_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE hive.default." + tableName + "(a bigint)"); assertThat(onTrino().executeQuery("SHOW TABLES FROM iceberg.default")).contains(row(tableName)); onTrino().executeQuery("DROP TABLE hive.default." + tableName); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergInsert.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergInsert.java index a94ca440ce3f..f341dc56b117 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergInsert.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergInsert.java @@ -31,10 +31,10 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HMS_ONLY; import static io.trino.tests.product.TestGroups.ICEBERG; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS_DETAILED; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.util.concurrent.TimeUnit.SECONDS; @@ -52,7 +52,7 @@ public void testIcebergConcurrentInsert() int threads = 3; int insertsPerThread = 7; - String tableName = "iceberg.default.test_insert_concurrent_" + randomTableSuffix(); + String tableName = "iceberg.default.test_insert_concurrent_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE " + tableName + "(a bigint)"); ExecutorService executor = Executors.newFixedThreadPool(threads); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergOptimize.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergOptimize.java index d16ca7a1e4e3..310edc520472 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergOptimize.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergOptimize.java @@ -22,9 +22,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +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.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onSpark; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -43,7 +43,7 @@ public class TestIcebergOptimize @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testOptimizeTableAfterDelete() { - String baseTableName = "test_optimize_with_small_split_size_" + randomTableSuffix(); + String baseTableName = "test_optimize_with_small_split_size_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); onTrino().executeQuery("DROP TABLE IF EXISTS " + trinoTableName); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergProcedureCalls.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergProcedureCalls.java index 2098537653c9..ab22a503147f 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergProcedureCalls.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergProcedureCalls.java @@ -18,9 +18,9 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +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.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; @@ -31,7 +31,7 @@ public class TestIcebergProcedureCalls public void testRollbackToSnapshot() throws InterruptedException { - String tableName = "test_rollback_to_snapshot_" + randomTableSuffix(); + String tableName = "test_rollback_to_snapshot_" + randomNameSuffix(); onTrino().executeQuery("USE iceberg.default"); onTrino().executeQuery(format("DROP TABLE IF EXISTS %s", tableName)); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRedirectionToHive.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRedirectionToHive.java index 74d1d316033e..b001975c4fe5 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRedirectionToHive.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRedirectionToHive.java @@ -27,9 +27,9 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; import static io.trino.tempto.query.QueryExecutor.param; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_ICEBERG_REDIRECTIONS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static java.sql.JDBCType.VARCHAR; @@ -52,7 +52,7 @@ public void createAdditionalSchema() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirect() { - String tableName = "redirect_" + randomTableSuffix(); + String tableName = "redirect_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -68,7 +68,7 @@ public void testRedirect() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectWithNonDefaultSchema() { - String tableName = "redirect_non_default_schema_" + randomTableSuffix(); + String tableName = "redirect_non_default_schema_" + randomNameSuffix(); String hiveTableName = "hive.nondefaultschema." + tableName; String icebergTableName = "iceberg.nondefaultschema." + tableName; @@ -84,7 +84,7 @@ public void testRedirectWithNonDefaultSchema() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectToNonexistentCatalog() { - String tableName = "redirect_to_nonexistent_hive_" + randomTableSuffix(); + String tableName = "redirect_to_nonexistent_hive_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -107,7 +107,7 @@ public void testRedirectToNonexistentCatalog() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectWithDefaultSchemaInSession() { - String tableName = "redirect_with_use_" + randomTableSuffix(); + String tableName = "redirect_with_use_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -129,7 +129,7 @@ public void testRedirectWithDefaultSchemaInSession() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectPartitionsToUnpartitioned() { - String tableName = "hive_unpartitioned_table_" + randomTableSuffix(); + String tableName = "hive_unpartitioned_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -144,7 +144,7 @@ public void testRedirectPartitionsToUnpartitioned() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRedirectPartitionsToPartitioned() { - String tableName = "hive_partitioned_table_" + randomTableSuffix(); + String tableName = "hive_partitioned_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; createHiveTable(hiveTableName, true); @@ -163,7 +163,7 @@ public void testRedirectPartitionsToPartitioned() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}, dataProvider = "schemaAndPartitioning") public void testInsert(String schema, boolean partitioned) { - String tableName = "hive_insert_" + randomTableSuffix(); + String tableName = "hive_insert_" + randomNameSuffix(); String hiveTableName = "hive." + schema + "." + tableName; String icebergTableName = "iceberg." + schema + "." + tableName; @@ -194,7 +194,7 @@ public static Object[][] schemaAndPartitioning() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDelete() { - String tableName = "hive_delete_" + randomTableSuffix(); + String tableName = "hive_delete_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -212,7 +212,7 @@ public void testDelete() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testUpdate() { - String tableName = "hive_update_" + randomTableSuffix(); + String tableName = "hive_update_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -227,7 +227,7 @@ public void testUpdate() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDropTable() { - String tableName = "iceberg_drop_hive_" + randomTableSuffix(); + String tableName = "iceberg_drop_hive_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -240,7 +240,7 @@ public void testDropTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDescribe() { - String tableName = "hive_describe_" + randomTableSuffix(); + String tableName = "hive_describe_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -256,7 +256,7 @@ public void testDescribe() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowCreateTable() { - String tableName = "hive_show_create_table_" + randomTableSuffix(); + String tableName = "hive_show_create_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -280,7 +280,7 @@ public void testShowCreateTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowStats() { - String tableName = "hive_show_create_table_" + randomTableSuffix(); + String tableName = "hive_show_create_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -300,7 +300,7 @@ public void testShowStats() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableRename() { - String tableName = "iceberg_rename_table_" + randomTableSuffix(); + String tableName = "iceberg_rename_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -326,7 +326,7 @@ public void testAlterTableRename() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableAddColumn() { - String tableName = "hive_alter_table_" + randomTableSuffix(); + String tableName = "hive_alter_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -346,7 +346,7 @@ public void testAlterTableAddColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableRenameColumn() { - String tableName = "hive_rename_column_" + randomTableSuffix(); + String tableName = "hive_rename_column_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -367,7 +367,7 @@ public void testAlterTableRenameColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testAlterTableDropColumn() { - String tableName = "hive_alter_table_drop_column_" + randomTableSuffix(); + String tableName = "hive_alter_table_drop_column_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -390,7 +390,7 @@ public void testAlterTableDropColumn() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testCommentTable() { - String tableName = "hive_comment_table_" + randomTableSuffix(); + String tableName = "hive_comment_table_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -411,7 +411,7 @@ public void testCommentTable() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testShowGrants() { - String tableName = "hive_show_grants_" + randomTableSuffix(); + String tableName = "hive_show_grants_" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -427,10 +427,10 @@ public void testShowGrants() public void testInformationSchemaColumns() { // use dedicated schema so that we control the number and shape of tables - String schemaName = "redirect_information_schema_" + randomTableSuffix(); + String schemaName = "redirect_information_schema_" + randomNameSuffix(); onTrino().executeQuery("CREATE SCHEMA IF NOT EXISTS hive." + schemaName); - String tableName = "redirect_information_schema_table_" + randomTableSuffix(); + String tableName = "redirect_information_schema_table_" + randomNameSuffix(); String hiveTableName = "hive." + schemaName + "." + tableName; createHiveTable(hiveTableName, false); @@ -470,10 +470,10 @@ public void testInformationSchemaColumns() public void testSystemJdbcColumns() { // use dedicated schema so that we control the number and shape of tables - String schemaName = "redirect_system_jdbc_columns_" + randomTableSuffix(); + String schemaName = "redirect_system_jdbc_columns_" + randomNameSuffix(); onTrino().executeQuery("CREATE SCHEMA IF NOT EXISTS hive." + schemaName); - String tableName = "redirect_system_jdbc_columns_table_" + randomTableSuffix(); + String tableName = "redirect_system_jdbc_columns_table_" + randomNameSuffix(); String hiveTableName = "hive." + schemaName + "." + tableName; createHiveTable(hiveTableName, false); @@ -512,7 +512,7 @@ public void testSystemJdbcColumns() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testGrant() { - String tableName = "hive_grant" + randomTableSuffix(); + String tableName = "hive_grant" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -527,7 +527,7 @@ public void testGrant() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testRevoke() { - String tableName = "hive_revoke" + randomTableSuffix(); + String tableName = "hive_revoke" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -542,7 +542,7 @@ public void testRevoke() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testSetTableAuthorization() { - String tableName = "hive_revoke" + randomTableSuffix(); + String tableName = "hive_revoke" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; @@ -557,7 +557,7 @@ public void testSetTableAuthorization() @Test(groups = {HIVE_ICEBERG_REDIRECTIONS, PROFILE_SPECIFIC_TESTS}) public void testDeny() { - String tableName = "hive_deny" + randomTableSuffix(); + String tableName = "hive_deny" + randomNameSuffix(); String hiveTableName = "hive.default." + tableName; String icebergTableName = "iceberg.default." + tableName; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRenameTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRenameTable.java index 167a933cc58a..d5a1594e9c98 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRenameTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergRenameTable.java @@ -18,10 +18,10 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HMS_ONLY; import static io.trino.tests.product.TestGroups.ICEBERG; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.utils.QueryExecutors.onTrino; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -31,8 +31,8 @@ public class TestIcebergRenameTable @Test(groups = {ICEBERG, STORAGE_FORMATS, HMS_ONLY}) public void testRenameTable() { - String tableName = "iceberg.default.test_rename_table_" + randomTableSuffix(); - String newName = "iceberg.default.test_rename_table_new_" + randomTableSuffix(); + String tableName = "iceberg.default.test_rename_table_" + randomNameSuffix(); + String newName = "iceberg.default.test_rename_table_new_" + randomNameSuffix(); onTrino().executeQuery("CREATE TABLE " + tableName + "(a bigint, b varchar)"); try { onTrino().executeQuery("INSERT INTO " + tableName + "(a, b) VALUES " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java index c733ea778d1b..6bcc31d6c6e9 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java @@ -59,9 +59,9 @@ import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tempto.assertions.QueryAssert.assertThat; +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.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.iceberg.TestIcebergSparkCompatibility.CreateMode.CREATE_TABLE_AND_INSERT; import static io.trino.tests.product.iceberg.TestIcebergSparkCompatibility.CreateMode.CREATE_TABLE_AS_SELECT; import static io.trino.tests.product.iceberg.TestIcebergSparkCompatibility.CreateMode.CREATE_TABLE_WITH_NO_DATA_AND_INSERT; @@ -835,8 +835,8 @@ public void testTrinoShowingSparkCreatedTables(int specVersion) public void testCreateAndDropTableWithSameLocationWorksOnSpark(int specVersion) { String dataPath = "hdfs://hadoop-master:9000/user/hive/warehouse/test_create_table_same_location/obj-data"; - String tableSameLocation1 = "test_same_location_spark_1_" + randomTableSuffix(); - String tableSameLocation2 = "test_same_location_spark_2_" + randomTableSuffix(); + String tableSameLocation1 = "test_same_location_spark_1_" + randomNameSuffix(); + String tableSameLocation2 = "test_same_location_spark_2_" + randomNameSuffix(); onSpark().executeQuery(format("CREATE TABLE %s (_integer INTEGER ) USING ICEBERG LOCATION '%s' TBLPROPERTIES('format-version' = %s)", sparkTableName(tableSameLocation1), dataPath, specVersion)); @@ -854,8 +854,8 @@ public void testCreateAndDropTableWithSameLocationWorksOnSpark(int specVersion) public void testCreateAndDropTableWithSameLocationFailsOnTrino(int specVersion) { String dataPath = "hdfs://hadoop-master:9000/user/hive/warehouse/test_create_table_same_location/obj-data"; - String tableSameLocation1 = "test_same_location_trino_1_" + randomTableSuffix(); - String tableSameLocation2 = "test_same_location_trino_2_" + randomTableSuffix(); + String tableSameLocation1 = "test_same_location_trino_1_" + randomNameSuffix(); + String tableSameLocation2 = "test_same_location_trino_2_" + randomNameSuffix(); onSpark().executeQuery(format("CREATE TABLE %s (_integer INTEGER ) USING ICEBERG LOCATION '%s' TBLPROPERTIES('format-version' = %s)", sparkTableName(tableSameLocation1), dataPath, specVersion)); @@ -1125,7 +1125,7 @@ public void testTrinoSparkConcurrentInsert() { int insertsPerEngine = 7; - String baseTableName = "trino_spark_insert_concurrent_" + randomTableSuffix(); + String baseTableName = "trino_spark_insert_concurrent_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); onTrino().executeQuery("CREATE TABLE " + trinoTableName + "(e varchar, a bigint)"); @@ -1192,7 +1192,7 @@ public void testTrinoReadingSparkCompressedData(StorageFormat storageFormat, Str String baseTableName = "test_spark_compression" + "_" + storageFormat + "_" + compressionCodec + - "_" + randomTableSuffix(); + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -1257,7 +1257,7 @@ public void testSparkReadingTrinoCompressedData(StorageFormat storageFormat, Str String baseTableName = "test_trino_compression" + "_" + storageFormat + "_" + compressionCodec + - "_" + randomTableSuffix(); + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -1323,7 +1323,7 @@ private List compressionCodecs() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testTrinoReadingMigratedNestedData(StorageFormat storageFormat) { - String baseTableName = "test_trino_reading_migrated_nested_data_" + randomTableSuffix(); + String baseTableName = "test_trino_reading_migrated_nested_data_" + randomNameSuffix(); String defaultCatalogTableName = sparkDefaultCatalogTableName(baseTableName); String sparkTableDefinition = "" + @@ -1389,7 +1389,7 @@ public void testTrinoReadingMigratedNestedData(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testMigratedDataWithAlteredSchema(StorageFormat storageFormat) { - String baseTableName = "test_migrated_data_with_altered_schema_" + randomTableSuffix(); + String baseTableName = "test_migrated_data_with_altered_schema_" + randomNameSuffix(); String defaultCatalogTableName = sparkDefaultCatalogTableName(baseTableName); String sparkTableDefinition = "" + @@ -1431,7 +1431,7 @@ public void testMigratedDataWithAlteredSchema(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testMigratedDataWithPartialNameMapping(StorageFormat storageFormat) { - String baseTableName = "test_migrated_data_with_partial_name_mapping_" + randomTableSuffix(); + String baseTableName = "test_migrated_data_with_partial_name_mapping_" + randomNameSuffix(); String defaultCatalogTableName = sparkDefaultCatalogTableName(baseTableName); String sparkTableDefinition = "CREATE TABLE %s (a INT, b INT) USING " + storageFormat.name() + " OPTIONS ('compression'='snappy')"; @@ -1454,7 +1454,7 @@ public void testMigratedDataWithPartialNameMapping(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testPartialStats() { - String tableName = "test_partial_stats_" + randomTableSuffix(); + String tableName = "test_partial_stats_" + randomNameSuffix(); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1474,7 +1474,7 @@ public void testPartialStats() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testStatsAfterAddingPartitionField() { - String tableName = "test_stats_after_adding_partition_field_" + randomTableSuffix(); + String tableName = "test_stats_after_adding_partition_field_" + randomNameSuffix(); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1500,7 +1500,7 @@ public void testStatsAfterAddingPartitionField() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "tableFormatWithDeleteFormat") public void testTrinoReadsSparkRowLevelDeletes(StorageFormat tableStorageFormat, StorageFormat deleteFileStorageFormat) { - String tableName = format("test_trino_reads_spark_row_level_deletes_%s_%s_%s", tableStorageFormat.name(), deleteFileStorageFormat.name(), randomTableSuffix()); + String tableName = format("test_trino_reads_spark_row_level_deletes_%s_%s_%s", tableStorageFormat.name(), deleteFileStorageFormat.name(), randomNameSuffix()); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1533,7 +1533,7 @@ public void testTrinoReadsSparkRowLevelDeletes(StorageFormat tableStorageFormat, @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "tableFormatWithDeleteFormat") public void testTrinoReadsSparkRowLevelDeletesWithRowTypes(StorageFormat tableStorageFormat, StorageFormat deleteFileStorageFormat) { - String tableName = format("test_trino_reads_spark_row_level_deletes_row_types_%s_%s_%s", tableStorageFormat.name(), deleteFileStorageFormat.name(), randomTableSuffix()); + String tableName = format("test_trino_reads_spark_row_level_deletes_row_types_%s_%s_%s", tableStorageFormat.name(), deleteFileStorageFormat.name(), randomNameSuffix()); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1558,7 +1558,7 @@ public void testTrinoReadsSparkRowLevelDeletesWithRowTypes(StorageFormat tableSt @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testSparkReadsTrinoRowLevelDeletes(StorageFormat storageFormat) { - String tableName = format("test_spark_reads_trino_row_level_deletes_%s_%s", storageFormat.name(), randomTableSuffix()); + String tableName = format("test_spark_reads_trino_row_level_deletes_%s_%s", storageFormat.name(), randomNameSuffix()); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1585,7 +1585,7 @@ public void testSparkReadsTrinoRowLevelDeletes(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testSparkReadsTrinoRowLevelDeletesWithRowTypes(StorageFormat storageFormat) { - String tableName = format("test_spark_reads_trino_row_level_deletes_row_types_%s_%s", storageFormat.name(), randomTableSuffix()); + String tableName = format("test_spark_reads_trino_row_level_deletes_row_types_%s_%s", storageFormat.name(), randomNameSuffix()); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); @@ -1604,7 +1604,7 @@ public void testSparkReadsTrinoRowLevelDeletesWithRowTypes(StorageFormat storage @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testDeleteAfterPartitionEvolution(StorageFormat storageFormat) { - String baseTableName = "test_delete_after_partition_evolution_" + storageFormat + randomTableSuffix(); + String baseTableName = "test_delete_after_partition_evolution_" + storageFormat + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -1657,7 +1657,7 @@ public void testDeleteAfterPartitionEvolution(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testMissingMetrics() { - String tableName = "test_missing_metrics_" + randomTableSuffix(); + String tableName = "test_missing_metrics_" + randomNameSuffix(); String sparkTableName = sparkTableName(tableName); onSpark().executeQuery("CREATE TABLE " + sparkTableName + " (name STRING, country STRING) USING ICEBERG " + "PARTITIONED BY (country) TBLPROPERTIES ('write.metadata.metrics.default'='none')"); @@ -1671,7 +1671,7 @@ public void testMissingMetrics() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testOptimizeOnV2IcebergTable() { - String tableName = format("test_optimize_on_v2_iceberg_table_%s", randomTableSuffix()); + String tableName = format("test_optimize_on_v2_iceberg_table_%s", randomNameSuffix()); String sparkTableName = sparkTableName(tableName); String trinoTableName = trinoTableName(tableName); onSpark().executeQuery("CREATE TABLE " + sparkTableName + "(a INT, b INT) " + @@ -1687,7 +1687,7 @@ public void testOptimizeOnV2IcebergTable() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testAlterTableExecuteProceduresOnEmptyTable() { - String baseTableName = "test_alter_table_execute_procedures_on_empty_table_" + randomTableSuffix(); + String baseTableName = "test_alter_table_execute_procedures_on_empty_table_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -1943,7 +1943,7 @@ public void testCleaningUpIcebergTableWithRowLevelDeletes(StorageFormat tableSto @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testUpdateAfterSchemaEvolution() { - String baseTableName = "test_update_after_schema_evolution_" + randomTableSuffix(); + String baseTableName = "test_update_after_schema_evolution_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); onTrino().executeQuery("DROP TABLE IF EXISTS " + trinoTableName); @@ -1985,7 +1985,7 @@ public void testUpdateAfterSchemaEvolution() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testUpdateOnPartitionColumn() { - String baseTableName = "test_update_on_partition_column" + randomTableSuffix(); + String baseTableName = "test_update_on_partition_column" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); onTrino().executeQuery("DROP TABLE IF EXISTS " + trinoTableName); @@ -2018,7 +2018,7 @@ public void testUpdateOnPartitionColumn() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testAddNotNullColumn() { - String baseTableName = "test_add_not_null_column_" + randomTableSuffix(); + String baseTableName = "test_add_not_null_column_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2037,7 +2037,7 @@ public void testAddNotNullColumn() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testHandlingPartitionSchemaEvolutionInPartitionMetadata() { - String baseTableName = "test_handling_partition_schema_evolution_" + randomTableSuffix(); + String baseTableName = "test_handling_partition_schema_evolution_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2127,7 +2127,7 @@ public void testHandlingPartitionSchemaEvolutionInPartitionMetadata() public void testMetadataCompressionCodecGzip() { // Verify that Trino can read and write to a table created by Spark - String baseTableName = "test_metadata_compression_codec_gzip" + randomTableSuffix(); + String baseTableName = "test_metadata_compression_codec_gzip" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2183,7 +2183,7 @@ private void validatePartitioning(String baseTableName, String sparkTableName, L @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testTrinoAnalyze() { - String baseTableName = "test_trino_analyze_" + randomTableSuffix(); + String baseTableName = "test_trino_analyze_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); onTrino().executeQuery("DROP TABLE IF EXISTS " + trinoTableName); @@ -2202,7 +2202,7 @@ public void testTrinoAnalyze() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) public void testTrinoAnalyzeWithNonLowercaseColumnName() { - String baseTableName = "test_trino_analyze_with_uppercase_filed" + randomTableSuffix(); + String baseTableName = "test_trino_analyze_with_uppercase_filed" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2222,7 +2222,7 @@ public void testTrinoAnalyzeWithNonLowercaseColumnName() public void testRegisterTableWithTableLocation(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_table_location_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_table_location_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2246,7 +2246,7 @@ public void testRegisterTableWithTableLocation(StorageFormat storageFormat) public void testRegisterTableWithComments(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_comments_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_comments_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2275,7 +2275,7 @@ public void testRegisterTableWithComments(StorageFormat storageFormat) public void testRegisterTableWithShowCreateTable(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_show_create_table_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_show_create_table_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2307,7 +2307,7 @@ public void testRegisterTableWithShowCreateTable(StorageFormat storageFormat) public void testRegisterTableWithReInsert(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_re_insert_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_re_insert_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2332,7 +2332,7 @@ public void testRegisterTableWithReInsert(StorageFormat storageFormat) @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "storageFormats") public void testRegisterTableWithDroppedTable(StorageFormat storageFormat) { - String baseTableName = "test_register_table_with_dropped_table_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_dropped_table_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2354,7 +2354,7 @@ public void testRegisterTableWithDroppedTable(StorageFormat storageFormat) public void testRegisterTableWithDifferentTableName(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_different_table_name_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_different_table_name_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); @@ -2382,7 +2382,7 @@ public void testRegisterTableWithDifferentTableName(StorageFormat storageFormat) public void testRegisterTableWithMetadataFile(StorageFormat storageFormat) throws TException { - String baseTableName = "test_register_table_with_metadata_file_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomTableSuffix(); + String baseTableName = "test_register_table_with_metadata_file_" + storageFormat.name().toLowerCase(ENGLISH) + "_" + randomNameSuffix(); String trinoTableName = trinoTableName(baseTableName); String sparkTableName = sparkTableName(baseTableName); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkDropTableCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkDropTableCompatibility.java index 74b30f058199..38044e358902 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkDropTableCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkDropTableCompatibility.java @@ -28,11 +28,11 @@ import java.util.List; import java.util.stream.Collectors; +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.hive.Engine.SPARK; import static io.trino.tests.product.hive.Engine.TRINO; -import static io.trino.tests.product.hive.util.TemporaryHiveTable.randomTableSuffix; import static io.trino.tests.product.iceberg.util.IcebergTestUtils.getTableLocation; import static io.trino.tests.product.iceberg.util.IcebergTestUtils.stripNamenodeURI; import static io.trino.tests.product.utils.QueryExecutors.onSpark; @@ -70,7 +70,7 @@ public static Object[][] tableCleanupEngineConfigurations() @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}, dataProvider = "tableCleanupEngineConfigurations") public void testCleanupOnDropTable(Engine tableCreatorEngine, Engine tableDropperEngine) { - String tableName = "test_cleanup_on_drop_table" + randomTableSuffix(); + String tableName = "test_cleanup_on_drop_table" + randomNameSuffix(); tableCreatorEngine.queryExecutor().executeQuery("CREATE TABLE " + tableName + "(col0 INT, col1 INT)"); onTrino().executeQuery("INSERT INTO " + tableName + " VALUES (1, 2)"); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/TestingNames.java b/testing/trino-testing-services/src/main/java/io/trino/testing/TestingNames.java similarity index 100% rename from testing/trino-testing/src/main/java/io/trino/testing/TestingNames.java rename to testing/trino-testing-services/src/main/java/io/trino/testing/TestingNames.java