diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 59e9c9d75f5e..1f0726e30d0d 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -337,8 +337,7 @@ public void testCaseSensitiveAggregationPushdown() PlanMatchPattern aggregationOverTableScan = node(AggregationNode.class, node(TableScanNode.class)); PlanMatchPattern groupingAggregationOverTableScan = node(AggregationNode.class, node(TableScanNode.class)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_cs_agg_pushdown", "(a_string varchar(1), a_char char(1), a_bigint bigint)", ImmutableList.of( @@ -666,7 +665,7 @@ public void testCountDistinctWithStringTypes() .setSystemProperty(DISTINCT_AGGREGATIONS_STRATEGY, "pre_aggregate") .build(); - try (TestTable testTable = new TestTable(getQueryRunner()::execute, "distinct_strings", "(t_char CHAR(5), t_varchar VARCHAR(5))", rows)) { + try (TestTable testTable = newTrinoTable("distinct_strings", "(t_char CHAR(5), t_varchar VARCHAR(5))", rows)) { if (!(hasBehavior(SUPPORTS_AGGREGATION_PUSHDOWN) && hasBehavior(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY))) { // disabling hash generation to prevent extra projections in the plan which make it hard to write matchers for isNotFullyPushedDown Session optimizeHashGenerationDisabled = Session.builder(getSession()) @@ -1121,8 +1120,7 @@ public void testNullSensitiveTopNPushdown() return; } - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_null_sensitive_topn_pushdown", "(name varchar(10), a bigint)", List.of( @@ -1186,8 +1184,7 @@ public void testCaseSensitiveTopNPushdown() boolean expectTopNPushdown = hasBehavior(SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR); PlanMatchPattern topNOverTableScan = project(node(TopNNode.class, anyTree(node(TableScanNode.class)))); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_case_sensitive_topn_pushdown", "(a_string varchar(10), a_char char(10), a_bigint bigint)", List.of( @@ -1278,9 +1275,8 @@ public void testJoinPushdown() return; } - try (TestTable nationLowercaseTable = new TestTable( + try (TestTable nationLowercaseTable = newTrinoTable( // If a connector supports Join pushdown, but does not allow CTAS, we need to make the table creation here overridable. - getQueryRunner()::execute, "nation_lowercase", "AS SELECT nationkey, lower(name) name, regionkey FROM nation")) { for (JoinOperator joinOperator : JoinOperator.values()) { @@ -1547,12 +1543,10 @@ public void testBulkColumnListingOptions() String schemaName = "test_columns_listing_" + randomNameSuffix(); assertUpdate("CREATE SCHEMA " + schemaName); try { - try (TestTable newNation = new TestTable( - getQueryRunner()::execute, + try (TestTable newNation = newTrinoTable( schemaName + ".nation", "(name varchar(25), nationkey bigint)"); - TestTable newRegion = new TestTable( - getQueryRunner()::execute, + TestTable newRegion = newTrinoTable( schemaName + ".region", "(name varchar(25), regionkey bigint)")) { if (hasBehavior(SUPPORTS_COMMENT_ON_TABLE)) { @@ -1755,7 +1749,7 @@ public void testUpdateNotNullColumn() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "update_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { + try (TestTable table = newTrinoTable("update_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { assertUpdate(format("INSERT INTO %s (nullable_col, not_null_col) VALUES (1, 10)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (1, 10)"); assertQueryFails("UPDATE " + table.getName() + " SET not_null_col = NULL WHERE nullable_col = 1", MODIFYING_ROWS_MESSAGE); @@ -1774,7 +1768,7 @@ public void testUpdateRowType() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_with_predicates_on_row_types", "(int_t INT, row_t ROW(f1 INT, f2 INT))")) { + try (TestTable table = newTrinoTable("test_update_with_predicates_on_row_types", "(int_t INT, row_t ROW(f1 INT, f2 INT))")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES (1, ROW(2, 3)), (11, ROW(12, 13)), (21, ROW(22, 23))", 3); assertQueryFails("UPDATE " + tableName + " SET int_t = int_t - 1 WHERE row_t.f2 = 3", MODIFYING_ROWS_MESSAGE); @@ -1793,7 +1787,7 @@ public void testUpdateRowConcurrently() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_row", "(a INT, b INT, c INT)", ImmutableList.of("1, 2, 3"))) { + try (TestTable table = newTrinoTable("test_update_row", "(a INT, b INT, c INT)", ImmutableList.of("1, 2, 3"))) { assertQueryFails("UPDATE " + table.getName() + " SET a = a + 1", MODIFYING_ROWS_MESSAGE); } } @@ -1809,7 +1803,7 @@ public void testUpdateAllValues() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_all", "(a INT, b INT, c INT)", ImmutableList.of("1, 2, 3"))) { + try (TestTable table = newTrinoTable("test_update_all", "(a INT, b INT, c INT)", ImmutableList.of("1, 2, 3"))) { assertUpdate("UPDATE " + table.getName() + " SET a = 1, b = 1, c = 2", 1); } } @@ -1826,7 +1820,7 @@ public void testUpdateWithPredicates() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_predicates", "(a INT, b INT, c INT)")) { + try (TestTable table = newTrinoTable("test_row_predicates", "(a INT, b INT, c INT)")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES (1, 2, 3), (11, 12, 13), (21, 22, 23)", 3); assertUpdate("UPDATE " + tableName + " SET a = 5 WHERE c = 3", 1); @@ -1861,7 +1855,7 @@ public void testUpdateWithPredicates() public void testConstantUpdateWithVarcharEqualityPredicates() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_varchar", "(col1 INT, col2 varchar(1))", ImmutableList.of("1, 'a'", "2, 'A'"))) { + try (TestTable table = newTrinoTable("test_update_varchar", "(col1 INT, col2 varchar(1))", ImmutableList.of("1, 'a'", "2, 'A'"))) { if (!hasBehavior(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_EQUALITY)) { assertQueryFails("UPDATE " + table.getName() + " SET col1 = 20 WHERE col2 = 'A'", MODIFYING_ROWS_MESSAGE); return; @@ -1910,7 +1904,7 @@ public void testDeleteWithBigintEqualityPredicate() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)); // TODO (https://github.com/trinodb/trino/issues/5901) Use longer table name once Oracle version is updated - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_bigint", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_delete_bigint", "AS SELECT * FROM region")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE regionkey = 1", 1); assertQuery( "SELECT regionkey, name FROM " + table.getName(), @@ -1927,7 +1921,7 @@ public void testDeleteWithVarcharEqualityPredicate() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)); // TODO (https://github.com/trinodb/trino/issues/5901) Use longer table name once Oracle version is updated - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_varchar", "(col varchar(1))", ImmutableList.of("'a'", "'A'", "null"))) { + try (TestTable table = newTrinoTable("test_delete_varchar", "(col varchar(1))", ImmutableList.of("'a'", "'A'", "null"))) { if (!hasBehavior(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_EQUALITY)) { assertQueryFails("DELETE FROM " + table.getName() + " WHERE col = 'A'", MODIFYING_ROWS_MESSAGE); return; @@ -2042,8 +2036,7 @@ public void testInsertWithoutTemporaryTable() .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "non_transactional_insert", "false") .build(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_bypass_temp", "(a varchar(36), b bigint)")) { int numberOfRows = 50; @@ -2072,8 +2065,7 @@ private void testWriteBatchSizeSessionProperty(int batchSize, int numberOfRows) .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "write_batch_size", Integer.toString(batchSize)) .build(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "write_batch_size", "(a varchar(36), b bigint)")) { String values = String.join(",", buildRowsForInsert(numberOfRows)); @@ -2103,8 +2095,7 @@ private void testWriteTaskParallelismSessionProperty(int parallelism, int number .build(); QueryRunner queryRunner = getQueryRunner(); - try (TestTable table = new TestTable( - queryRunner::execute, + try (TestTable table = newTrinoTable( "write_parallelism", "(a varchar(128), b bigint)")) { Plan plan = newTransaction() @@ -2274,8 +2265,8 @@ public void testJoinPushdownWithLongIdentifiers() .orElse(65536 + 5); String validColumnName = baseColumnName + "z".repeat(maxLength - baseColumnName.length()); - try (TestTable left = new TestTable(getQueryRunner()::execute, "test_long_id_l", format("(%s BIGINT)", validColumnName)); - TestTable right = new TestTable(getQueryRunner()::execute, "test_long_id_r", format("(%s BIGINT)", validColumnName))) { + try (TestTable left = newTrinoTable("test_long_id_l", format("(%s BIGINT)", validColumnName)); + TestTable right = newTrinoTable("test_long_id_r", format("(%s BIGINT)", validColumnName))) { assertThat(query(joinPushdownEnabled(getSession()), """ SELECT l.%1$s, r.%1$s @@ -2403,8 +2394,7 @@ public void testDynamicFilteringCaseInsensitiveDomainCompaction() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); skipTestUnless(hasBehavior(SUPPORTS_DYNAMIC_FILTER_PUSHDOWN)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_caseinsensitive", "(id varchar(1))", ImmutableList.of("'0'", "'a'", "'B'"))) { diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcTableStatisticsTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcTableStatisticsTest.java index 243038d66d3e..8cfd98769680 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcTableStatisticsTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcTableStatisticsTest.java @@ -171,8 +171,7 @@ public void testStatsWithVarcharPredicatePushdown() "('comment', 1e0, 0e0, null)," + "(null, null, null, 1e0)"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "varchar_duplicates", // each letter A-E repeated 5 times " AS SELECT nationkey, chr(codepoint('A') + nationkey / 5) fl FROM tpch.tiny.nation")) { diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java index 48646ef1c81f..f81bbac4a50a 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java @@ -205,7 +205,7 @@ public void testCreateTableWithRowTypeWithoutField() @Test public void testCreateTableAlreadyExists() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_table_already_exists", "(col1 int)")) { + try (TestTable table = newTrinoTable("test_create_table_already_exists", "(col1 int)")) { assertQueryFails( "CREATE TABLE " + table.getName() + "(col1 int)", "\\Qline 1:1: Table 'bigquery.tpch." + table.getName() + "' already exists\\E"); @@ -387,7 +387,7 @@ public void testStreamCommentTableSpecialCharacter() @Override // Override because the base test exceeds rate limits per a table public void testCommentColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column_", "(a integer)")) { + try (TestTable table = newTrinoTable("test_comment_column_", "(a integer)")) { // comment set assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS 'new comment'"); assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())).contains("COMMENT 'new comment'"); @@ -398,7 +398,7 @@ public void testCommentColumn() assertThat(getColumnComment(table.getName(), "a")).isEqualTo(null); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column_", "(a integer COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_comment_column_", "(a integer COMMENT 'test comment')")) { assertThat(getColumnComment(table.getName(), "a")).isEqualTo("test comment"); // comment set new value assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS 'updated comment'"); @@ -550,7 +550,7 @@ public void testPredicatePushdownPrunnedColumns() @Test public void testColumnPositionMismatch() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test.test_column_position_mismatch", "(c_varchar VARCHAR, c_int INT, c_date DATE)")) { + try (TestTable table = newTrinoTable("test.test_column_position_mismatch", "(c_varchar VARCHAR, c_int INT, c_date DATE)")) { onBigQuery("INSERT INTO " + table.getName() + " VALUES ('a', 1, '2021-01-01')"); // Adding a CAST makes BigQuery return columns in a different order assertQuery("SELECT c_varchar, CAST(c_int AS SMALLINT), c_date FROM " + table.getName(), "VALUES ('a', 1, '2021-01-01')"); @@ -1458,7 +1458,7 @@ private void assertLimitPushdownReadsLessData(Session session, String tableName) public void testInsertArray() { // Override because the connector disallows writing a NULL in ARRAY - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_array_", "(a ARRAY, b ARRAY)")) { + try (TestTable table = newTrinoTable("test_insert_array_", "(a ARRAY, b ARRAY)")) { assertUpdate("INSERT INTO " + table.getName() + " (a, b) VALUES (ARRAY[1.23E1], ARRAY[1.23E1])", 1); assertQuery("SELECT a[1], b[1] FROM " + table.getName(), "VALUES (12.3, 12)"); } diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java index 9724da1b9b55..d23d22a4d099 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java @@ -738,7 +738,7 @@ public void testArray() @Test public void testArrayType() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_array_", "(a BIGINT, b ARRAY, c ARRAY)")) { + try (TestTable table = newTrinoTable("test_array_", "(a BIGINT, b ARRAY, c ARRAY)")) { assertUpdate("INSERT INTO " + table.getName() + " (a, b, c) VALUES (5, ARRAY[1.23E1], ARRAY[15]), (6, ARRAY[1.24E1, 1.27E1, 2.23E1], ARRAY[25, 26, 36])", 2); assertThat(query("SELECT * FROM " + table.getName())) .matches("VALUES " + @@ -752,7 +752,7 @@ public void testUnsupportedNullArray() { // BigQuery translates a NULL ARRAY into an empty ARRAY in the query result // This test ensures that the connector disallows writing a NULL ARRAY - try (TestTable table = new TestTable(getQueryRunner()::execute, "test.test_null_array", "(col ARRAY(INT))")) { + try (TestTable table = newTrinoTable("test.test_null_array", "(col ARRAY(INT))")) { assertQueryFails("INSERT INTO " + table.getName() + " VALUES (NULL)", "NULL value not allowed for NOT NULL column: col"); } } diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryWithProxyTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryWithProxyTest.java index a875ff62f53d..cd98f539c8c2 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryWithProxyTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryWithProxyTest.java @@ -48,7 +48,7 @@ protected QueryRunner createQueryRunner() void testCreateTableAsSelect() { // This test covers all client (BigQuery, BigQueryReadClient and BigQueryWriteClient) - try (TestTable table = new TestTable(getQueryRunner()::execute, "test.test_ctas", "AS SELECT 42 x")) { + try (TestTable table = newTrinoTable("test.test_ctas", "AS SELECT 42 x")) { assertQuery("SELECT * FROM " + table.getName(), "VALUES 42"); } } diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraConnectorSmokeTest.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraConnectorSmokeTest.java index e4dbdc5fc0b4..a9d4ce254ee7 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraConnectorSmokeTest.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/BaseCassandraConnectorSmokeTest.java @@ -76,7 +76,7 @@ public void testRowLevelDelete() @Test public void testInsertDate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", "(a_date date)")) { + try (TestTable table = newTrinoTable("test_insert_", "(a_date date)")) { assertUpdate("INSERT INTO " + table.getName() + " (a_date) VALUES ( DATE '2020-05-11')", 1); assertThat(query("SELECT a_date FROM " + table.getName())).matches("VALUES (DATE '2020-05-11')"); } diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraProtocolVersionV3ConnectorSmokeTest.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraProtocolVersionV3ConnectorSmokeTest.java index 428291628ea1..7ba3a1e68795 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraProtocolVersionV3ConnectorSmokeTest.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraProtocolVersionV3ConnectorSmokeTest.java @@ -43,7 +43,7 @@ protected QueryRunner createQueryRunner() @Override public void testInsertDate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", "(a_date date)")) { + try (TestTable table = newTrinoTable("test_insert_", "(a_date date)")) { assertUpdate("INSERT INTO " + table.getName() + " (a_date) VALUES ('2020-05-11')", 1); assertThat(query("SELECT a_date FROM " + table.getName())).matches("VALUES (CAST('2020-05-11' AS varchar))"); } diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java index 2444ea4bf6c6..90f9744c288d 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java @@ -889,7 +889,7 @@ private void testUnsupportedDate(String unsupportedDate) String minSupportedDate = "1970-01-01"; String maxSupportedDate = "2149-06-06"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_unsupported_date", "(dt date)")) { + try (TestTable table = newTrinoTable("test_unsupported_date", "(dt date)")) { assertQueryFails( format("INSERT INTO %s VALUES (DATE '%s')", table.getName(), unsupportedDate), format("Date must be between %s and %s in ClickHouse: %s", minSupportedDate, maxSupportedDate, unsupportedDate)); @@ -984,7 +984,7 @@ public void testUnsupportedTimestamp(String unsupportedTimestamp) String minSupportedTimestamp = "1970-01-01 00:00:00"; String maxSupportedTimestamp = "2106-02-07 06:28:15"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_unsupported_timestamp", "(dt timestamp(0))")) { + try (TestTable table = newTrinoTable("test_unsupported_timestamp", "(dt timestamp(0))")) { assertQueryFails( format("INSERT INTO %s VALUES (TIMESTAMP '%s')", table.getName(), unsupportedTimestamp), format("Timestamp must be between %s and %s in ClickHouse: %s", minSupportedTimestamp, maxSupportedTimestamp, unsupportedTimestamp)); diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 6f5e0093c65b..4069feced7f8 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -126,8 +126,7 @@ public void testRenameColumn() @Override public void testRenameColumnWithComment() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_rename_column_", "(id INT NOT NULL, col INT COMMENT 'test column comment') WITH (engine = 'MergeTree', order_by = ARRAY['id'])")) { assertThat(getColumnComment(table.getName(), "col")).isEqualTo("test column comment"); @@ -141,7 +140,7 @@ public void testRenameColumnWithComment() public void testAddColumnWithCommentSpecialCharacter(String comment) { // Override because default storage engine doesn't support renaming columns - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_", "(a_varchar varchar NOT NULL) WITH (engine = 'mergetree', order_by = ARRAY['a_varchar'])")) { + try (TestTable table = newTrinoTable("test_add_column_", "(a_varchar varchar NOT NULL) WITH (engine = 'mergetree', order_by = ARRAY['a_varchar'])")) { assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN b_varchar varchar COMMENT " + varcharLiteral(comment)); assertThat(getColumnComment(table.getName(), "b_varchar")).isEqualTo(comment); } @@ -151,7 +150,7 @@ public void testAddColumnWithCommentSpecialCharacter(String comment) @Override public void testDropAndAddColumnWithSameName() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_add_column", "(x int NOT NULL, y int, z int) WITH (engine = 'MergeTree', order_by = ARRAY['x'])", ImmutableList.of("1,2,3"))) { + try (TestTable table = newTrinoTable("test_drop_add_column", "(x int NOT NULL, y int, z int) WITH (engine = 'MergeTree', order_by = ARRAY['x'])", ImmutableList.of("1,2,3"))) { assertUpdate("ALTER TABLE " + table.getName() + " DROP COLUMN y"); assertQuery("SELECT * FROM " + table.getName(), "VALUES (1, 3)"); @@ -210,7 +209,7 @@ public void testDropColumn() @Override protected TestTable createTableWithOneIntegerColumn(String namePrefix) { - return new TestTable(getQueryRunner()::execute, namePrefix, "(col integer NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['col'])"); + return newTrinoTable(namePrefix, "(col integer NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['col'])"); } @Override @@ -223,7 +222,7 @@ protected String tableDefinitionForAddColumn() @Override // Overridden because the default storage type doesn't support adding columns public void testAddNotNullColumnToEmptyTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col_to_empty", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { + try (TestTable table = newTrinoTable("test_add_notnull_col_to_empty", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { String tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); @@ -239,7 +238,7 @@ public void testAddNotNullColumnToEmptyTable() @Override // Overridden because (a) the default storage type doesn't support adding columns and (b) ClickHouse has implicit default value for new NON NULL column public void testAddNotNullColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { + try (TestTable table = newTrinoTable("test_add_notnull_col", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { String tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); @@ -259,7 +258,7 @@ public void testAddNotNullColumn() public void testAddColumnWithComment() { // Override because the default storage type doesn't support adding columns - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_col_desc_", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { + try (TestTable table = newTrinoTable("test_add_col_desc_", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { String tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar COMMENT 'test new column comment'"); @@ -532,8 +531,7 @@ public void testTableProperty() public void testSetTableProperties() throws Exception { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_alter_table_properties", "(p1 int NOT NULL, p2 boolean NOT NULL, x VARCHAR) WITH (engine = 'MergeTree', order_by = ARRAY['p1', 'p2'], primary_key = ARRAY['p1', 'p2'])")) { assertThat(getTableProperties("tpch", table.getName())) @@ -558,8 +556,7 @@ public void testSetTableProperties() @Test public void testAlterInvalidTableProperties() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_alter_table_properties", "(p1 int NOT NULL, p2 int NOT NULL, x VARCHAR) WITH (engine = 'MergeTree', order_by = ARRAY['p1', 'p2'], primary_key = ARRAY['p1', 'p2'])")) { assertQueryFails( @@ -644,7 +641,7 @@ protected TestTable createTableWithDoubleAndRealColumns(String name, List> futures = new ArrayList<>(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace", "(col integer)")) { + try (TestTable table = newTrinoTable("test_create_or_replace", "(col integer)")) { String tableName = table.getName(); getQueryRunner().execute("CREATE OR REPLACE TABLE " + tableName + " AS SELECT 1 a"); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAnalyze.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAnalyze.java index ac3f29b7413e..b17d8344812b 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAnalyze.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAnalyze.java @@ -451,8 +451,7 @@ public void testAnalyzeSomeColumns() @Test public void testDropExtendedStats() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_extended_stats", "AS SELECT * FROM tpch.sf1.nation")) { String query = "SHOW STATS FOR " + table.getName(); @@ -485,8 +484,7 @@ public void testDropExtendedStats() @Test public void testDropMissingStats() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_missing_stats", "AS SELECT * FROM tpch.sf1.nation")) { // When there are no extended stats, the procedure should have no effect @@ -506,8 +504,7 @@ public void testDropMissingStats() @Test public void testDropStatsAccessControl() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_deny_drop_stats", "AS SELECT * FROM tpch.sf1.nation")) { assertAccessDenied( @@ -524,8 +521,7 @@ public void testDropStatsAccessControl() @Test public void testStatsOnTpcDsData() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_old_date_stats", "AS SELECT d_date FROM tpcds.tiny.date_dim")) { assertUpdate("ANALYZE " + table.getName()); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java index f02b6c6e10e2..617514e09f41 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java @@ -347,8 +347,7 @@ private void testPartitionValuesParsedCheckpoint(ColumnMappingMode columnMapping checkArgument(inputValues.size() == 2, "inputValues size must be 2"); checkArgument(expectedPartitionValuesParsed.size() == 2, "expectedPartitionValuesParsed size must be 2"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_values_parsed_checkpoint", "(x int, part " + inputType + ") WITH (checkpoint_interval = 2, column_mapping_mode = '" + columnMappingMode + "', partitioned_by = ARRAY['part'])")) { for (String inputValue : inputValues) { @@ -1199,7 +1198,7 @@ public void testDeletionVectorsEnabledCreateTable() private void testDeletionVectorsEnabledCreateTable(String tableDefinition) throws Exception { - try (TestTable table = new TestTable(getQueryRunner()::execute, "deletion_vectors", tableDefinition)) { + try (TestTable table = newTrinoTable("deletion_vectors", tableDefinition)) { assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())) .contains("deletion_vectors_enabled = true"); @@ -1232,7 +1231,7 @@ public void testDeletionVectorsDisabledCreateTable() private void testDeletionVectorsDisabledCreateTable(String tableDefinition) throws Exception { - try (TestTable table = new TestTable(getQueryRunner()::execute, "deletion_vectors", tableDefinition)) { + try (TestTable table = newTrinoTable("deletion_vectors", tableDefinition)) { assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())) .doesNotContain("deletion_vectors_enabled"); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java index 943caec8b254..11f1544e92bc 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java @@ -344,8 +344,7 @@ public void testPartialFilterWhenPartitionColumnOrderIsDifferentFromTableDefinit private void testPartialFilterWhenPartitionColumnOrderIsDifferentFromTableDefinition(ColumnMappingMode columnMappingMode) { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_delete_with_partial_filter_composed_partition", "(_bigint BIGINT, _date DATE, _varchar VARCHAR) WITH (column_mapping_mode='" + columnMappingMode + "', partitioned_by = ARRAY['_varchar', '_date'])")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1, CAST('2019-09-10' AS DATE), 'a'), (2, CAST('2019-09-10' AS DATE), 'a')", 2); @@ -403,8 +402,7 @@ public void testInsertIntoUnsupportedVarbinaryPartitionType() { // TODO https://github.com/trinodb/trino/issues/24155 Cannot insert varbinary values into partitioned columns // Update TestDeltaLakeBasic.testPartitionValuesParsedCheckpoint() when fixing this issue - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_varbinary_partition", "(x int, part varbinary) WITH (partitioned_by = ARRAY['part'])")) { assertQueryFails("INSERT INTO " + table.getName() + " VALUES (1, X'01')", "Unsupported type for partition: varbinary"); @@ -588,8 +586,7 @@ public void testRenameColumnName() public void testCharVarcharComparison() { // with char->varchar coercion on table creation, this is essentially varchar/varchar comparison - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_char_varchar", "(k, v) AS VALUES" + " (-1, CAST(NULL AS CHAR(3))), " + @@ -818,7 +815,7 @@ public void testTimestampWithTimeZoneOptimization() @Test public void testShowStatsForTimestampWithTimeZone() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_stats_timestamptz_", "(x TIMESTAMP(3) WITH TIME ZONE) WITH (checkpoint_interval = 2)")) { + try (TestTable table = newTrinoTable("test_stats_timestamptz_", "(x TIMESTAMP(3) WITH TIME ZONE) WITH (checkpoint_interval = 2)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (TIMESTAMP '+10000-01-02 13:34:56.123 +01:00')", 1); assertThat(query("SHOW STATS FOR " + table.getName())) .result() @@ -843,7 +840,7 @@ public void testShowStatsForTimestampWithTimeZone() @Test public void testAddColumnToPartitionedTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_partitioned_table_", "(x VARCHAR, part VARCHAR) WITH (partitioned_by = ARRAY['part'])")) { + try (TestTable table = newTrinoTable("test_add_column_partitioned_table_", "(x VARCHAR, part VARCHAR) WITH (partitioned_by = ARRAY['part'])")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first', 'part-0001'", 1); assertQueryFails("ALTER TABLE " + table.getName() + " ADD COLUMN x bigint", ".* Column 'x' already exists"); assertQueryFails("ALTER TABLE " + table.getName() + " ADD COLUMN part bigint", ".* Column 'part' already exists"); @@ -877,7 +874,7 @@ private QueryInfo getQueryInfo(QueryRunner queryRunner, MaterializedResultWithPl @Test public void testAddColumnAndOptimize() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_and_optimize", "(x VARCHAR)")) { + try (TestTable table = newTrinoTable("test_add_column_and_optimize", "(x VARCHAR)")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first'", 1); assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN a varchar(50)"); @@ -905,7 +902,7 @@ public void testAddColumnAndVacuum() .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "vacuum_min_retention", "0s") .build(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_and_optimize", "(x VARCHAR)")) { + try (TestTable table = newTrinoTable("test_add_column_and_optimize", "(x VARCHAR)")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first'", 1); assertUpdate("INSERT INTO " + table.getName() + " SELECT 'second'", 1); @@ -1009,7 +1006,7 @@ public void testTargetMaxFileSize() @Test public void testPathColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_path_column", "(x VARCHAR)")) { + try (TestTable table = newTrinoTable("test_path_column", "(x VARCHAR)")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first'", 1); String firstFilePath = (String) computeScalar("SELECT \"$path\" FROM " + table.getName()); assertUpdate("INSERT INTO " + table.getName() + " SELECT 'second'", 1); @@ -1331,11 +1328,11 @@ private void testTableWithNonNullableColumns(ColumnMappingMode mode) public void testCreateTableWithChangeDataFeedColumnName() { for (String columnName : CHANGE_DATA_FEED_COLUMN_NAMES) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_table_cdf", "(" + columnName + " int)")) { + try (TestTable table = newTrinoTable("test_create_table_cdf", "(" + columnName + " int)")) { assertTableColumnNames(table.getName(), columnName); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_table_cdf", "AS SELECT 1 AS " + columnName)) { + try (TestTable table = newTrinoTable("test_create_table_cdf", "AS SELECT 1 AS " + columnName)) { assertTableColumnNames(table.getName(), columnName); } } @@ -1344,7 +1341,7 @@ public void testCreateTableWithChangeDataFeedColumnName() @Test public void testCreateTableWithChangeDataFeed() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cdf", "(x int) WITH (change_data_feed_enabled = true)")) { + try (TestTable table = newTrinoTable("test_cdf", "(x int) WITH (change_data_feed_enabled = true)")) { assertThat(query("SELECT * FROM \"" + table.getName() + "$properties\"")) .skippingTypesCheck() .matches("VALUES " + @@ -1355,7 +1352,7 @@ public void testCreateTableWithChangeDataFeed() } // timestamp type requires reader version 3 and writer version 7 - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cdf", "(x timestamp) WITH (change_data_feed_enabled = true)")) { + try (TestTable table = newTrinoTable("test_cdf", "(x timestamp) WITH (change_data_feed_enabled = true)")) { assertThat(query("SELECT * FROM \"" + table.getName() + "$properties\"")) .skippingTypesCheck() .matches("VALUES " + @@ -1371,8 +1368,7 @@ public void testCreateTableWithChangeDataFeed() @Test public void testChangeDataFeedWithDeletionVectors() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_cdf", "(x VARCHAR, y INT) WITH (change_data_feed_enabled = true, deletion_vectors_enabled = true)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES('test1', 1)", 1); @@ -1415,7 +1411,7 @@ public void testUnsupportedCreateTableWithChangeDataFeed() public void testUnsupportedAddColumnWithChangeDataFeed() { for (String columnName : CHANGE_DATA_FEED_COLUMN_NAMES) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column", "(col int) WITH (change_data_feed_enabled = true)")) { + try (TestTable table = newTrinoTable("test_add_column", "(col int) WITH (change_data_feed_enabled = true)")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ADD COLUMN " + columnName + " int", "\\QColumn name %s is forbidden when change data feed is enabled\\E".formatted(columnName)); @@ -1432,7 +1428,7 @@ public void testUnsupportedAddColumnWithChangeDataFeed() public void testUnsupportedRenameColumnWithChangeDataFeed() { for (String columnName : CHANGE_DATA_FEED_COLUMN_NAMES) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_rename_column", "(col int) WITH (change_data_feed_enabled = true)")) { + try (TestTable table = newTrinoTable("test_rename_column", "(col int) WITH (change_data_feed_enabled = true)")) { assertQueryFails( "ALTER TABLE " + table.getName() + " RENAME COLUMN col TO " + columnName, "Cannot rename column when change data feed is enabled"); @@ -1445,7 +1441,7 @@ public void testUnsupportedRenameColumnWithChangeDataFeed() public void testUnsupportedSetTablePropertyWithChangeDataFeed() { for (String columnName : CHANGE_DATA_FEED_COLUMN_NAMES) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_properties", "(" + columnName + " int)")) { + try (TestTable table = newTrinoTable("test_set_properties", "(" + columnName + " int)")) { assertQueryFails( "ALTER TABLE " + table.getName() + " SET PROPERTIES change_data_feed_enabled = true", "\\QUnable to enable change data feed because table contains [%s] columns\\E".formatted(columnName)); @@ -1534,7 +1530,7 @@ private void testCreateTableColumnMappingMode(ColumnMappingMode mode, Consumer catalogProperties = getSession().getCatalogProperties(getSession().getCatalog().orElseThrow()); assertThat(catalogProperties).doesNotContainKey("query_partition_filter_required"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_not_demanded", "(x varchar, part varchar) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("'a', 'part_a'", "'b', 'part_b'"))) { @@ -3717,8 +3697,7 @@ public void testPartitionFilterQueryNotDemanded() public void testQueryWithoutPartitionOnNonPartitionedTableNotDemanded() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_no_partition_table_", "(x varchar, part varchar)", ImmutableList.of("('a', 'part_a')", "('b', 'part_b')"))) { @@ -3731,8 +3710,7 @@ public void testQueryWithoutPartitionOnNonPartitionedTableNotDemanded() public void testQueryWithoutPartitionFilterNotAllowed() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_no_partition_filter_", "(x varchar, part varchar) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("('a', 'part_a')", "('b', 'part_b')"))) { @@ -3747,8 +3725,7 @@ public void testQueryWithoutPartitionFilterNotAllowed() public void testPartitionFilterRemovedByPlanner() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_removed_", "(x varchar, part varchar) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("('a', 'part_a')", "('b', 'part_b')"))) { @@ -3763,8 +3740,7 @@ public void testPartitionFilterRemovedByPlanner() public void testPartitionFilterIncluded() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_included", "(x varchar, part integer) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("('a', 1)", "('a', 2)", "('a', 3)", "('a', 4)", "('b', 1)", "('b', 2)", "('b', 3)", "('b', 4)"))) { @@ -3792,8 +3768,7 @@ public void testRequiredPartitionFilterOnJoin() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable leftTable = new TestTable( - getQueryRunner()::execute, + try (TestTable leftTable = newTrinoTable( "test_partition_left_", "(x varchar, part varchar)", ImmutableList.of("('a', 'part_a')")); @@ -3818,8 +3793,7 @@ public void testRequiredPartitionFilterOnJoinBothTablePartitioned() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable leftTable = new TestTable( - getQueryRunner()::execute, + try (TestTable leftTable = newTrinoTable( "test_partition_inferred_left_", "(x varchar, part varchar) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("('a', 'part_a')")); @@ -3843,8 +3817,7 @@ public void testRequiredPartitionFilterOnJoinBothTablePartitioned() public void testComplexPartitionPredicateWithCasting() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_predicate", "(x varchar, part varchar) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("('a', '1')", "('b', '2')"))) { @@ -3856,8 +3829,7 @@ public void testComplexPartitionPredicateWithCasting() public void testPartitionPredicateInOuterQuery() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_predicate", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("(1, 11)", "(2, 22)"))) { @@ -3869,8 +3841,7 @@ public void testPartitionPredicateInOuterQuery() public void testPartitionPredicateInInnerQuery() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_predicate", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("(1, 11)", "(2, 22)"))) { @@ -3882,8 +3853,7 @@ public void testPartitionPredicateInInnerQuery() public void testPartitionPredicateFilterAndAnalyzeOnPartitionedTable() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_predicate_analyze_", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("(1, 11)", "(2, 22)"))) { @@ -3898,8 +3868,7 @@ public void testPartitionPredicateFilterAndAnalyzeOnPartitionedTable() public void testPartitionPredicateFilterAndAnalyzeOnNonPartitionedTable() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable nonPartitioned = new TestTable( - getQueryRunner()::execute, + try (TestTable nonPartitioned = newTrinoTable( "test_partition_predicate_analyze_nonpartitioned", "(a integer, b integer) ", ImmutableList.of("(1, 11)", "(2, 22)"))) { @@ -3912,8 +3881,7 @@ public void testPartitionPredicateFilterAndAnalyzeOnNonPartitionedTable() public void testPartitionFilterMultiplePartition() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_multiple_partition_", "(x varchar, part1 integer, part2 integer) WITH (partitioned_by = ARRAY['part1', 'part2'])", ImmutableList.of("('a', 1, 1)", "('a', 1, 2)", "('a', 2, 1)", "('a', 2, 2)", "('b', 1, 1)", "('b', 1, 2)", "('b', 2, 1)", "('b', 2, 2)"))) { @@ -3942,8 +3910,7 @@ public void testPartitionFilterMultiplePartition() public void testPartitionFilterRequiredAndOptimize() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_optimize", "(part integer, name varchar(50)) WITH (partitioned_by = ARRAY['part'])", ImmutableList.of("(1, 'Bob')", "(2, 'Alice')"))) { @@ -3976,8 +3943,7 @@ public void testPartitionFilterRequiredAndOptimize() public void testPartitionFilterEnabledAndOptimizeForNonPartitionedTable() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_nonpartitioned_optimize", "(part integer, name varchar(50))", ImmutableList.of("(1, 'Bob')", "(2, 'Alice')"))) { @@ -4002,8 +3968,7 @@ public void testPartitionFilterEnabledAndOptimizeForNonPartitionedTable() public void testPartitionFilterRequiredAndWriteOperation() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_table_changes", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'], change_data_feed_enabled = true)", ImmutableList.of("(1, 11)", "(2, 22)", "(3, 33)"))) { @@ -4041,8 +4006,7 @@ public void testPartitionFilterRequiredAndWriteOperation() public void testPartitionFilterRequiredAndTableChanges() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_table_changes", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'], change_data_feed_enabled = true)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1, 11)", 1); @@ -4093,8 +4057,7 @@ public void testPartitionFilterRequiredAndTableChanges() public void testPartitionFilterRequiredAndHistoryTable() { Session session = sessionWithPartitionFilterRequirement(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partition_filter_table_changes", "(x integer, part integer) WITH (partitioned_by = ARRAY['part'], change_data_feed_enabled = true)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1, 11)", 1); @@ -4274,8 +4237,7 @@ public void testTypeCoercionOnCreateTable() private void testTimestampCoercionOnCreateTable(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table", "(ts TIMESTAMP)")) { assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (" + actualValue + ")", 1); @@ -4287,8 +4249,7 @@ private void testTimestampCoercionOnCreateTable(@Language("SQL") String actualVa private void testCharCoercionOnCreateTable(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table", "(vch VARCHAR)")) { assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (" + actualValue + ")", 1); @@ -4336,8 +4297,7 @@ public void testTypeCoercionOnCreateTableAsSelect() private void testTimestampCoercionOnCreateTableAsSelect(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table_as_select", "AS SELECT %s ts".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "ts")).isEqualTo("timestamp(6)"); @@ -4348,8 +4308,7 @@ private void testTimestampCoercionOnCreateTableAsSelect(@Language("SQL") String private void testCharCoercionOnCreateTableAsSelect(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table_as_select", "AS SELECT %s col".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "col")).isEqualTo("varchar"); @@ -4396,8 +4355,7 @@ public void testTypeCoercionOnCreateTableAsSelectWithNoData() private void testTimestampCoercionOnCreateTableAsSelectWithNoData(@Language("SQL") String actualValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table_as_select_with_no_data", "AS SELECT %s ts WITH NO DATA".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "ts")).isEqualTo("timestamp(6)"); @@ -4407,8 +4365,7 @@ private void testTimestampCoercionOnCreateTableAsSelectWithNoData(@Language("SQL private void testCharCoercionOnCreateTableAsSelectWithNoData(@Language("SQL") String actualValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table_as_select_with_no_data", "AS SELECT %s col WITH NO DATA".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "col")).isEqualTo("varchar"); @@ -4455,8 +4412,7 @@ public void testTypeCoercionOnCreateTableAsWithRowType() private void testTimestampCoercionOnCreateTableAsWithRowType(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table_as_with_row_type", "AS SELECT CAST(row(%s) AS row(value timestamp(6))) ts".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "ts")).isEqualTo("row(value timestamp(6))"); @@ -4469,8 +4425,7 @@ private void testTimestampCoercionOnCreateTableAsWithRowType(@Language("SQL") St private void testCharCoercionOnCreateTableAsWithRowType(@Language("SQL") String actualValue, @Language("SQL") String actualTypeLiteral, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table_as_with_row_type", "AS SELECT CAST(row(%s) AS row(value %s)) col".formatted(actualValue, actualTypeLiteral))) { assertThat(getColumnType(testTable.getName(), "col")).isEqualTo("row(value varchar)"); @@ -4519,8 +4474,7 @@ public void testTypeCoercionOnCreateTableAsWithArrayType() private void testTimestampCoercionOnCreateTableAsWithArrayType(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table_as_with_array_type", "AS SELECT array[%s] ts".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "ts")).isEqualTo("array(timestamp(6))"); @@ -4533,8 +4487,7 @@ private void testTimestampCoercionOnCreateTableAsWithArrayType(@Language("SQL") private void testCharCoercionOnCreateTableAsWithArrayType(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table_as_with_array_type", "AS SELECT array[%s] col".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "col")).isEqualTo("array(varchar)"); @@ -4583,8 +4536,7 @@ public void testTypeCoercionOnCreateTableAsWithMapType() private void testTimestampCoercionOnCreateTableAsWithMapType(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_timestamp_coercion_on_create_table_as_with_map_type", "AS SELECT map(array[%1$s], array[%1$s]) ts".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "ts")).isEqualTo("map(timestamp(6), timestamp(6))"); @@ -4597,8 +4549,7 @@ private void testTimestampCoercionOnCreateTableAsWithMapType(@Language("SQL") St private void testCharCoercionOnCreateTableAsWithMapType(@Language("SQL") String actualValue, @Language("SQL") String expectedValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_char_coercion_on_create_table_as_with_map_type", "AS SELECT map(array[%1$s], array[%1$s]) col".formatted(actualValue))) { assertThat(getColumnType(testTable.getName(), "col")).isEqualTo("map(varchar, varchar)"); @@ -4635,7 +4586,7 @@ public void testAddColumnWithTypeCoercion() private void testAddColumnWithTypeCoercion(String columnType, String expectedColumnType) { - try (TestTable testTable = new TestTable(getQueryRunner()::execute, "test_coercion_add_column", "(a varchar, b row(x integer))")) { + try (TestTable testTable = newTrinoTable("test_coercion_add_column", "(a varchar, b row(x integer))")) { // TODO: Update this test once the connector supports adding a new field to a row type assertQueryFails("ALTER TABLE " + testTable.getName() + " ADD COLUMN b.y " + columnType, "This connector does not support adding fields"); @@ -4654,8 +4605,7 @@ private void assertTimestampNtzFeature(String tableName) @Test public void testSelectTableUsingVersion() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_select_table_using_version", "(id INT, country VARCHAR)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1, 'India')", 1); @@ -4703,7 +4653,7 @@ public void testSelectTableUsingVersion() @Test public void testReadMultipleVersions() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_multiple_versions", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_read_multiple_versions", "AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); assertQuery( "SELECT * FROM " + table.getName() + " FOR VERSION AS OF 0 " + @@ -4716,7 +4666,7 @@ public void testReadMultipleVersions() @Test public void testReadVersionedTableWithOptimize() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_versioned_optimize", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_read_versioned_optimize", "AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); Set beforeActiveFiles = getActiveFiles(table.getName()); @@ -4743,7 +4693,7 @@ public void testReadVersionedTableWithVacuum() .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "vacuum_min_retention", "0s") .build(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_and_vacuum", "(x VARCHAR)")) { + try (TestTable table = newTrinoTable("test_add_column_and_vacuum", "(x VARCHAR)")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first'", 1); assertUpdate("INSERT INTO " + table.getName() + " SELECT 'second'", 1); @@ -4784,8 +4734,8 @@ public void testReadVersionedTableWithVacuum() @Test public void testInsertFromVersionedTable() { - try (TestTable targetTable = new TestTable(getQueryRunner()::execute, "test_read_versioned_insert", "(col int)"); - TestTable sourceTable = new TestTable(getQueryRunner()::execute, "test_read_versioned_insert", "AS SELECT 1 col")) { + try (TestTable targetTable = newTrinoTable("test_read_versioned_insert", "(col int)"); + TestTable sourceTable = newTrinoTable("test_read_versioned_insert", "AS SELECT 1 col")) { assertUpdate("INSERT INTO " + sourceTable.getName() + " VALUES 2", 1); assertUpdate("INSERT INTO " + sourceTable.getName() + " VALUES 3", 1); @@ -4800,7 +4750,7 @@ public void testInsertFromVersionedTable() @Test public void testInsertFromVersionedSameTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_versioned_insert", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_read_versioned_insert", "AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); assertUpdate("INSERT INTO " + table.getName() + " SELECT * FROM " + table.getName() + " FOR VERSION AS OF 0", 1); @@ -4824,7 +4774,7 @@ public void testInsertFromVersionedSameTable() @Test public void testInsertFromMultipleVersionedSameTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_versioned_insert", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_read_versioned_insert", "AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES 1, 2"); @@ -4841,7 +4791,7 @@ public void testInsertFromMultipleVersionedSameTable() @Test public void testReadVersionedTableWithChangeDataFeed() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_versioned_cdf", "WITH (change_data_feed_enabled=true) AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_read_versioned_cdf", "WITH (change_data_feed_enabled=true) AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); assertUpdate("UPDATE " + table.getName() + " SET id = -2 WHERE id = 2", 1); assertUpdate("DELETE FROM " + table.getName() + " WHERE id = 1", 1); @@ -4857,7 +4807,7 @@ public void testReadVersionedTableWithChangeDataFeed() public void testSelectTableUsingVersionSchemaEvolution() { // Delta Lake respects the old schema unlike Iceberg connector - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_select_table_using_version", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_select_table_using_version", "AS SELECT 1 id")) { assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN new_col VARCHAR"); assertQuery("SELECT * FROM " + table.getName() + " FOR VERSION AS OF 0", "VALUES 1"); assertQuery("SELECT * FROM " + table.getName() + " FOR VERSION AS OF 1", "VALUES (1, NULL)"); @@ -4897,7 +4847,7 @@ public void testSelectTableUsingVersionDeletedCheckpoints() public void testSelectAfterReadVersionedTable() { // Run normal SELECT after reading from versioned table - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_select_after_version", "AS SELECT 1 id")) { + try (TestTable table = newTrinoTable("test_select_after_version", "AS SELECT 1 id")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 2", 1); assertQuery("SELECT * FROM " + table.getName() + " FOR VERSION AS OF 0", "VALUES 1"); assertQuery("SELECT * FROM " + table.getName(), "VALUES 1, 2"); @@ -4965,7 +4915,7 @@ public void testDuplicatedFieldNames() assertQueryFails("CREATE TABLE " + tableName + "(col row(a row(x int, \"X\" int)))", "Field name 'x' specified more than once"); assertQueryFails("CREATE TABLE " + tableName + " AS SELECT cast(NULL AS row(a row(x int, \"X\" int))) col", "Field name 'x' specified more than once"); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_duplicated_field_names_", "(id int)")) { + try (TestTable table = newTrinoTable("test_duplicated_field_names_", "(id int)")) { assertQueryFails("ALTER TABLE " + table.getName() + " ADD COLUMN col row(x int, \"X\" int)", ".* Field name 'x' specified more than once"); assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN col row(\"X\" int)"); @@ -4992,7 +4942,7 @@ void testRegisterTableAccessControl() @Test public void testMetastoreAfterCreateTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int) COMMENT 'test comment'")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int) COMMENT 'test comment'")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( entry("comment", "test comment"), @@ -5004,7 +4954,7 @@ public void testMetastoreAfterCreateTable() @Test public void testMetastoreAfterCreateOrReplaceTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int) COMMENT 'test comment'")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int) COMMENT 'test comment'")) { assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + "(new_col varchar) COMMENT 'new comment'"); assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( @@ -5017,7 +4967,7 @@ public void testMetastoreAfterCreateOrReplaceTable() @Test public void testMetastoreAfterCreateTableAsSelect() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "COMMENT 'test comment' AS SELECT 1 col")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "COMMENT 'test comment' AS SELECT 1 col")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( entry("comment", "test comment"), @@ -5029,7 +4979,7 @@ public void testMetastoreAfterCreateTableAsSelect() @Test public void testMetastoreAfterCreateOrReplaceTableAsSelect() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "COMMENT 'test comment' AS SELECT 1 col")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "COMMENT 'test comment' AS SELECT 1 col")) { assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " COMMENT 'new comment' AS SELECT 'test' new_col", 1); assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( @@ -5042,7 +4992,7 @@ public void testMetastoreAfterCreateOrReplaceTableAsSelect() @Test public void testMetastoreAfterCommentTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .doesNotContainKey("comment") .contains( @@ -5061,7 +5011,7 @@ public void testMetastoreAfterCommentTable() @Test public void testMetastoreAfterCommentColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int COMMENT 'test comment')")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .doesNotContainKey("comment") .contains( @@ -5081,7 +5031,7 @@ public void testMetastoreAfterCommentColumn() public void testMetastoreAfterAlterColumn() { // Use 'name' column mapping mode to allow renaming columns - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int NOT NULL) WITH (column_mapping_mode = 'name')")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int NOT NULL) WITH (column_mapping_mode = 'name')")) { Map initialParameters = metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters(); assertThat(initialParameters) .doesNotContainKey("comment") @@ -5142,7 +5092,7 @@ public void testMetastoreAfterAlterColumn() @Test public void testMetastoreAfterSetTableProperties() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertUpdate("ALTER TABLE " + table.getName() + " SET PROPERTIES change_data_feed_enabled = true"); assertEventually(() -> assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( @@ -5154,7 +5104,7 @@ public void testMetastoreAfterSetTableProperties() @Test public void testMetastoreAfterOptimize() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertUpdate("ALTER TABLE " + table.getName() + " EXECUTE optimize"); assertEventually(() -> assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains( @@ -5166,7 +5116,7 @@ public void testMetastoreAfterOptimize() @Test public void testMetastoreAfterRegisterTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int) COMMENT 'test comment'")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int) COMMENT 'test comment'")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 1", 1); String tableLocation = metastore.getTable(SCHEMA, table.getName()).orElseThrow().getStorage().getLocation(); metastore.dropTable(SCHEMA, table.getName(), false); @@ -5183,7 +5133,7 @@ public void testMetastoreAfterRegisterTable() @Test public void testMetastoreAfterCreateTableRemotely() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int) COMMENT 'test comment'")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int) COMMENT 'test comment'")) { Table metastoreTable = metastore.getTable(SCHEMA, table.getName()).orElseThrow(); metastore.dropTable(SCHEMA, table.getName(), false); @@ -5211,7 +5161,7 @@ public void testMetastoreAfterDataManipulation() { String schemaString = "{\"type\":\"struct\",\"fields\":[{\"name\":\"col\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains(entry("trino_last_transaction_version", "0"), entry("trino_metadata_schema_string", schemaString)); @@ -5245,7 +5195,7 @@ public void testMetastoreAfterTruncateTable() { String schemaString = "{\"type\":\"struct\",\"fields\":[{\"name\":\"col\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "AS SELECT 1 col")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "AS SELECT 1 col")) { assertThat(metastore.getTable(SCHEMA, table.getName()).orElseThrow().getParameters()) .contains(entry("trino_last_transaction_version", "0"), entry("trino_metadata_schema_string", schemaString)); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index 9e0c17aa8692..e2f7348e2477 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -408,7 +408,7 @@ public void testStoreMetastoreCommentTable() private void testStoreMetastoreCommentTable(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertMetastoreInvocations(session, "COMMENT ON TABLE " + table.getName() + " IS 'test comment'", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); } } @@ -423,7 +423,7 @@ public void testStoreMetastoreCommentColumn() private void testStoreMetastoreCommentColumn(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int COMMENT 'test comment')")) { assertMetastoreInvocations(session, "COMMENT ON COLUMN " + table.getName() + ".col IS 'new test comment'", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); } } @@ -440,7 +440,7 @@ private void testStoreMetastoreAlterColumn(boolean storeTableMetadata) Session session = sessionWithStoreTableMetadata(storeTableMetadata); // Use 'name' column mapping mode to allow renaming columns - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int NOT NULL) WITH (column_mapping_mode = 'name')")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int NOT NULL) WITH (column_mapping_mode = 'name')")) { assertMetastoreInvocations(session, "ALTER TABLE " + table.getName() + " ALTER COLUMN col DROP NOT NULL", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); assertMetastoreInvocations(session, "ALTER TABLE " + table.getName() + " ADD COLUMN new_col int COMMENT 'test comment'", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); assertMetastoreInvocations(session, "ALTER TABLE " + table.getName() + " RENAME COLUMN new_col TO renamed_col", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); @@ -460,7 +460,7 @@ public void testStoreMetastoreSetTableProperties() private void testStoreMetastoreSetTableProperties(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertMetastoreInvocations(session, "ALTER TABLE " + table.getName() + " SET PROPERTIES change_data_feed_enabled = true", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); } } @@ -475,7 +475,7 @@ public void testStoreMetastoreOptimize() private void testStoreMetastoreOptimize(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertMetastoreInvocations(session, "ALTER TABLE " + table.getName() + " EXECUTE optimize", ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); } } @@ -494,7 +494,7 @@ private void testStoreMetastoreVacuum(boolean storeTableMetadata) .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "vacuum_min_retention", "0s") .build(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "AS SELECT 1 a")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "AS SELECT 1 a")) { assertUpdate("UPDATE " + table.getName() + " SET a = 2", 1); assertMetastoreInvocations( session, @@ -513,7 +513,7 @@ public void testStoreMetastoreRegisterTable() private void testStoreMetastoreRegisterTable(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int) COMMENT 'test comment'")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int) COMMENT 'test comment'")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES 1", 1); String tableLocation = metastore.getTable(TPCH_SCHEMA, table.getName()).orElseThrow().getStorage().getLocation(); metastore.dropTable(TPCH_SCHEMA, table.getName(), false); @@ -537,7 +537,7 @@ private void testStoreMetastoreDataManipulation(boolean storeTableMetadata) Session session = sessionWithStoreTableMetadata(storeTableMetadata); String schemaString = "{\"type\":\"struct\",\"fields\":[{\"name\":\"col\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "(col int)")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "(col int)")) { assertThat(metastore.getTable(TPCH_SCHEMA, table.getName()).orElseThrow().getParameters()) .contains(entry("trino_last_transaction_version", "0"), entry("trino_metadata_schema_string", schemaString)); @@ -563,7 +563,7 @@ public void testStoreMetastoreTruncateTable() private void testStoreMetastoreTruncateTable(boolean storeTableMetadata) { Session session = sessionWithStoreTableMetadata(storeTableMetadata); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_cache_metastore", "AS SELECT 1 col")) { + try (TestTable table = newTrinoTable("test_cache_metastore", "AS SELECT 1 col")) { assertMetastoreInvocations(session, "TRUNCATE TABLE " + table.getName(), ImmutableMultiset.of(GET_TABLE), asyncInvocations(storeTableMetadata)); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java index beeebb49ec41..a87d8e960d1e 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java @@ -70,7 +70,7 @@ public void testCreateView() { String tableName = "test_glue_table_" + randomNameSuffix(); String viewName = "test_glue_view_" + randomNameSuffix(); - try (TestTable table = new TestTable(getQueryRunner()::execute, tableName, "AS SELECT 'test' x"); + try (TestTable table = newTrinoTable(tableName, "AS SELECT 'test' x"); TestView view = new TestView(getQueryRunner()::execute, viewName, "SELECT * FROM " + table.getName())) { assertQuery(format("SELECT * FROM %s", view.getName()), "VALUES 'test'"); assertQuery(format("SELECT table_type FROM information_schema.tables WHERE table_name = '%s' AND table_schema='%s'", view.getName(), schema), "VALUES 'VIEW'"); diff --git a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java index f5b537bcb36b..bf48ea914bd1 100644 --- a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java +++ b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java @@ -52,7 +52,7 @@ void testShowTables() @Test void testColumnComment() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "comment", "(id INTEGER, name VARCHAR)")) { + try (TestTable table = newTrinoTable("comment", "(id INTEGER, name VARCHAR)")) { assertUpdate("COMMENT ON COLUMN %s.name IS 'comment text'".formatted(table.getName())); assertQuery("SHOW COLUMNS FROM " + table.getName(), "VALUES ('id', 'integer', '', ''), ('name', 'varchar', '', 'comment text')"); } @@ -61,7 +61,7 @@ void testColumnComment() @Test void testCannotCommentRowId() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "cannot_comment", "(id INTEGER, name VARCHAR)")) { + try (TestTable table = newTrinoTable("cannot_comment", "(id INTEGER, name VARCHAR)")) { assertThat(query("COMMENT ON COLUMN \"%s\".\"$row_id\" IS 'comment text'".formatted(table.getName()))) .failure() .hasErrorCode(INVALID_COLUMN_REFERENCE) @@ -124,7 +124,7 @@ void testSelectFromTable() @Test void testSelectLimit() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "single_column", "(rnd_bigint bigint NOT NULL)")) { + try (TestTable table = newTrinoTable("single_column", "(rnd_bigint bigint NOT NULL)")) { assertQuery("SELECT count(rnd_bigint) FROM (SELECT rnd_bigint FROM %s LIMIT 5) a".formatted(table.getName()), "VALUES (5)"); @@ -161,7 +161,7 @@ SELECT count(1) @Test void testSelectDefaultTableLimit() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "default_table_limit", "(rnd_bigint bigint NOT NULL) WITH (default_limit = 100)")) { + try (TestTable table = newTrinoTable("default_table_limit", "(rnd_bigint bigint NOT NULL) WITH (default_limit = 100)")) { assertQuery("SELECT count(distinct rnd_bigint) FROM " + table.getName(), "VALUES (100)"); } } @@ -169,7 +169,7 @@ void testSelectDefaultTableLimit() @Test public void selectOnlyNulls() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "only_nulls", "(rnd_bigint bigint) WITH (null_probability = 1.0)")) { + try (TestTable table = newTrinoTable("only_nulls", "(rnd_bigint bigint) WITH (null_probability = 1.0)")) { assertQuery("SELECT count(distinct rnd_bigint) FROM " + table.getName(), "VALUES (0)"); } } @@ -177,7 +177,7 @@ public void selectOnlyNulls() @Test void testSelectGenerator() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "generators", + try (TestTable table = newTrinoTable("generators", """ ( name VARCHAR NOT NULL WITH (generator = '#{Name.first_name} #{Name.last_name}'), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index f182f68d5a36..2f2e3b53a63b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -268,7 +268,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) @Override public void verifySupportsUpdateDeclaration() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = 100 WHERE regionkey = 2", MODIFYING_NON_TRANSACTIONAL_TABLE_MESSAGE); } } @@ -277,7 +277,7 @@ public void verifySupportsUpdateDeclaration() @Override public void verifySupportsRowLevelUpdateDeclaration() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_supports_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = nationkey * 100 WHERE regionkey = 2", MODIFYING_NON_TRANSACTIONAL_TABLE_MESSAGE); } } @@ -4403,8 +4403,7 @@ public void testShowCreateTable() @Test public void testShowCreateTableWithColumnProperties() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_show_create_table_with_column_properties", "(a INT, b INT WITH (partition_projection_type = 'INTEGER', partition_projection_range = ARRAY['0', '10'])) " + "WITH (" + @@ -8439,8 +8438,7 @@ public void testWriteInvalidPrecisionTimestamp() @Test public void testCoercingVarchar0ToVarchar1() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_create_table_varchar", "(var_column_0 varchar(0), var_column_1 varchar(1), var_column_10 varchar(10))")) { assertThat(getColumnType(testTable.getName(), "var_column_0")).isEqualTo("varchar(1)"); @@ -8452,8 +8450,7 @@ public void testCoercingVarchar0ToVarchar1() @Test public void testCoercingVarchar0ToVarchar1WithCTAS() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_ctas_varchar", "AS SELECT '' AS var_column")) { assertThat(getColumnType(testTable.getName(), "var_column")).isEqualTo("varchar(1)"); @@ -8463,8 +8460,7 @@ public void testCoercingVarchar0ToVarchar1WithCTAS() @Test public void testCoercingVarchar0ToVarchar1WithCTASNoData() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_ctas_nd_varchar", "AS SELECT '' AS var_column WITH NO DATA")) { assertThat(getColumnType(testTable.getName(), "var_column")).isEqualTo("varchar(1)"); @@ -8474,8 +8470,7 @@ public void testCoercingVarchar0ToVarchar1WithCTASNoData() @Test public void testCoercingVarchar0ToVarchar1WithAddColumn() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_add_column_varchar", "(col integer)")) { assertUpdate("ALTER TABLE " + testTable.getName() + " ADD COLUMN var_column varchar(0)"); @@ -8974,8 +8969,7 @@ public void testHiddenColumnNameConflict() private void testHiddenColumnNameConflict(String columnName) { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_hidden_column_name_conflict", format("(\"%s\" int, _bucket int, _partition int) WITH (partitioned_by = ARRAY['_partition'], bucketed_by = ARRAY['_bucket'], bucket_count = 10)", columnName))) { assertThat(query("SELECT * FROM " + table.getName())) @@ -9383,8 +9377,7 @@ public void testSelectWithShortZoneId() Resources.copy(resourceLocation, out); } - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_select_with_short_zone_id_", "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(tempDir))) { assertThat(query("SELECT * FROM %s".formatted(testTable.getName()))) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreSkipArchive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreSkipArchive.java index 19a0c276d336..338091df23ce 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreSkipArchive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreSkipArchive.java @@ -63,7 +63,7 @@ void cleanUpSchema() @Test void testSkipArchive() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_skip_archive", "(col int)")) { + try (TestTable table = newTrinoTable("test_skip_archive", "(col int)")) { List tableVersionsBeforeInsert = getTableVersions(testSchema, table.getName()); assertThat(tableVersionsBeforeInsert).hasSize(1); String versionIdBeforeInsert = getOnlyElement(tableVersionsBeforeInsert).versionId(); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java index 8ab4dc0c8ba5..b450e3950324 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java @@ -63,8 +63,7 @@ protected QueryRunner createQueryRunner() public void testSelectWithShortZoneId() { // When table is created using ORC file that contains short zone id in stripe footer - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_select_with_short_zone_id_", "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(dataFile.parentDirectory()))) { assertQuery("SELECT * FROM " + testTable.getName(), "VALUES (1, 'John', 'Doe')"); @@ -75,8 +74,7 @@ public void testSelectWithShortZoneId() public void testSelectWithoutShortZoneId() { // When table is created by trino - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_select_without_short_zone_id_", "(id INT, firstName VARCHAR, lastName VARCHAR)", ImmutableList.of("2, 'Alice', 'Doe'"))) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java index 91f31976d3ce..2d2c37bcdd30 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java @@ -124,7 +124,7 @@ public void testShowCreateTable() @Test public void testHiddenPathColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "hidden_file_path", "(a int, b VARCHAR)", ImmutableList.of("(1, 'a')"))) { + try (TestTable table = newTrinoTable("hidden_file_path", "(a int, b VARCHAR)", ImmutableList.of("(1, 'a')"))) { String filePath = (String) computeScalar(format("SELECT file_path FROM \"%s$files\"", table.getName())); assertQuery("SELECT DISTINCT \"$path\" FROM " + table.getName(), "VALUES " + "'" + filePath + "'"); @@ -149,8 +149,7 @@ public void testDeleteRowsConcurrently() String[] expectedErrors = new String[] {"Failed to commit the transaction during write:", "Failed to replace table due to concurrent updates:", "Failed to commit during write:"}; - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_delete", "(col0 INTEGER, col1 INTEGER, col2 INTEGER, col3 INTEGER)")) { String tableName = table.getName(); @@ -190,8 +189,7 @@ public void testDeleteRowsConcurrently() @Test public void testCreateOrReplaceTable() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_create_or_replace", " AS SELECT BIGINT '42' a, DOUBLE '-38.5' b")) { assertThat(query("SELECT a, b FROM " + table.getName())) @@ -535,8 +533,7 @@ public void testCreateTableWithNonExistingSchemaVerifyLocation() public void testSortedNationTable() { Session withSmallRowGroups = withSmallRowGroups(getSession()); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_sorted_nation_table", "WITH (sorted_by = ARRAY['comment'], format = '" + format.name() + "') AS SELECT * FROM nation WITH NO DATA")) { assertUpdate(withSmallRowGroups, "INSERT INTO " + table.getName() + " SELECT * FROM nation", 25); @@ -556,8 +553,7 @@ public void testFileSortingWithLargerTable() .setCatalogSessionProperty("iceberg", "parquet_writer_block_size", "20kB") .setCatalogSessionProperty("iceberg", "parquet_writer_batch_size", "200") .build(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_sorted_lineitem_table", "WITH (sorted_by = ARRAY['comment'], format = '" + format.name() + "') AS TABLE tpch.tiny.lineitem WITH NO DATA")) { assertUpdate( @@ -701,8 +697,7 @@ public void testDropTableWithNonExistentTableLocation() @Test public void testMetadataTables() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_metadata_tables", "(id int, part varchar) WITH (partitioning = ARRAY['part'])")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1, 'p1')", 1); @@ -752,8 +747,7 @@ public void testTableChangesFunction() { DateTimeFormatter instantMillisFormatter = DateTimeFormatter.ofPattern("uuuu-MM-dd'T'HH:mm:ss.SSSVV").withZone(UTC); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_table_changes_function_", "AS SELECT nationkey, name FROM tpch.tiny.nation WITH NO DATA")) { long initialSnapshot = getMostRecentSnapshotId(table.getName()); @@ -793,8 +787,7 @@ public void testTableChangesFunction() @Test public void testRowLevelDeletesWithTableChangesFunction() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_row_level_deletes_with_table_changes_function_", "AS SELECT nationkey, regionkey, name FROM tpch.tiny.nation WITH NO DATA")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT nationkey, regionkey, name FROM nation", 25); @@ -814,8 +807,7 @@ public void testCreateOrReplaceWithTableChangesFunction() { DateTimeFormatter instantMillisFormatter = DateTimeFormatter.ofPattern("uuuu-MM-dd'T'HH:mm:ss.SSSVV").withZone(UTC); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_table_changes_function_", "AS SELECT nationkey, name FROM tpch.tiny.nation WITH NO DATA")) { long initialSnapshot = getMostRecentSnapshotId(table.getName()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 2de9c8dfa04e..8f56f7fede55 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -215,7 +215,7 @@ public void initFileSystem() @BeforeAll public void initStorageTimePrecision() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "inspect_storage_precision", "(i int)")) { + try (TestTable table = newTrinoTable("inspect_storage_precision", "(i int)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1)", 1); assertUpdate("INSERT INTO " + table.getName() + " VALUES (2)", 1); assertUpdate("INSERT INTO " + table.getName() + " VALUES (3)", 1); @@ -243,7 +243,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) @Test public void testAddRowFieldCaseInsensitivity() { - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_row_field_case_insensitivity_", "AS SELECT CAST(row(row(2)) AS row(\"CHILD\" row(grandchild_1 integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(CHILD row(grandchild_1 integer))"); @@ -301,7 +301,7 @@ protected void verifyConcurrentAddColumnFailurePermissible(Exception e) @Test public void testDeleteOnV1Table() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_", "WITH (format_version = 1) AS SELECT * FROM orders")) { + try (TestTable table = newTrinoTable("test_delete_", "WITH (format_version = 1) AS SELECT * FROM orders")) { assertQueryFails("DELETE FROM " + table.getName() + " WHERE custkey <= 100", "Iceberg table updates require at least format version 2"); } } @@ -311,8 +311,7 @@ public void testDeleteOnV1Table() public void testCharVarcharComparison() { // with char->varchar coercion on table creation, this is essentially varchar/varchar comparison - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_char_varchar", "(k, v) AS VALUES" + " (-1, CAST(NULL AS CHAR(3))), " + @@ -1329,7 +1328,7 @@ private void testCreatePartitionedTableWithQuotedIdentifierCasing(String columnN @Test public void testPartitionColumnNameConflict() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_conflict_partition", "(ts timestamp, ts_day int) WITH (partitioning = ARRAY['day(ts)'])")) { + try (TestTable table = newTrinoTable("test_conflict_partition", "(ts timestamp, ts_day int) WITH (partitioning = ARRAY['day(ts)'])")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (TIMESTAMP '2021-07-24 03:43:57.987654', 1)", 1); assertThat(query("SELECT * FROM " + table.getName())) @@ -1338,7 +1337,7 @@ public void testPartitionColumnNameConflict() .matches("VALUES DATE '2021-07-24'"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_conflict_partition", "(ts timestamp, ts_day int)")) { + try (TestTable table = newTrinoTable("test_conflict_partition", "(ts timestamp, ts_day int)")) { assertUpdate("ALTER TABLE " + table.getName() + " SET PROPERTIES partitioning = ARRAY['day(ts)']"); assertUpdate("INSERT INTO " + table.getName() + " VALUES (TIMESTAMP '2021-07-24 03:43:57.987654', 1)", 1); @@ -1515,8 +1514,7 @@ private void testCreateSortedTableWithSortTransform(String columnName, String so public void testSortOrderChange() { Session withSmallRowGroups = withSmallRowGroups(getSession()); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_sort_order_change", "WITH (sorted_by = ARRAY['comment']) AS SELECT * FROM nation WITH NO DATA")) { assertUpdate(withSmallRowGroups, "INSERT INTO " + table.getName() + " SELECT * FROM nation", 25); @@ -1546,8 +1544,7 @@ public void testSortingDisabled() Session withSortingDisabled = Session.builder(withSmallRowGroups(getSession())) .setCatalogSessionProperty(ICEBERG_CATALOG, "sorted_writing_enabled", "false") .build(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_sorting_disabled", "WITH (sorted_by = ARRAY['comment']) AS SELECT * FROM nation WITH NO DATA")) { assertUpdate(withSortingDisabled, "INSERT INTO " + table.getName() + " SELECT * FROM nation", 25); @@ -1562,8 +1559,7 @@ public void testSortingDisabled() public void testOptimizeWithSortOrder() { Session withSmallRowGroups = withSmallRowGroups(getSession()); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_optimize_with_sort_order", "WITH (sorted_by = ARRAY['comment']) AS SELECT * FROM nation WITH NO DATA")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT * FROM nation WHERE nationkey < 10", 10); @@ -1584,8 +1580,7 @@ public void testOptimizeWithSortOrder() public void testUpdateWithSortOrder() { Session withSmallRowGroups = withSmallRowGroups(getSession()); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_sorted_update", "WITH (sorted_by = ARRAY['comment']) AS TABLE tpch.tiny.customer WITH NO DATA")) { assertUpdate( @@ -1623,8 +1618,7 @@ public void testSortingOnNestedField() public void testDroppingSortColumn() { Session withSmallRowGroups = withSmallRowGroups(getSession()); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_dropping_sort_column", "WITH (sorted_by = ARRAY['comment']) AS SELECT * FROM nation WITH NO DATA")) { assertUpdate(withSmallRowGroups, "INSERT INTO " + table.getName() + " SELECT * FROM nation", 25); @@ -1782,7 +1776,7 @@ public void testDuplicatedFieldNames() assertQueryFails("CREATE TABLE " + tableName + "(col row(a row(x int, \"X\" int)))", "Field name 'x' specified more than once"); assertQueryFails("CREATE TABLE " + tableName + " AS SELECT cast(NULL AS row(a row(x int, \"X\" int))) col", "Field name 'x' specified more than once"); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_duplicated_field_names_", "(id int)")) { + try (TestTable table = newTrinoTable("test_duplicated_field_names_", "(id int)")) { assertQueryFails("ALTER TABLE " + table.getName() + " ADD COLUMN col row(x int, \"X\" int)", ".* Field name 'x' specified more than once"); assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN col row(\"X\" int)"); @@ -4114,8 +4108,7 @@ public void testPredicatePushdown() @Test public void testPredicateOnDataColumnIsNotPushedDown() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_predicate_on_data_column_is_not_pushed_down", "(a integer)")) { assertThat(query("SELECT * FROM " + testTable.getName() + " WHERE a = 10")) @@ -5029,7 +5022,7 @@ public void testSplitPruningForFilterOnNonPartitionColumn() if (testSetup.isUnsupportedType()) { return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_split_pruning_non_partitioned", "(row_id int, col " + testSetup.getTrinoTypeName() + ")")) { + try (TestTable table = newTrinoTable("test_split_pruning_non_partitioned", "(row_id int, col " + testSetup.getTrinoTypeName() + ")")) { String tableName = table.getName(); String sampleValue = testSetup.getSampleValueLiteral(); String highValue = testSetup.getHighValueLiteral(); @@ -5108,8 +5101,7 @@ public void testSplitPruningFromDataFileStatistics() if (testSetup.isUnsupportedType()) { return; } - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_split_pruning_data_file_statistics", // Random double is needed to make sure rows are different. Otherwise compression may deduplicate rows, resulting in only one row group "(col " + testSetup.getTrinoTypeName() + ", r double)")) { @@ -6080,7 +6072,7 @@ public void testCollectingStatisticsWithFileModifiedTimeColumnPredicate() @Test public void testDeleteWithPathColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_with_path_", "(key int)")) { + try (TestTable table = newTrinoTable("test_delete_with_path_", "(key int)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1)", 1); sleepUninterruptibly(1, MILLISECONDS); assertUpdate("INSERT INTO " + table.getName() + " VALUES (2)", 1); @@ -6099,7 +6091,7 @@ public void testFileModifiedTimeHiddenColumn() if (storageTimePrecision.toMillis(1) > 1) { storageTimePrecision.sleep(1); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_file_modified_time_", "(col) AS VALUES (1)")) { + try (TestTable table = newTrinoTable("test_file_modified_time_", "(col) AS VALUES (1)")) { // Describe output should not have the $file_modified_time hidden column assertThat(query("DESCRIBE " + table.getName())) .skippingTypesCheck() @@ -6185,7 +6177,7 @@ public void testOptimizeWithFileModifiedTimeColumn() public void testDeleteWithFileModifiedTimeColumn() throws Exception { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_with_file_modified_time_", "(key int)")) { + try (TestTable table = newTrinoTable("test_delete_with_file_modified_time_", "(key int)")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (1)", 1); storageTimePrecision.sleep(1); assertUpdate("INSERT INTO " + table.getName() + " VALUES (2)", 1); @@ -6583,7 +6575,7 @@ public void testEmptyDelete() @Test public void testEmptyFilesTruncate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_empty_files_truncate_", "AS SELECT 1 AS id")) { + try (TestTable table = newTrinoTable("test_empty_files_truncate_", "AS SELECT 1 AS id")) { assertUpdate("TRUNCATE TABLE " + table.getName()); assertQueryReturnsEmptyResult("SELECT * FROM \"" + table.getName() + "$files\""); } @@ -6954,7 +6946,7 @@ public void testDeleteRetainsMetadataFile() @Test public void testCreateOrReplaceTableSnapshots() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " AS SELECT BIGINT '42' a, DOUBLE '-38.5' b")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " AS SELECT BIGINT '42' a, DOUBLE '-38.5' b")) { long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS SELECT BIGINT '-42' a, DOUBLE '38.5' b", 1); @@ -6969,7 +6961,7 @@ public void testCreateOrReplaceTableSnapshots() @Test public void testCreateOrReplaceTableChangeColumnNamesAndTypes() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " AS SELECT BIGINT '42' a, DOUBLE '-38.5' b")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " AS SELECT BIGINT '42' a, DOUBLE '-38.5' b")) { long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS SELECT CAST(ARRAY[ROW('test')] AS ARRAY(ROW(field VARCHAR))) a, VARCHAR 'test2' b", 1); @@ -6984,7 +6976,7 @@ public void testCreateOrReplaceTableChangeColumnNamesAndTypes() @Test public void testCreateOrReplaceTableChangePartitionedTableIntoUnpartitioned() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " WITH (partitioning=ARRAY['a']) AS SELECT BIGINT '42' a, 'some data' b UNION ALL SELECT BIGINT '43' a, 'another data' b")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " WITH (partitioning=ARRAY['a']) AS SELECT BIGINT '42' a, 'some data' b UNION ALL SELECT BIGINT '43' a, 'another data' b")) { long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " WITH (sorted_by=ARRAY['a']) AS SELECT BIGINT '22' a, 'new data' b", 1); @@ -7007,7 +6999,7 @@ public void testCreateOrReplaceTableChangePartitionedTableIntoUnpartitioned() @Test public void testCreateOrReplaceTableChangeUnpartitionedTableIntoPartitioned() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " WITH (sorted_by=ARRAY['a']) AS SELECT BIGINT '22' a, CAST('some data' AS VARCHAR) b")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " WITH (sorted_by=ARRAY['a']) AS SELECT BIGINT '22' a, CAST('some data' AS VARCHAR) b")) { long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " WITH (partitioning=ARRAY['a']) AS SELECT BIGINT '42' a, 'some data' b UNION ALL SELECT BIGINT '43' a, 'another data' b", 2); @@ -7030,7 +7022,7 @@ public void testCreateOrReplaceTableChangeUnpartitionedTableIntoPartitioned() @Test public void testCreateOrReplaceTableWithComments() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " (a BIGINT COMMENT 'This is a column') COMMENT 'This is a table'")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " (a BIGINT COMMENT 'This is a column') COMMENT 'This is a table'")) { long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS SELECT 1 a", 1); @@ -7057,8 +7049,7 @@ public void testCreateOrReplaceTableWithComments() @Test public void testCreateOrReplaceTableWithSameLocation() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_create_or_replace_with_same_location_", "(a integer)")) { String initialTableLocation = getTableLocation(table.getName()); @@ -7089,7 +7080,7 @@ public void testCreateOrReplaceTableWithSameLocation() @Test public void testCreateOrReplaceTableWithChangeInLocation() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_change_location_", "(a integer) ")) { + try (TestTable table = newTrinoTable("test_create_or_replace_change_location_", "(a integer) ")) { String initialTableLocation = getTableLocation(table.getName()) + randomNameSuffix(); long v1SnapshotId = getCurrentSnapshotId(table.getName()); assertQueryFails( @@ -7452,7 +7443,7 @@ protected OptionalInt maxTableRenameLength() @Test public void testSetPartitionedColumnType() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_partitioned_column_type_", "WITH (partitioning = ARRAY['part']) AS SELECT 1 AS id, CAST(123 AS integer) AS part")) { + try (TestTable table = newTrinoTable("test_set_partitioned_column_type_", "WITH (partitioning = ARRAY['part']) AS SELECT 1 AS id, CAST(123 AS integer) AS part")) { assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN part SET DATA TYPE bigint"); assertThat(query("SELECT part FROM " + table.getName())) @@ -7467,7 +7458,7 @@ public void testSetPartitionedColumnType() @Test public void testSetTransformPartitionedColumnType() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_partitioned_column_type_", "WITH (partitioning = ARRAY['bucket(part, 10)']) AS SELECT CAST(123 AS integer) AS part")) { + try (TestTable table = newTrinoTable("test_set_partitioned_column_type_", "WITH (partitioning = ARRAY['bucket(part, 10)']) AS SELECT CAST(123 AS integer) AS part")) { assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN part SET DATA TYPE bigint"); assertThat(query("SELECT * FROM " + table.getName())) @@ -7775,8 +7766,7 @@ public void testNoRetryWhenMetadataFileInvalid() @Test public void testTableChangesFunctionAfterSchemaChange() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_table_changes_function_", "AS SELECT nationkey, name FROM tpch.tiny.nation WITH NO DATA")) { long initialSnapshot = getCurrentSnapshotId(table.getName()); @@ -8152,8 +8142,8 @@ private static Session withPartitionFilterRequired(Session session) public void testUuidDynamicFilter() { String catalog = getSession().getCatalog().orElseThrow(); - try (TestTable dataTable = new TestTable(getQueryRunner()::execute, "data_table", "(value uuid)"); - TestTable filteringTable = new TestTable(getQueryRunner()::execute, "filtering_table", "(filtering_value uuid)")) { + try (TestTable dataTable = newTrinoTable("data_table", "(value uuid)"); + TestTable filteringTable = newTrinoTable("filtering_table", "(filtering_value uuid)")) { assertUpdate("INSERT INTO " + dataTable.getName() + " VALUES UUID 'f73894f0-5447-41c5-a727-436d04c7f8ab', UUID '4f676658-67c9-4e80-83be-ec75f0b9d0c9'", 2); assertUpdate("INSERT INTO " + filteringTable.getName() + " VALUES UUID 'f73894f0-5447-41c5-a727-436d04c7f8ab'", 1); @@ -8170,8 +8160,8 @@ public void testUuidDynamicFilter() public void testDynamicFilterWithExplicitPartitionFilter() { String catalog = getSession().getCatalog().orElseThrow(); - try (TestTable salesTable = new TestTable(getQueryRunner()::execute, "sales_table", "(date date, receipt_id varchar, amount decimal(10,2)) with (partitioning=array['date'])"); - TestTable dimensionTable = new TestTable(getQueryRunner()::execute, "dimension_table", "(date date, following_holiday boolean, year int)")) { + try (TestTable salesTable = newTrinoTable("sales_table", "(date date, receipt_id varchar, amount decimal(10,2)) with (partitioning=array['date'])"); + TestTable dimensionTable = newTrinoTable("dimension_table", "(date date, following_holiday boolean, year int)")) { assertUpdate( """ INSERT INTO %s @@ -8265,8 +8255,7 @@ private void testCreateTableWithCompressionCodec(HiveCompressionCodec compressio public void testTypeCoercionOnCreateTableAsSelect() { for (TypeCoercionTestSetup setup : typeCoercionOnCreateTableAsSelectProvider()) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_show_create_table", format("AS SELECT %s a", setup.sourceValueLiteral))) { assertThat(getColumnType(testTable.getName(), "a")).isEqualTo(setup.newColumnType); @@ -8282,8 +8271,7 @@ public void testTypeCoercionOnCreateTableAsSelect() public void testTypeCoercionOnCreateTableAsSelectWithNoData() { for (TypeCoercionTestSetup setup : typeCoercionOnCreateTableAsSelectProvider()) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_show_create_table", format("AS SELECT %s a WITH NO DATA", setup.sourceValueLiteral))) { assertThat(getColumnType(testTable.getName(), "a")).isEqualTo(setup.newColumnType); @@ -8470,7 +8458,7 @@ public void testAddColumnWithTypeCoercion() private void testAddColumnWithTypeCoercion(String columnType, String expectedColumnType) { - try (TestTable testTable = new TestTable(getQueryRunner()::execute, "test_coercion_add_column", "(a varchar, b row(x integer))")) { + try (TestTable testTable = newTrinoTable("test_coercion_add_column", "(a varchar, b row(x integer))")) { assertUpdate("ALTER TABLE " + testTable.getName() + " ADD COLUMN b.y " + columnType); assertThat(getColumnType(testTable.getName(), "b")).isEqualTo("row(x integer, y %s)".formatted(expectedColumnType)); @@ -8595,7 +8583,7 @@ public void testIllegalExtraPropertyKey() @Test public void testSetIllegalExtraPropertyKey() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_illegal_table_properties", "(x int)")) { + try (TestTable table = newTrinoTable("test_set_illegal_table_properties", "(x int)")) { assertQueryFails( "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id'])", "\\QIllegal keys in extra_properties: [sorted_by]"); @@ -8611,8 +8599,7 @@ public void testSetIllegalExtraPropertyKey() @Test // regression test for https://github.com/trinodb/trino/issues/22922 void testArrayElementChange() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_array_schema_change", "(col array(row(a varchar, b varchar)))", List.of("CAST(array[row('a', 'b')] AS array(row(a varchar, b varchar)))"))) { @@ -8631,8 +8618,7 @@ void testArrayElementChange() @Test void testRowFieldChange() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_row_schema_change", "(col row(a varchar, b varchar))")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT CAST(row('a', 'b') AS row(a varchar, b varchar))", 1); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergSystemTables.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergSystemTables.java index ff5b78fc48f7..be499a524122 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergSystemTables.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergSystemTables.java @@ -393,7 +393,7 @@ public void testManifestsTable() @Test public void testFilesTable() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_files_table", "AS SELECT 1 x")) { + try (TestTable table = newTrinoTable("test_files_table", "AS SELECT 1 x")) { MaterializedResult result = computeActual("DESCRIBE " + table.getName()); assertThat(result.getMaterializedRows().stream().map(row -> (String) row.getField(0))) .doesNotContain("partition"); @@ -509,7 +509,7 @@ void testFilesTableReadableMetrics() private void testFilesTableReadableMetrics(@Language("SQL") String type, @Language("SQL") String values, @Language("JSON") String... readableMetrics) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_files_table", "(x " + type + ")")) { + try (TestTable table = newTrinoTable("test_files_table", "(x " + type + ")")) { getQueryRunner().execute("INSERT INTO " + table.getName() + " " + values); assertThat(computeActual("SELECT readable_metrics FROM \"" + table.getName() + "$files\"").getOnlyColumnAsSet()) .containsExactlyInAnyOrder(readableMetrics); @@ -519,7 +519,7 @@ private void testFilesTableReadableMetrics(@Language("SQL") String type, @Langua @Test public void testFilesSchemaEvolution() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_files_table", "WITH (partitioning = ARRAY['part']) AS SELECT 1 x, 2 part")) { + try (TestTable table = newTrinoTable("test_files_table", "WITH (partitioning = ARRAY['part']) AS SELECT 1 x, 2 part")) { assertThat(query("SELECT partition FROM \"" + table.getName() + "$files\"")) .matches("SELECT CAST(ROW(2) AS ROW(part int))"); @@ -537,8 +537,7 @@ public void testFilesSchemaEvolution() @Test public void testFilesNestedPartition() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_files_table", "WITH (partitioning = ARRAY['\"part.nested\"']) AS SELECT 1 x, CAST(ROW(2) AS ROW(nested int)) part")) { assertThat(query("SELECT partition.\"part.nested\" FROM \"" + table.getName() + "$files\"")) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java index bcc365bff98b..c698ba1434a4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java @@ -138,9 +138,9 @@ public void testRowConstructorColumnLimitForMergeQuery() columns += "orderkey, custkey, orderstatus, totalprice, orderpriority) "; notMatchedClause += "s.orderkey, s.custkey, s.orderstatus, s.totalprice, s.orderpriority "; matchedClause += "orderkey = s.orderkey, custkey = s.custkey, orderstatus = s.orderstatus, totalprice = t.totalprice, orderpriority = s.orderpriority "; - TestTable table = new TestTable(getQueryRunner()::execute, "test_merge_", tableDefinition); + TestTable table = newTrinoTable("test_merge_", tableDefinition); assertUpdate("INSERT INTO " + table.getName() + " " + columns + " " + selectQuery, 1); - TestTable mergeTable = new TestTable(getQueryRunner()::execute, "test_table_", tableDefinition); + TestTable mergeTable = newTrinoTable("test_table_", tableDefinition); assertUpdate("INSERT INTO " + mergeTable.getName() + " " + columns + " " + selectQuery, 1); assertUpdate( """ diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergLocalConcurrentWrites.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergLocalConcurrentWrites.java index 951e8b7c1533..ed3cc5313aa7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergLocalConcurrentWrites.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergLocalConcurrentWrites.java @@ -806,8 +806,7 @@ void testConcurrentUpdateWithPartitionTransformation() List rows = ImmutableList.of("('A', DATE '2024-01-01')", "('B', DATE '2024-02-02')", "('C', DATE '2024-03-03')", "('D', DATE '2024-04-04')"); List partitions = ImmutableList.of("DATE '2024-01-01'", "DATE '2024-02-02'", "DATE '2024-03-03'", "DATE '2024-04-04'"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_update_partition_transform_table_", "(data varchar, part date) with (partitioning = array['month(part)'])")) { String tableName = table.getName(); @@ -850,8 +849,7 @@ void testConcurrentUpdateWithNestedPartitionTransformation() List rows = ImmutableList.of("('A', ROW(DATE '2024-01-01'))", "('B', ROW(DATE '2024-02-02'))", "('C', ROW(DATE '2024-03-03'))", "('D', ROW(DATE '2024-04-04'))"); List partitions = ImmutableList.of("DATE '2024-01-01'", "DATE '2024-02-02'", "DATE '2024-03-03'", "DATE '2024-04-04'"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_update_partition_transform_table_", "(data varchar, parent ROW (part date)) with (partitioning = array['month(\"parent.part\")'])")) { String tableName = table.getName(); @@ -900,8 +898,7 @@ void testConcurrentUpdateWithMultiplePartitionTransformation() List partitions2 = ImmutableList.of("1", "1", "1", "1"); List partitions3 = ImmutableList.of("'aaa'", "'aab'", "'aac'", "'aad'"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_update_multiple_partition_transform_table_", "(data varchar, part1 timestamp, part2 int, part3 varchar) with (partitioning = array['hour(part1)', 'bucket(part2, 10)', 'truncate(part3, 2)'])")) { String tableName = table.getName(); @@ -952,8 +949,7 @@ void testConcurrentUpdateWithOverlappingPartitionTransformation() List rows = ImmutableList.of("('A', DATE '2024-01-01')", "('B', DATE '2024-01-02')", "('C', DATE '2024-03-03')", "('D', DATE '2024-04-04')"); List partitions = ImmutableList.of("DATE '2024-01-01'", "DATE '2024-01-02'", "DATE '2024-03-03'", "DATE '2024-04-04'"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_update_overlapping_partition_transform_table_", "(data varchar, part date) with (partitioning = array['month(part)'])")) { String tableName = table.getName(); @@ -1015,8 +1011,7 @@ void testConcurrentUpdateWithEnforcedAndUnenforcedPartitions() List partitions1 = ImmutableList.of("'a'", "'b'", "'c'", "'d'"); List partitions2 = ImmutableList.of("DATE '2024-01-01'", "DATE '2024-02-02'", "DATE '2024-03-03'", "DATE '2024-04-04'"); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_concurrent_update_enforced_unenforced_partition_transform_table_", // part1 is enforced and part2 is unenforced as it has transformation "(data varchar, part1 varchar, part2 date) with (partitioning = array['part1', 'month(part2)'])")) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java index 114728084a47..23e3e3d68b3b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java @@ -131,7 +131,7 @@ private void testReadSingleIntegerColumnOrcFile(String orcFileResourceName, int throws Exception { checkArgument(expectedValue != 0); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_as_integer", "(\"_col0\") AS VALUES 0, NULL")) { + try (TestTable table = newTrinoTable("test_read_as_integer", "(\"_col0\") AS VALUES 0, NULL")) { String orcFilePath = (String) computeScalar(format("SELECT DISTINCT file_path FROM \"%s$files\"", table.getName())); byte[] orcFileData = Resources.toByteArray(getResource(orcFileResourceName)); fileSystem.newOutputFile(Location.of(orcFilePath)).createOrOverwrite(orcFileData); @@ -150,7 +150,7 @@ private void testReadSingleIntegerColumnOrcFile(String orcFileResourceName, int public void testTimeType() { // Regression test for https://github.com/trinodb/trino/issues/15603 - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_time", "(col time(6))")) { + try (TestTable table = newTrinoTable("test_time", "(col time(6))")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (TIME '13:30:00'), (TIME '14:30:00'), (NULL)", 3); assertQuery("SELECT * FROM " + table.getName(), "VALUES '13:30:00', '14:30:00', NULL"); assertQuery( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java index 99a2b2220dee..6c6a9023e59e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java @@ -60,8 +60,7 @@ protected boolean supportsRowGroupStatistics(String typeName) @Test public void testRowGroupResetDictionary() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_row_group_reset_dictionary", "(plain_col varchar, dict_col int)")) { String tableName = table.getName(); @@ -89,8 +88,7 @@ protected Optional filterSetColumnTypesDataProvider(SetColum @Test public void testIgnoreParquetStatistics() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_ignore_parquet_statistics", "WITH (sorted_by = ARRAY['custkey']) AS TABLE tpch.tiny.customer WITH NO DATA")) { assertUpdate( @@ -122,8 +120,7 @@ public void testIgnoreParquetStatistics() @Test public void testPushdownPredicateToParquetAfterColumnRename() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_pushdown_predicate_statistics", "WITH (sorted_by = ARRAY['custkey']) AS TABLE tpch.tiny.customer WITH NO DATA")) { assertUpdate( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index a5e59ec83610..f150da7b8394 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -188,7 +188,7 @@ public void testDefaultFormatVersion() @Test public void testSetPropertiesObjectStoreLayoutEnabled() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_object_store", "(x int) WITH (object_store_layout_enabled = false)")) { + try (TestTable table = newTrinoTable("test_object_store", "(x int) WITH (object_store_layout_enabled = false)")) { assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())) .doesNotContain("object_store_layout_enabled"); assertThat(loadTable(table.getName()).properties()) @@ -205,7 +205,7 @@ public void testSetPropertiesObjectStoreLayoutEnabled() @Test public void testSetPropertiesDataLocation() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_data_location", "(x int)")) { + try (TestTable table = newTrinoTable("test_data_location", "(x int)")) { assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())) .doesNotContain("data_location ="); assertThat(loadTable(table.getName()).properties()) @@ -415,7 +415,7 @@ public void testOptimizePopulateSplitOffsets() .setSystemProperty("task_min_writer_count", "1") .build(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_optimize_split_offsets", "AS SELECT * FROM tpch.tiny.nation")) { + try (TestTable table = newTrinoTable("test_optimize_split_offsets", "AS SELECT * FROM tpch.tiny.nation")) { assertUpdate(session, "ALTER TABLE " + table.getName() + " EXECUTE optimize"); assertThat(computeActual("SELECT split_offsets FROM \"" + table.getName() + "$files\"")) .isEqualTo(resultBuilder(getSession(), ImmutableList.of(new ArrayType(BIGINT))) @@ -689,8 +689,7 @@ private void runOptimizeDuringWriteOperations(boolean useSmallFiles) String blackholeTable = "blackhole_table_" + randomNameSuffix(); assertUpdate("CREATE TABLE blackhole.default.%s (a INT, b INT) WITH (split_count = 1, pages_per_split = 1, rows_per_page = 1, page_processing_delay = '3s')".formatted(blackholeTable)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_optimize_during_write_operations", "(int_col INT)")) { String tableName = table.getName(); @@ -1022,7 +1021,7 @@ public void testFilesTable() @Test public void testStatsFilePruning() { - try (TestTable testTable = new TestTable(getQueryRunner()::execute, "test_stats_file_pruning_", "(a INT, b INT) WITH (partitioning = ARRAY['b'])")) { + try (TestTable testTable = newTrinoTable("test_stats_file_pruning_", "(a INT, b INT) WITH (partitioning = ARRAY['b'])")) { assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (1, 10), (10, 10)", 2); assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (200, 10), (300, 20)", 2); @@ -1072,7 +1071,7 @@ public void testStatsFilePruning() @Test public void testColumnStatsPruning() { - try (TestTable testTable = new TestTable(getQueryRunner()::execute, "test_column_stats_pruning_", "(a INT, b INT) WITH (partitioning = ARRAY['b'])")) { + try (TestTable testTable = newTrinoTable("test_column_stats_pruning_", "(a INT, b INT) WITH (partitioning = ARRAY['b'])")) { assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (1, 10), (10, 10)", 2); assertUpdate("INSERT INTO " + testTable.getName() + " VALUES (200, 10), (300, 20)", 2); @@ -1479,8 +1478,7 @@ void testMapValueSchemaChange() private void testMapValueSchemaChange(String format, String expectedValue) { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_map_value_schema_change", "WITH (format = '" + format + "') AS SELECT CAST(map(array[1], array[row(2)]) AS map(integer, row(field integer))) col")) { Table icebergTable = loadTable(table.getName()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java index 8b0c017155c2..d88407b09309 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java @@ -154,7 +154,7 @@ public void testRenameSchema() @Test void testGlueTableLocation() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_table_location", "AS SELECT 1 x")) { + try (TestTable table = newTrinoTable("test_table_location", "AS SELECT 1 x")) { String initialLocation = getStorageDescriptor(getGlueTable(table.getName())).orElseThrow().getLocation(); assertThat(getStorageDescriptor(getGlueTable(table.getName())).orElseThrow().getLocation()) // Using startsWith because the location has UUID suffix diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java index 97529860099b..69be06111502 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java @@ -100,7 +100,7 @@ public void cleanup() @Test public void testSkipArchive() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_skip_archive", "(col int)")) { + try (TestTable table = newTrinoTable("test_skip_archive", "(col int)")) { List tableVersionsBeforeInsert = getTableVersions(schemaName, table.getName()); assertThat(tableVersionsBeforeInsert).hasSize(1); String versionIdBeforeInsert = getOnlyElement(tableVersionsBeforeInsert).getVersionId(); @@ -118,7 +118,7 @@ public void testSkipArchive() @Test public void testNotRemoveExistingArchive() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_remove_archive", "(col int)")) { + try (TestTable table = newTrinoTable("test_remove_archive", "(col int)")) { List tableVersionsBeforeInsert = getTableVersions(schemaName, table.getName()); assertThat(tableVersionsBeforeInsert).hasSize(1); TableVersion initialVersion = getOnlyElement(tableVersionsBeforeInsert); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestIcebergNessieCatalogWithBearerAuth.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestIcebergNessieCatalogWithBearerAuth.java index 85ddbcbaa524..d47384cc0595 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestIcebergNessieCatalogWithBearerAuth.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestIcebergNessieCatalogWithBearerAuth.java @@ -74,7 +74,7 @@ protected QueryRunner createQueryRunner() @Test public void testWithValidAccessToken() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_valid_access_token", "(a INT, b VARCHAR)", ImmutableList.of("(1, 'a')"))) { + try (TestTable table = newTrinoTable("test_valid_access_token", "(a INT, b VARCHAR)", ImmutableList.of("(1, 'a')"))) { assertQuery("SELECT * FROM " + table.getName(), "VALUES(1, 'a')"); } } diff --git a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteConnectorTest.java b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteConnectorTest.java index c4ee09cdd966..142eaef4a076 100644 --- a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteConnectorTest.java +++ b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteConnectorTest.java @@ -109,8 +109,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) @Test public void testLikeWithEscape() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_like_with_escape", "(id int, a varchar(4))", List.of( @@ -140,8 +139,7 @@ public void testIsNullPredicatePushdown() assertThat(query("SELECT nationkey FROM nation WHERE name IS NULL")).isFullyPushedDown(); assertThat(query("SELECT nationkey FROM nation WHERE name IS NULL OR name = 'a' OR regionkey = 4")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_null_predicate_pushdown", "(a_int integer, a_varchar varchar(1))", List.of( @@ -158,8 +156,7 @@ public void testIsNotNullPredicatePushdown() { assertThat(query("SELECT nationkey FROM nation WHERE name IS NOT NULL OR regionkey = 4")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_not_null_predicate_pushdown", "(a_int integer, a_varchar varchar(1))", List.of( @@ -176,8 +173,7 @@ public void testNotExpressionPushdown() { assertThat(query("SELECT nationkey FROM nation WHERE NOT(name LIKE '%A%')")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_not_predicate_pushdown", "(a_int integer, a_varchar varchar(2))", List.of( @@ -332,8 +328,7 @@ public void testShowCreateTable() @Test public void testAvgDecimalExceedingSupportedPrecision() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_avg_decimal_exceeding_supported_precision", "(a decimal(38, 38), b bigint)", List.of( @@ -405,7 +400,7 @@ public void testDropAndAddColumnWithSameName() { // Override because Ignite can access old data after dropping and adding a column with same name executeExclusively(() -> { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_add_column", "AS SELECT 1 x, 2 y, 3 z")) { + try (TestTable table = newTrinoTable("test_drop_add_column", "AS SELECT 1 x, 2 y, 3 z")) { assertUpdate("ALTER TABLE " + table.getName() + " DROP COLUMN y"); assertQuery("SELECT * FROM " + table.getName(), "VALUES (1, 3)"); diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduConnectorTest.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduConnectorTest.java index 6150f160a19d..ef506197672b 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduConnectorTest.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduConnectorTest.java @@ -523,8 +523,7 @@ public void testAddColumnWithComment() public void testAddColumnWithCommentSpecialCharacter(String comment) { // Override because Kudu connector doesn't support creating a new table without partition columns - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_col_", "(id INT WITH (primary_key=true), a_varchar varchar) WITH (partition_by_hash_columns = ARRAY['id'], partition_by_hash_buckets = 2)")) { assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN b_varchar varchar COMMENT " + varcharLiteral(comment)); @@ -557,7 +556,7 @@ public void testAddColumnWithDecimal() @Test public void testInsertIntoTableHavingRowUuid() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", " AS SELECT * FROM region WITH NO DATA")) { + try (TestTable table = newTrinoTable("test_insert_", " AS SELECT * FROM region WITH NO DATA")) { assertUpdate("INSERT INTO " + table.getName() + " SELECT * FROM region", 5); assertThat(query("SELECT * FROM " + table.getName())) @@ -570,7 +569,7 @@ public void testInsertIntoTableHavingRowUuid() public void testInsertUnicode() { // TODO Remove this overriding test once kudu connector can create tables with default partitions - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50) WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['test'], partition_by_hash_buckets = 2)")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'Hello', U&'hello\\6d4B\\8Bd5world\\7F16\\7801' ", 2); @@ -578,7 +577,7 @@ public void testInsertUnicode() .containsExactlyInAnyOrder("Hello", "hello测试world编码"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50) WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['test'], partition_by_hash_buckets = 2)")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'aa', 'bé'", 2); @@ -589,7 +588,7 @@ public void testInsertUnicode() assertQueryReturnsEmptyResult("SELECT test FROM " + table.getName() + " WHERE test = 'ba'"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50) WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['test'], partition_by_hash_buckets = 2)")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'a', 'é'", 2); @@ -606,7 +605,7 @@ public void testInsertUnicode() public void testInsertHighestUnicodeCharacter() { // TODO Remove this overriding test once kudu connector can create tables with default partitions - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50) WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['test'], partition_by_hash_buckets = 2)")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'Hello', U&'hello\\6d4B\\8Bd5\\+10FFFFworld\\7F16\\7801' ", 2); @@ -620,7 +619,7 @@ public void testInsertHighestUnicodeCharacter() public void testInsertNegativeDate() { // TODO Remove this overriding test once kudu connector can create tables with default partitions - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_date", + try (TestTable table = newTrinoTable("insert_date", "(dt DATE WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['dt'], partition_by_hash_buckets = 2)")) { assertQueryFails(format("INSERT INTO %s VALUES (DATE '-0001-01-01')", table.getName()), errorMessageForInsertNegativeDate("-0001-01-01")); @@ -695,7 +694,7 @@ public void testDeleteWithLike() protected TestTable createTableWithOneIntegerColumn(String namePrefix) { // TODO Remove this overriding method once kudu connector can create tables with default partitions - return new TestTable(getQueryRunner()::execute, namePrefix, + return newTrinoTable(namePrefix, "(col integer WITH (primary_key=true)) " + "WITH (partition_by_hash_columns = ARRAY['col'], partition_by_hash_buckets = 2)"); } @@ -1027,7 +1026,7 @@ public void testCreateTableWithTableComment() protected void testCreateTableWithTableCommentSpecialCharacter(String comment) { // TODO Remove this overriding test once kudu connector can create tables with default partitions - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_create_", "(a bigint WITH (primary_key=true)) COMMENT " + varcharLiteral(comment) + "WITH (partition_by_hash_columns = ARRAY['a'], partition_by_hash_buckets = 2)")) { diff --git a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/BaseMariaDbConnectorTest.java b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/BaseMariaDbConnectorTest.java index 8c1bac7c0115..c299e9c9a510 100644 --- a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/BaseMariaDbConnectorTest.java +++ b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/BaseMariaDbConnectorTest.java @@ -190,7 +190,7 @@ public void testAddNotNullColumn() .isInstanceOf(AssertionError.class) .hasMessage("Should fail to add not null column without a default value to a non-empty table"); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_col", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_nn_col", "(a_varchar varchar)")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES ('a')", 1); @@ -269,7 +269,7 @@ public void testDeleteWithLike() @Override public void testInsertIntoNotNullColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { + try (TestTable table = newTrinoTable("insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); assertQueryFails(format("INSERT INTO %s (nullable_col) VALUES (1)", table.getName()), errorMessageForInsertIntoNotNullColumn("not_null_col")); @@ -282,7 +282,7 @@ public void testInsertIntoNotNullColumn() assertQueryFails(format("INSERT INTO %s (nullable_col) SELECT nationkey FROM nation WHERE regionkey < 0", table.getName()), ".*Field 'not_null_col' doesn't have a default value.*"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { + try (TestTable table = newTrinoTable("commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); // This is enforced by the engine and not the connector diff --git a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java index 8ec8b95227c7..bdb1dd2e737a 100644 --- a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java +++ b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java @@ -627,7 +627,7 @@ private void testDate(ZoneId sessionZone) @Test public void testUnsupportedDate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_negative_date", "(dt DATE)")) { + try (TestTable table = newTrinoTable("test_negative_date", "(dt DATE)")) { assertQueryFails(format("INSERT INTO %s VALUES (DATE '-0001-01-01')", table.getName()), ".*Failed to insert data.*"); assertQueryFails(format("INSERT INTO %s VALUES (DATE '10000-01-01')", table.getName()), ".*Failed to insert data.*"); } @@ -777,7 +777,7 @@ private void testTimestamp(ZoneId sessionZone) @Test public void testIncorrectTimestamp() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_incorrect_timestamp", "(dt TIMESTAMP)")) { + try (TestTable table = newTrinoTable("test_incorrect_timestamp", "(dt TIMESTAMP)")) { assertQueryFails(format("INSERT INTO %s VALUES (TIMESTAMP '1970-01-01 00:00:00.000')", table.getName()), ".*Failed to insert data.*"); assertQueryFails(format("INSERT INTO %s VALUES (TIMESTAMP '2038-01-19 03:14:08.000')", table.getName()), ".*Failed to insert data.*"); } diff --git a/plugin/trino-memory/src/test/java/io/trino/plugin/memory/TestMemoryConnectorTest.java b/plugin/trino-memory/src/test/java/io/trino/plugin/memory/TestMemoryConnectorTest.java index abc54b106411..f1d71b812e1e 100644 --- a/plugin/trino-memory/src/test/java/io/trino/plugin/memory/TestMemoryConnectorTest.java +++ b/plugin/trino-memory/src/test/java/io/trino/plugin/memory/TestMemoryConnectorTest.java @@ -607,7 +607,7 @@ public void testRenameView() @Test void testInsertAfterTruncate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_truncate", "AS SELECT 1 x")) { + try (TestTable table = newTrinoTable("test_truncate", "AS SELECT 1 x")) { assertUpdate("TRUNCATE TABLE " + table.getName()); assertQueryReturnsEmptyResult("SELECT * FROM " + table.getName()); diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java index f39404a42ea0..b5de69af1d6d 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/BaseMongoConnectorSmokeTest.java @@ -42,8 +42,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) @Test public void testProjectionPushdown() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_multiple_rows_", "(id INT, nested1 ROW(child1 INT, child2 VARCHAR))", ImmutableList.of( @@ -60,8 +59,7 @@ public void testProjectionPushdown() @Test public void testReadDottedField() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_read_dotted_field_", "(root ROW(\"dotted.field\" VARCHAR, field VARCHAR))", ImmutableList.of("ROW(ROW('foo', 'bar'))"))) { @@ -76,8 +74,7 @@ public void testReadDottedField() @Test public void testReadDollarPrefixedField() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_read_dotted_field_", "(root ROW(\"$field1\" VARCHAR, field2 VARCHAR))", ImmutableList.of("ROW(ROW('foo', 'bar'))"))) { @@ -92,8 +89,7 @@ public void testReadDollarPrefixedField() @Test public void testProjectionPushdownWithHighlyNestedData() { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_highly_nested_data_", "(id INT, row1_t ROW(f1 INT, f2 INT, row2_t ROW (f1 INT, f2 INT, row3_t ROW(f1 INT, f2 INT))))", ImmutableList.of("(1, ROW(2, 3, ROW(4, 5, ROW(6, 7))))", diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoConnectorTest.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoConnectorTest.java index fd85c4abb8a6..512da391977c 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoConnectorTest.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoConnectorTest.java @@ -356,7 +356,7 @@ public void testPredicatePushdown() private void testPredicatePushdown(String value) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_predicate_pushdown", "AS SELECT %s col".formatted(value))) { + try (TestTable table = newTrinoTable("test_predicate_pushdown", "AS SELECT %s col".formatted(value))) { testPredicatePushdown(table.getName(), "col = " + value); testPredicatePushdown(table.getName(), "col != " + value); testPredicatePushdown(table.getName(), "col < " + value); @@ -380,7 +380,7 @@ public void testPredicatePushdownDoubleType() private void testPredicatePushdownFloatingPoint(String value) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_floating_point_pushdown", "AS SELECT %s col".formatted(value))) { + try (TestTable table = newTrinoTable("test_floating_point_pushdown", "AS SELECT %s col".formatted(value))) { assertThat(query("SELECT * FROM " + table.getName() + " WHERE col = " + value)) .isFullyPushedDown(); assertThat(query("SELECT * FROM " + table.getName() + " WHERE col <= " + value)) @@ -402,8 +402,7 @@ private void testPredicatePushdownFloatingPoint(String value) @Test public void testPredicatePushdownCharWithPaddedSpace() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_predicate_pushdown_char_with_padded_space", "(k, v) AS VALUES" + " (-1, CAST(NULL AS char(3))), " + @@ -436,8 +435,7 @@ public void testPredicatePushdownCharWithPaddedSpace() public void testPredicatePushdownMultipleNotEquals() { // Regression test for https://github.com/trinodb/trino/issues/19404 - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_predicate_pushdown_with_multiple_not_equals", "(id, value) AS VALUES (1, 10), (2, 20), (3, 30)")) { assertThat(query("SELECT * FROM " + table.getName() + " WHERE id != 1 AND value != 20")) @@ -449,8 +447,7 @@ public void testPredicatePushdownMultipleNotEquals() @Test public void testHighPrecisionDecimalPredicate() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_high_precision_decimal_predicate", "(col DECIMAL(34, 0))", Arrays.asList("decimal '3141592653589793238462643383279502'", null))) { @@ -1354,8 +1351,7 @@ private void testFiltersOnDereferenceColumnReadsLessData(String expectedValue, S .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "projection_pushdown_enabled", "false") .build(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "filter_on_projection_columns", format("(col_0 ROW(col_1 %1$s, col_2 ROW(col_3 %1$s, col_4 ROW(col_5 %1$s))))", expectedType))) { assertUpdate(format("INSERT INTO %s VALUES NULL", table.getName()), 1); diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoTypeMapping.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoTypeMapping.java index 5d98edd7db21..b3346797f3fb 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoTypeMapping.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoTypeMapping.java @@ -398,7 +398,7 @@ public void testArray() public void testArrayNulls() { // Verify only SELECT instead of using SqlDataTypeTest because array comparison not supported for arrays with null elements - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_array_nulls", "(c1 ARRAY(boolean), c2 ARRAY(varchar), c3 ARRAY(varchar))", ImmutableList.of("(NULL, ARRAY[NULL], ARRAY['foo', NULL, 'bar', NULL])"))) { + try (TestTable table = newTrinoTable("test_array_nulls", "(c1 ARRAY(boolean), c2 ARRAY(varchar), c3 ARRAY(varchar))", ImmutableList.of("(NULL, ARRAY[NULL], ARRAY['foo', NULL, 'bar', NULL])"))) { assertThat(query("SELECT c1 FROM " + table.getName())).matches("VALUES CAST(NULL AS ARRAY(boolean))"); assertThat(query("SELECT c2 FROM " + table.getName())).matches("VALUES CAST(ARRAY[NULL] AS ARRAY(varchar))"); assertThat(query("SELECT c3 FROM " + table.getName())).matches("VALUES CAST(ARRAY['foo', NULL, 'bar', NULL] AS ARRAY(varchar))"); diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlConnectorTest.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlConnectorTest.java index 31e3801153fb..d030e98a11fa 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlConnectorTest.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/BaseMySqlConnectorTest.java @@ -271,7 +271,7 @@ public void testAddNotNullColumn() .isInstanceOf(AssertionError.class) .hasMessage("Should fail to add not null column without a default value to a non-empty table"); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_col", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_nn_col", "(a_varchar varchar)")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES ('a')", 1); diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java index ba076101d7b4..c70e80df5d53 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java @@ -692,7 +692,7 @@ private void testDate(ZoneId sessionZone) public void testJulianGregorianDate() { // Oracle TO_DATE function returns +10 days during julian and gregorian calendar switch - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_julian_dt", "(ts date)")) { + try (TestTable table = newTrinoTable("test_julian_dt", "(ts date)")) { assertUpdate(format("INSERT INTO %s VALUES (DATE '1582-10-05')", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES TIMESTAMP '1582-10-15 00:00:00'"); } @@ -701,7 +701,7 @@ public void testJulianGregorianDate() @Test public void testUnsupportedDate() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_unsupported_dt", "(ts date)")) { + try (TestTable table = newTrinoTable("test_unsupported_dt", "(ts date)")) { assertQueryFails( format("INSERT INTO %s VALUES (DATE '-4713-12-31')", table.getName()), """ @@ -960,7 +960,7 @@ public void testTimestampAllPrecisionsOnOracle() public void testJulianGregorianTimestamp() { // Oracle TO_DATE function returns +10 days during julian and gregorian calendar switch - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_julian_ts", "(ts date)")) { + try (TestTable table = newTrinoTable("test_julian_ts", "(ts date)")) { assertUpdate(format("INSERT INTO %s VALUES (timestamp '1582-10-05')", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES TIMESTAMP '1582-10-15 00:00:00'"); } @@ -969,7 +969,7 @@ public void testJulianGregorianTimestamp() @Test public void testUnsupportedTimestamp() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_unsupported_ts", "(ts timestamp)")) { + try (TestTable table = newTrinoTable("test_unsupported_ts", "(ts timestamp)")) { assertQueryFails( format("INSERT INTO %s VALUES (TIMESTAMP '-4713-12-31 00:00:00.000')", table.getName()), """ diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java index b85473807bf6..435a0fc2f07f 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java @@ -198,8 +198,7 @@ public void testCharVarcharComparison() { // test overridden because super uses all-space char values (' ') that are null-out by Oracle - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_char_varchar", "(k, v) AS VALUES" + " (-1, CAST(NULL AS char(3))), " + @@ -222,8 +221,7 @@ public void testVarcharCharComparison() { // test overridden because Oracle nulls-out '' varchar value, impacting results - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_varchar_char", "(k, v) AS VALUES" + " (-1, CAST(NULL AS varchar(3))), " + @@ -500,8 +498,8 @@ private void predicatePushdownTest(String oracleType, String oracleLiteral, Stri @Test public void testJoinPushdownWithImplicitCast() { - try (TestTable leftTable = new TestTable(getQueryRunner()::execute, "left_table", "(id int, varchar_50 varchar(50))", ImmutableList.of("(1, 'India')", "(2, 'Poland')")); - TestTable rightTable = new TestTable(getQueryRunner()::execute, "right_table_", "(varchar_100 varchar(100), varchar_unbounded varchar)", ImmutableList.of("('India', 'Japan')", "('France', 'Poland')"))) { + try (TestTable leftTable = newTrinoTable("left_table", "(id int, varchar_50 varchar(50))", ImmutableList.of("(1, 'India')", "(2, 'Poland')")); + TestTable rightTable = newTrinoTable("right_table_", "(varchar_100 varchar(100), varchar_unbounded varchar)", ImmutableList.of("('India', 'Japan')", "('France', 'Poland')"))) { String leftTableName = leftTable.getName(); String rightTableName = rightTable.getName(); Session session = joinPushdownEnabled(getSession()); diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConnectorTest.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConnectorTest.java index 4797960ab08d..b34a36913e9f 100644 --- a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConnectorTest.java +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixConnectorTest.java @@ -335,8 +335,7 @@ public void testCharVarcharComparison() { // test overridden because super uses all-space char values (' ') that are null-out by Phoenix - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_char_varchar", "(k, v) AS VALUES" + " (-1, CAST(NULL AS char(3))), " + @@ -359,8 +358,7 @@ public void testVarcharCharComparison() { // test overridden because Phoenix nulls-out '' varchar value, impacting results - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_varchar_char", "(k, v) AS VALUES" + " (-1, CAST(NULL AS varchar(3))), " + @@ -409,7 +407,7 @@ public void testCountDistinctWithStringTypes() .collect(toImmutableList()); String tableName = "count_distinct_strings" + randomNameSuffix(); - try (TestTable testTable = new TestTable(getQueryRunner()::execute, tableName, "(id int, t_char CHAR(5), t_varchar VARCHAR(5)) WITH (ROWKEYS='id')", rows)) { + try (TestTable testTable = newTrinoTable(tableName, "(id int, t_char CHAR(5), t_varchar VARCHAR(5)) WITH (ROWKEYS='id')", rows)) { assertQuery("SELECT count(DISTINCT t_varchar) FROM " + testTable.getName(), "VALUES 6"); assertQuery("SELECT count(DISTINCT t_char) FROM " + testTable.getName(), "VALUES 6"); assertQuery("SELECT count(DISTINCT t_char), count(DISTINCT t_varchar) FROM " + testTable.getName(), "VALUES (6, 6)"); diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java index 19b9a0879327..fc59390ded20 100644 --- a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java @@ -715,7 +715,7 @@ public void testArray() public void testArrayNulls() { // Verify only SELECT instead of using SqlDataTypeTest because array comparison not supported for arrays with null elements - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_array_nulls", "(c1 ARRAY(boolean), c2 ARRAY(varchar), c3 ARRAY(varchar))", ImmutableList.of("(NULL, ARRAY[NULL], ARRAY['foo', NULL, 'bar', NULL])"))) { + try (TestTable table = newTrinoTable("test_array_nulls", "(c1 ARRAY(boolean), c2 ARRAY(varchar), c3 ARRAY(varchar))", ImmutableList.of("(NULL, ARRAY[NULL], ARRAY['foo', NULL, 'bar', NULL])"))) { assertThat(query("SELECT c1 FROM " + table.getName())).matches("VALUES CAST(NULL AS ARRAY(boolean))"); assertThat(query("SELECT c2 FROM " + table.getName())).matches("VALUES CAST(ARRAY[NULL] AS ARRAY(varchar))"); assertThat(query("SELECT c3 FROM " + table.getName())).matches("VALUES CAST(ARRAY['foo', NULL, 'bar', NULL] AS ARRAY(varchar))"); diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java index 4e39f890ab86..03e3389798eb 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java @@ -194,8 +194,7 @@ public void testTimestampPrecisionOnCreateTable() private void testTimestampPrecisionOnCreateTable(String inputType, String expectedType) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_show_create_table", format("(a %s)", inputType))) { assertThat(getColumnType(testTable.getName(), "a")).isEqualTo(expectedType); @@ -234,8 +233,7 @@ public void testTimestampPrecisionOnCreateTableAsSelect() private void testTimestampPrecisionOnCreateTableAsSelect(String inputType, String tableType, String tableValue) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_show_create_table", format("AS SELECT %s a", inputType))) { assertThat(getColumnType(testTable.getName(), "a")).isEqualTo(tableType); @@ -277,8 +275,7 @@ public void testTimestampPrecisionOnCreateTableAsSelectWithNoData() private void testTimestampPrecisionOnCreateTableAsSelectWithNoData(String inputType, String tableType) { - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_coercion_show_create_table", format("AS SELECT %s a WITH NO DATA", inputType))) { assertThat(getColumnType(testTable.getName(), "a")).isEqualTo(tableType); @@ -833,8 +830,7 @@ public void testLikePredicatePushdown() assertThat(query("SELECT nationkey FROM nation WHERE name LIKE '%A%'")) .isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_like_predicate_pushdown", "(id integer, a_varchar varchar(1))", List.of( @@ -856,8 +852,7 @@ public void testLikeWithEscapePredicatePushdown() assertThat(query("SELECT nationkey FROM nation WHERE name LIKE '%A%' ESCAPE '\\'")) .isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_like_with_escape_predicate_pushdown", "(id integer, a_varchar varchar(4))", List.of( @@ -878,8 +873,7 @@ public void testIsNullPredicatePushdown() assertThat(query("SELECT nationkey FROM nation WHERE name IS NULL")).isFullyPushedDown(); assertThat(query("SELECT nationkey FROM nation WHERE name IS NULL OR regionkey = 4")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_null_predicate_pushdown", "(a_int integer, a_varchar varchar(1))", List.of( @@ -896,8 +890,7 @@ public void testIsNotNullPredicatePushdown() { assertThat(query("SELECT nationkey FROM nation WHERE name IS NOT NULL OR regionkey = 4")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_not_null_predicate_pushdown", "(a_int integer, a_varchar varchar(1))", List.of( @@ -935,8 +928,7 @@ public void testNotExpressionPushdown() { assertThat(query("SELECT nationkey FROM nation WHERE NOT(name LIKE '%A%' ESCAPE '\\')")).isFullyPushedDown(); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_is_not_predicate_pushdown", "(a_int integer, a_varchar varchar(2))", List.of( @@ -952,8 +944,7 @@ public void testNotExpressionPushdown() @Test public void testInPredicatePushdown() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_in_predicate_pushdown", "(id varchar(1), id2 varchar(1))", List.of( @@ -1095,8 +1086,7 @@ public void testTimestampColumnAndTimestampWithTimeZoneConstant() @Test public void testReverseFunctionProjectionPushDown() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_reverse_pushdown_for_project", "(id BIGINT, varchar_col VARCHAR)", ImmutableList.of("1, 'abc'", "2, null"))) { @@ -1151,8 +1141,7 @@ public void testReverseFunctionProjectionPushDown() @Test public void testPartialProjectionPushDown() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_partial_projection_pushdown", "(id BIGINT, cola VARCHAR, colb VARCHAR)", ImmutableList.of("1, 'abc', 'def'"))) { @@ -1202,8 +1191,7 @@ public void testPartialProjectionPushDown() @Test public void testProjectionsNotPushDownWhenFilterAppliedOnProjectedColumn() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_projection_push_down_with_filter", "(id BIGINT, cola VARCHAR, colb VARCHAR)", ImmutableList.of("1, 'abc', 'def'"))) { diff --git a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftCastPushdown.java b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftCastPushdown.java index 7d86e743e085..5c1dabe79353 100644 --- a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftCastPushdown.java +++ b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftCastPushdown.java @@ -579,8 +579,7 @@ protected List invalidCast() @Test void testCastPushdownWithCharConvertedToVarchar() { - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( TEST_SCHEMA + "." + "char_converted_to_varchar_", "(a char(4097))", // char(REDSHIFT_MAX_CHAR` + 1) in Trino is mapped to varchar(REDSHIFT_MAX_CHAR` + 1) in Redshift ImmutableList.of("'hello'"))) { diff --git a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftConnectorTest.java b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftConnectorTest.java index 29f53a68514a..df7eb44513ff 100644 --- a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftConnectorTest.java +++ b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftConnectorTest.java @@ -273,7 +273,7 @@ private void testReadNullFromView(String redshiftType, String trinoType, boolean @Test public void testRedshiftAddNotNullColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, TEST_SCHEMA + ".test_add_column_", "(col int)")) { + try (TestTable table = newTrinoTable(TEST_SCHEMA + ".test_add_column_", "(col int)")) { assertThatThrownBy(() -> onRemoteDatabase().execute("ALTER TABLE " + table.getName() + " ADD COLUMN new_col int NOT NULL")) .hasMessageContaining("ERROR: ALTER TABLE ADD COLUMN defined as NOT NULL must have a non-null default expression"); } @@ -316,7 +316,7 @@ public void testRangeQueryConvertedToInClauseQuery() public void testDelete() { // The base tests is very slow because Redshift CTAS is really slow, so use a smaller test - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_delete_", "AS SELECT * FROM nation")) { // delete without matching any rows assertUpdate("DELETE FROM " + table.getName() + " WHERE nationkey < 0", 0); @@ -449,7 +449,7 @@ public void testCountDistinctWithStringTypes() .collect(toImmutableList()); String tableName = "distinct_strings" + randomNameSuffix(); - try (TestTable testTable = new TestTable(getQueryRunner()::execute, tableName, "(t_char CHAR(5), t_varchar VARCHAR(5))", rows)) { + try (TestTable testTable = newTrinoTable(tableName, "(t_char CHAR(5), t_varchar VARCHAR(5))", rows)) { // Single count(DISTINCT ...) can be pushed even down even if SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT == false as GROUP BY assertThat(query("SELECT count(DISTINCT t_varchar) FROM " + testTable.getName())) .matches("VALUES BIGINT '6'") @@ -661,7 +661,7 @@ public void testDecimalAvgPushdownForMaximumDecimalScale() "12345789.9876543210", format("%s.%s", "1".repeat(28), "9".repeat(10))); - try (TestTable testTable = new TestTable(getQueryRunner()::execute, TEST_SCHEMA + ".test_agg_pushdown_avg_max_decimal", + try (TestTable testTable = newTrinoTable(TEST_SCHEMA + ".test_agg_pushdown_avg_max_decimal", "(t_decimal DECIMAL(38, 10))", rows)) { // Redshift avg rounds down decimal result which doesn't match Presto semantics assertThatThrownBy(() -> assertThat(query("SELECT avg(t_decimal) FROM " + testTable.getName())).isFullyPushedDown()) @@ -683,7 +683,7 @@ public void testDecimalAvgPushdownFoShortDecimalScale() "0.987654321234567890", format("0.%s", "1".repeat(18))); - try (TestTable testTable = new TestTable(getQueryRunner()::execute, TEST_SCHEMA + ".test_agg_pushdown_avg_max_decimal", + try (TestTable testTable = newTrinoTable(TEST_SCHEMA + ".test_agg_pushdown_avg_max_decimal", "(t_decimal DECIMAL(18, 18))", rows)) { assertThat(query("SELECT avg(t_decimal) FROM " + testTable.getName())).isFullyPushedDown(); } @@ -699,15 +699,13 @@ public void testInsertRowConcurrently() @Test public void testJoinPushdownWithImplicitCast() { - try (TestTable leftTable = new TestTable( - getQueryRunner()::execute, + try (TestTable leftTable = newTrinoTable( "left_table_", "(id int, c_boolean boolean, c_tinyint tinyint, c_smallint smallint, c_integer integer, c_bigint bigint, c_real real, c_double_precision double precision, c_decimal_10_2 decimal(10, 2), c_varchar_50 varchar(50))", ImmutableList.of( "(11, true, 12, 12, 12, 12, 12.34, 12.34, 12.34, 'India')", "(12, false, 123, 123, 123, 123, 123.67, 123.67, 123.67, 'Poland')")); - TestTable rightTable = new TestTable( - getQueryRunner()::execute, + TestTable rightTable = newTrinoTable( "right_table_", "(id int, c_boolean boolean, c_tinyint tinyint, c_smallint smallint, c_integer integer, c_bigint bigint, c_real real, c_double_precision double precision, c_decimal_10_2 decimal(10, 2), c_varchar_100 varchar(100), c_varchar varchar)", ImmutableList.of( diff --git a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreConnectorTest.java b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreConnectorTest.java index 6ba379deed51..dede72d518ee 100644 --- a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreConnectorTest.java +++ b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreConnectorTest.java @@ -207,7 +207,7 @@ public void testSingleStoreTinyint() @Override public void testInsertIntoNotNullColumn() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { + try (TestTable table = newTrinoTable("insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); assertQueryFails(format("INSERT INTO %s (nullable_col) VALUES (1)", table.getName()), errorMessageForInsertIntoNotNullColumn("not_null_col")); @@ -220,7 +220,7 @@ public void testInsertIntoNotNullColumn() assertQueryFails(format("INSERT INTO %s (nullable_col) SELECT nationkey FROM nation WHERE regionkey < 0", table.getName()), ".*Field 'not_null_col' doesn't have a default value.*"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { + try (TestTable table = newTrinoTable("commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); // This is enforced by the engine and not the connector @@ -256,7 +256,7 @@ public void testAddNotNullColumn() .isInstanceOf(AssertionError.class) .hasMessage("Should fail to add not null column without a default value to a non-empty table"); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_col", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_nn_col", "(a_varchar varchar)")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES ('a')", 1); diff --git a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreLatestTypeMapping.java b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreLatestTypeMapping.java index db43725c72ea..447ff14babe2 100644 --- a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreLatestTypeMapping.java +++ b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreLatestTypeMapping.java @@ -37,7 +37,7 @@ protected QueryRunner createQueryRunner() @Test void testUnsupportedTinyint() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "tpch.test_unsupported_tinyint", "(value tinyint)")) { + try (TestTable table = newTrinoTable("tpch.test_unsupported_tinyint", "(value tinyint)")) { assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (-129)", table.getName()))) .hasMessageContaining("Out of range value"); assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (128)", table.getName()))) @@ -48,7 +48,7 @@ void testUnsupportedTinyint() @Test void testUnsupportedSmallint() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "tpch.test_unsupported_smallint", "(value smallint)")) { + try (TestTable table = newTrinoTable("tpch.test_unsupported_smallint", "(value smallint)")) { assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (-32769)", table.getName()))) .hasMessageContaining("Out of range value"); assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (32768)", table.getName()))) @@ -59,7 +59,7 @@ void testUnsupportedSmallint() @Test void testUnsupportedInteger() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "tpch.test_unsupported_integer", "(value integer)")) { + try (TestTable table = newTrinoTable("tpch.test_unsupported_integer", "(value integer)")) { assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (-2147483649)", table.getName()))) .hasMessageContaining("Out of range value"); assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (2147483648)", table.getName()))) @@ -70,7 +70,7 @@ void testUnsupportedInteger() @Test void testUnsupportedBigint() { - try (TestTable table = new TestTable(getQueryRunner()::execute, "tpch.test_unsupported_bigint", "(value bigint)")) { + try (TestTable table = newTrinoTable("tpch.test_unsupported_bigint", "(value bigint)")) { assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (-9223372036854775809)", table.getName()))) .hasMessageContaining("Out of range value"); assertThatThrownBy(() -> singleStoreServer.execute(format("INSERT INTO %s VALUES (9223372036854775808)", table.getName()))) diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java index 903c8b9c523e..ec67c6c1798d 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java @@ -363,7 +363,7 @@ public void testDeleteWithVarcharInequalityPredicate() // Override this because by enabling this flag SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY, // we assume that we also support range pushdowns, but for now we only support 'not equal' pushdown, // so cannot enable this flag for now - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_varchar", "(col varchar(1))", ImmutableList.of("'a'", "'A'", "null"))) { + try (TestTable table = newTrinoTable("test_delete_varchar", "(col varchar(1))", ImmutableList.of("'a'", "'A'", "null"))) { assertUpdate("DELETE FROM " + table.getName() + " WHERE col != 'A'", 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES 'A', null"); } @@ -834,7 +834,7 @@ THEN INSERT(id) VALUES(SOURCE.id) public void testConstantUpdateWithVarcharInequalityPredicates() { // Sql Server supports push down predicate for not equal operator - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_varchar", "(col1 INT, col2 varchar(1))", ImmutableList.of("1, 'a'", "2, 'A'"))) { + try (TestTable table = newTrinoTable("test_update_varchar", "(col1 INT, col2 varchar(1))", ImmutableList.of("1, 'a'", "2, 'A'"))) { assertUpdate("UPDATE " + table.getName() + " SET col1 = 20 WHERE col2 != 'A'", 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (20, 'a'), (2, 'A')"); } diff --git a/plugin/trino-vertica/src/test/java/io/trino/plugin/vertica/TestVerticaConnectorTest.java b/plugin/trino-vertica/src/test/java/io/trino/plugin/vertica/TestVerticaConnectorTest.java index d65709629852..fb38c381de52 100644 --- a/plugin/trino-vertica/src/test/java/io/trino/plugin/vertica/TestVerticaConnectorTest.java +++ b/plugin/trino-vertica/src/test/java/io/trino/plugin/vertica/TestVerticaConnectorTest.java @@ -125,9 +125,8 @@ private void testJoinPushdown(JoinOperator joinOperator) Stream.of(notDistinctOperator)) .collect(toImmutableList()); - try (TestTable nationLowercaseTable = new TestTable( + try (TestTable nationLowercaseTable = newTrinoTable( // If a connector supports Join pushdown, but does not allow CTAS, we need to make the table creation here overridable. - getQueryRunner()::execute, "nation_lowercase", "AS SELECT nationkey, lower(name) name, regionkey FROM nation")) { // basic case diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionCoordinatorExcludedTest.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionCoordinatorExcludedTest.java index 39cfa116bc6d..34448e4ef58a 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionCoordinatorExcludedTest.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionCoordinatorExcludedTest.java @@ -67,7 +67,7 @@ public void testInsert() { String query = "SELECT name, nationkey, regionkey FROM nation"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", "AS " + query + " WITH NO DATA")) { + try (TestTable table = newTrinoTable("test_insert_", "AS " + query + " WITH NO DATA")) { assertQuery("SELECT count(*) FROM " + table.getName(), "SELECT 0"); assertUpdate("INSERT INTO " + table.getName() + " " + query, 25); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java index 2517e923d4d1..ad314ce27967 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java @@ -13,6 +13,7 @@ */ package io.trino.testing; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.MoreCollectors; import com.google.errorprone.annotations.CanIgnoreReturnValue; @@ -58,6 +59,7 @@ import io.trino.sql.tree.ExplainType; import io.trino.testing.QueryRunner.MaterializedResultWithPlan; import io.trino.testing.TestingAccessControlManager.TestingPrivilege; +import io.trino.testing.sql.TestTable; import org.assertj.core.api.AssertProvider; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; @@ -667,6 +669,16 @@ private Optional tryGetDistributedQueryRunner() return Optional.empty(); } + protected TestTable newTrinoTable(String namePrefix, @Language("SQL") String tableDefinition) + { + return newTrinoTable(namePrefix, tableDefinition, ImmutableList.of()); + } + + protected TestTable newTrinoTable(String namePrefix, @Language("SQL") String tableDefinition, List rowsToInsert) + { + return new TestTable(getQueryRunner()::execute, namePrefix, tableDefinition, rowsToInsert); + } + protected Session noJoinReordering() { return noJoinReordering(JoinDistributionType.PARTITIONED); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java index b491e6ab375c..dfc875d2694a 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorSmokeTest.java @@ -189,7 +189,7 @@ public void testInsert() throw new AssertionError("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", getCreateTableDefaultDefinition())) { + try (TestTable table = newTrinoTable("test_insert_", getCreateTableDefaultDefinition())) { assertUpdate("INSERT INTO " + table.getName() + " (a, b) VALUES (42, -38.5), (13, 99.9)", 2); assertThat(query("SELECT CAST(a AS bigint), b FROM " + table.getName())) .matches(expectedValues("(42, -38.5), (13, 99.9)")); @@ -205,7 +205,7 @@ public void verifySupportsDeleteDeclaration() } assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_delete", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_supports_delete", "AS SELECT * FROM region")) { assertQueryFails("DELETE FROM " + table.getName(), MODIFYING_ROWS_MESSAGE); } } @@ -219,7 +219,7 @@ public void verifySupportsRowLevelDeleteDeclaration() } assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_row_level_delete", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_supports_row_level_delete", "AS SELECT * FROM region")) { assertQueryFails("DELETE FROM " + table.getName() + " WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -233,7 +233,7 @@ public void verifySupportsUpdateDeclaration() } assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_supports_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = 100 WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -247,7 +247,7 @@ public void verifySupportsRowLevelUpdateDeclaration() } assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = nationkey * 100 WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -256,7 +256,7 @@ public void verifySupportsRowLevelUpdateDeclaration() public void testUpdate() { assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM nation")) { assertUpdate("UPDATE " + table.getName() + " SET nationkey = 100 WHERE regionkey = 2", 5); assertQuery("SELECT count(*) FROM " + table.getName() + " WHERE nationkey = 100", "VALUES 5"); } @@ -266,7 +266,7 @@ public void testUpdate() public void testDeleteAllDataFromTable() { assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_DELETE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_all_data", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_delete_all_data", "AS SELECT * FROM region")) { // not using assertUpdate as some connectors provide update count and some do not getQueryRunner().execute("DELETE FROM " + table.getName()); assertQuery("SELECT count(*) FROM " + table.getName(), "VALUES 0"); @@ -278,7 +278,7 @@ public void testRowLevelDelete() { assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)); // TODO (https://github.com/trinodb/trino/issues/5901) Use longer table name once Oracle version is updated - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_delete", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_row_delete", "AS SELECT * FROM region")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE regionkey = 2", 1); assertThat(query("SELECT * FROM " + table.getName() + " WHERE regionkey = 2")) .returnsEmptyResult(); @@ -298,7 +298,7 @@ public void testTruncateTable() assumeTrue(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_truncate", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_truncate", "AS SELECT * FROM region")) { assertUpdate("TRUNCATE TABLE " + table.getName()); assertThat(query("TABLE " + table.getName())) .returnsEmptyResult(); @@ -318,7 +318,7 @@ public void testRowLevelUpdate() throw new AssertionError("Cannot test UPDATE without INSERT"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_", getCreateTableDefaultDefinition())) { + try (TestTable table = newTrinoTable("test_update_", getCreateTableDefaultDefinition())) { assertUpdate("INSERT INTO " + table.getName() + " (a, b) SELECT regionkey, regionkey * 2.5 FROM region", "SELECT count(*) FROM region"); assertThat(query("SELECT a, b FROM " + table.getName())) .matches(expectedValues("(0, 0.0), (1, 2.5), (2, 5.0), (3, 7.5), (4, 10.0)")); @@ -344,7 +344,7 @@ public void testMerge() throw new AssertionError("Cannot test MERGE without INSERT"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_merge_", getCreateTableDefaultDefinition())) { + try (TestTable table = newTrinoTable("test_merge_", getCreateTableDefaultDefinition())) { assertUpdate("INSERT INTO " + table.getName() + " (a, b) SELECT regionkey, regionkey * 2.5 FROM region", "SELECT count(*) FROM region"); assertThat(query("SELECT a, b FROM " + table.getName())) .matches(expectedValues("(0, 0.0), (1, 2.5), (2, 5.0), (3, 7.5), (4, 10.0)")); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index c9fbbc3bffa3..1cba708dc25f 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -355,8 +355,7 @@ public void testCharVarcharComparison() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_char_varchar", "(k, v) AS VALUES" + " (-1, CAST(NULL AS char(3))), " + @@ -389,8 +388,7 @@ public void testVarcharCharComparison() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_varchar_char", "(k, v) AS VALUES" + " (-1, CAST(NULL AS varchar(3))), " + @@ -589,8 +587,7 @@ public void testVarcharCastToDateInPredicate() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "varchar_as_date_pred", "(a varchar)", List.of( @@ -611,8 +608,7 @@ public void testVarcharCastToDateInPredicate() } } - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "varchar_as_date_pred", "(a varchar)", List.of("'2005-06-bad-date'", "'2005-09-10'"))) { @@ -642,8 +638,7 @@ public void testVarcharCastToDateInPredicate() failureAssert -> failureAssert .hasMessage("Value cannot be cast to date: 2005-06-bad-date")); } - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "varchar_as_date_pred", "(a varchar)", List.of("'2005-09-10'"))) { @@ -1300,8 +1295,7 @@ public void testMaterializedViewGracePeriod() return; } - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_base_table", "AS TABLE region")) { Session defaultSession = getSession(); @@ -2277,7 +2271,7 @@ public void testAddColumn() } String tableName; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_column_", tableDefinitionForAddColumn())) { + try (TestTable table = newTrinoTable("test_add_column_", tableDefinitionForAddColumn())) { tableName = table.getName(); assertUpdate("INSERT INTO " + table.getName() + " SELECT 'first'", 1); assertQueryFails("ALTER TABLE " + table.getName() + " ADD COLUMN x bigint", ".* Column 'x' already exists"); @@ -2337,7 +2331,7 @@ public void testAddColumnWithComment() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_col_desc_", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_col_desc_", "(a_varchar varchar)")) { String tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar COMMENT 'test new column comment'"); @@ -2353,7 +2347,7 @@ public void testAddNotNullColumnToEmptyTable() { skipTestUnless(hasBehavior(SUPPORTS_ADD_COLUMN)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_to_empty", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_nn_to_empty", "(a_varchar varchar)")) { String tableName = table.getName(); String addNonNullColumn = "ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"; @@ -2380,7 +2374,7 @@ public void testAddNotNullColumn() { skipTestUnless(hasBehavior(SUPPORTS_ADD_COLUMN_NOT_NULL_CONSTRAINT)); // covered by testAddNotNullColumnToEmptyTable - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_col", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_nn_col", "(a_varchar varchar)")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES ('a')", 1); @@ -2424,7 +2418,7 @@ public void testAddRowField() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_TYPE)); if (!hasBehavior(SUPPORTS_ADD_FIELD)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_field_", "AS SELECT CAST(row(1) AS row(x integer)) AS col")) { + try (TestTable table = newTrinoTable("test_add_field_", "AS SELECT CAST(row(1) AS row(x integer)) AS col")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ADD COLUMN col.y integer", "This connector does not support adding fields"); @@ -2432,7 +2426,7 @@ public void testAddRowField() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_field_", "AS SELECT CAST(row(1, row(10)) AS row(a integer, b row(x integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(a integer, b row(x integer))"); @@ -2462,7 +2456,7 @@ public void testAddRowFieldInArray() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_TYPE)); if (!hasBehavior(SUPPORTS_ADD_FIELD_IN_ARRAY)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_field_in_array_", "AS SELECT CAST(array[row(1)] AS array(row(x integer))) AS col")) { + try (TestTable table = newTrinoTable("test_add_field_in_array_", "AS SELECT CAST(array[row(1)] AS array(row(x integer))) AS col")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ADD COLUMN col.element.y integer", ".*does not support.*"); @@ -2470,7 +2464,7 @@ public void testAddRowFieldInArray() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_field_in_array_", "AS SELECT CAST(array[row(1, row(10), array[row(11)])] AS array(row(a integer, b row(x integer), c array(row(v integer))))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(row(a integer, b row(x integer), c array(row(v integer))))"); @@ -2506,7 +2500,7 @@ public void testAddRowFieldInArray() } // test row in array of arrays - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_field_in_array_nested_", "AS SELECT CAST(array[array[row(1, row(10), array[row(11)])]] AS array(array(row(a integer, b row(x integer), c array(row(v integer)))))) AS col")) { @@ -2537,7 +2531,7 @@ public void testDropColumn() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); String tableName; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_column_", "AS SELECT 123 x, 456 y, 111 a")) { + try (TestTable table = newTrinoTable("test_drop_column_", "AS SELECT 123 x, 456 y, 111 a")) { tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " DROP COLUMN x"); assertUpdate("ALTER TABLE " + tableName + " DROP COLUMN IF EXISTS y"); @@ -2561,7 +2555,7 @@ public void testDropRowField() if (!hasBehavior(SUPPORTS_DROP_COLUMN) || !hasBehavior(SUPPORTS_ROW_TYPE)) { return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_field_", "AS SELECT CAST(row(1, 2) AS row(x integer, y integer)) AS col")) { + try (TestTable table = newTrinoTable("test_drop_field_", "AS SELECT CAST(row(1, 2) AS row(x integer, y integer)) AS col")) { assertQueryFails( "ALTER TABLE " + table.getName() + " DROP COLUMN col.x", "This connector does not support dropping fields"); @@ -2569,7 +2563,7 @@ public void testDropRowField() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_field_", "AS SELECT CAST(row(1, 2, row(10, 20)) AS row(a integer, b integer, c row(x integer, y integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(a integer, b integer, c row(x integer, y integer))"); @@ -2610,7 +2604,7 @@ public void testDropRowFieldInArray() if (!hasBehavior(SUPPORTS_DROP_COLUMN) || !hasBehavior(SUPPORTS_ROW_TYPE)) { return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_field_in_array_", "AS SELECT CAST(array[row(1, 2)] AS array(row(x integer, y integer))) AS col")) { + try (TestTable table = newTrinoTable("test_drop_field_in_array_", "AS SELECT CAST(array[row(1, 2)] AS array(row(x integer, y integer))) AS col")) { assertQueryFails( "ALTER TABLE " + table.getName() + " DROP COLUMN col.element.x", ".*does not support.*"); @@ -2618,7 +2612,7 @@ public void testDropRowFieldInArray() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_field_in_array_", "AS SELECT CAST(array[row(1, 2, row(10, 20), array[row(30, 40)])] AS array(row(a integer, b integer, c row(x integer, y integer), d array(row(v integer, w integer))))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(row(a integer, b integer, c row(x integer, y integer), d array(row(v integer, w integer))))"); @@ -2672,7 +2666,7 @@ public void testDropRowFieldInArray() assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(row(a integer))"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_field_in_array_nested_", "AS SELECT CAST(array[array[row(1, 2, row(10, 20), array[row(30, 40)])]] AS array(array(row(a integer, b integer, c row(x integer, y integer), d array(row(v integer, w integer)))))) AS col")) { @@ -2705,7 +2699,7 @@ public void testDropRowFieldWhenDuplicates() { skipTestUnless(hasBehavior(SUPPORTS_DROP_FIELD)); - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_duplicated_field_", "AS SELECT CAST(row(1, 2, 3) AS row(a integer, a integer, b integer)) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(a integer, a integer, b integer)"); @@ -2722,7 +2716,7 @@ public void testDropRowFieldCaseSensitivity() { skipTestUnless(hasBehavior(SUPPORTS_DROP_FIELD)); - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_row_field_case_sensitivity_", "AS SELECT CAST(row(1, 2) AS row(lower integer, \"UPPER\" integer)) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(lower integer, UPPER integer)"); @@ -2745,7 +2739,7 @@ public void testDropAmbiguousRowFieldCaseSensitivity() { skipTestUnless(hasBehavior(SUPPORTS_DROP_FIELD)); - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_drop_row_field_case_sensitivity_", """ AS SELECT CAST(row(1, 2, 3, 4, 5) AS @@ -2771,7 +2765,7 @@ public void testDropAndAddColumnWithSameName() { skipTestUnless(hasBehavior(SUPPORTS_DROP_COLUMN) && hasBehavior(SUPPORTS_ADD_COLUMN)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_add_column", "AS SELECT 1 x, 2 y, 3 z")) { + try (TestTable table = newTrinoTable("test_drop_add_column", "AS SELECT 1 x, 2 y, 3 z")) { assertUpdate("ALTER TABLE " + table.getName() + " DROP COLUMN y"); assertQuery("SELECT * FROM " + table.getName(), "VALUES (1, 3)"); @@ -2791,7 +2785,7 @@ public void testRenameColumn() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); String tableName; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_rename_column_", "AS SELECT 'some value' x")) { + try (TestTable table = newTrinoTable("test_rename_column_", "AS SELECT 'some value' x")) { tableName = table.getName(); assertUpdate("ALTER TABLE " + tableName + " RENAME COLUMN x TO before_y"); assertUpdate("ALTER TABLE " + tableName + " RENAME COLUMN IF EXISTS before_y TO y"); @@ -2823,7 +2817,7 @@ public void testRenameColumnWithComment() { skipTestUnless(hasBehavior(SUPPORTS_RENAME_COLUMN) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_rename_column_", "(col INT COMMENT 'test column comment')")) { + try (TestTable table = newTrinoTable("test_rename_column_", "(col INT COMMENT 'test column comment')")) { assertThat(getColumnComment(table.getName(), "col")).isEqualTo("test column comment"); assertUpdate("ALTER TABLE " + table.getName() + " RENAME COLUMN col TO renamed_col"); @@ -2837,7 +2831,7 @@ public void testRenameRowField() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_TYPE)); if (!hasBehavior(SUPPORTS_RENAME_FIELD)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_rename_field_", "AS SELECT CAST(row(1) AS row(x integer)) AS col")) { + try (TestTable table = newTrinoTable("test_rename_field_", "AS SELECT CAST(row(1) AS row(x integer)) AS col")) { assertQueryFails( "ALTER TABLE " + table.getName() + " RENAME COLUMN col.x TO x_renamed", "This connector does not support renaming fields"); @@ -2845,7 +2839,7 @@ public void testRenameRowField() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_field_", "AS SELECT CAST(row(1, row(10)) AS row(a integer, b row(x integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(a integer, b row(x integer))"); @@ -2874,7 +2868,7 @@ public void testRenameRowFieldCaseSensitivity() { skipTestUnless(hasBehavior(SUPPORTS_RENAME_FIELD)); - try (TestTable table = new TestTable(getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_add_row_field_case_sensitivity_", "AS SELECT CAST(row(1, 2) AS row(lower integer, \"UPPER\" integer)) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(lower integer, UPPER integer)"); @@ -2903,7 +2897,7 @@ public void testSetColumnType() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", "AS SELECT CAST(123 AS integer) AS col")) { + try (TestTable table = newTrinoTable("test_set_column_type_", "AS SELECT CAST(123 AS integer) AS col")) { assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE bigint"); assertThat(getColumnType(table.getName(), "col")).isEqualTo("bigint"); @@ -2921,7 +2915,7 @@ public void testSetColumnTypes() for (SetColumnTypeSetup setup : setColumnTypesDataProvider()) { TestTable table; try { - table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", " AS SELECT CAST(" + setup.sourceValueLiteral + " AS " + setup.sourceColumnType + ") AS col"); + table = newTrinoTable("test_set_column_type_", " AS SELECT CAST(" + setup.sourceValueLiteral + " AS " + setup.sourceColumnType + ") AS col"); } catch (Exception e) { verifyUnsupportedTypeException(e, setup.sourceColumnType); @@ -3043,7 +3037,7 @@ public void testSetColumnTypeWithNotNull() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_null_", "(col int NOT NULL)")) { + try (TestTable table = newTrinoTable("test_set_column_type_null_", "(col int NOT NULL)")) { assertThat(columnIsNullable(table.getName(), "col")).isFalse(); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE bigint"); @@ -3056,7 +3050,7 @@ public void testSetColumnTypeWithComment() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_comment_", "(col int COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_set_column_type_comment_", "(col int COMMENT 'test comment')")) { assertThat(getColumnComment(table.getName(), "col")).isEqualTo("test comment"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE bigint"); @@ -3082,7 +3076,7 @@ public void testSetColumnIncompatibleType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_invalid_column_type_", "AS SELECT 'test' AS col")) { + try (TestTable table = newTrinoTable("test_set_invalid_column_type_", "AS SELECT 'test' AS col")) { assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) .satisfies(this::verifySetColumnTypeFailurePermissible); } @@ -3093,7 +3087,7 @@ public void testSetColumnOutOfRangeType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { + try (TestTable table = newTrinoTable("test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) .satisfies(this::verifySetColumnTypeFailurePermissible); } @@ -3110,7 +3104,7 @@ public void testSetFieldType() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_TYPE)); if (!hasBehavior(SUPPORTS_SET_FIELD_TYPE)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_", "(col row(field int))")) { + try (TestTable table = newTrinoTable("test_set_field_type_", "(col row(field int))")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ALTER COLUMN col.field SET DATA TYPE bigint", "This connector does not support setting field types"); @@ -3118,7 +3112,7 @@ public void testSetFieldType() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_", "AS SELECT CAST(row(123) AS row(field integer)) AS col")) { + try (TestTable table = newTrinoTable("test_set_field_type_", "AS SELECT CAST(row(123) AS row(field integer)) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(field integer)"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.field SET DATA TYPE bigint"); @@ -3138,8 +3132,7 @@ public void testSetFieldTypes() for (SetColumnTypeSetup setup : setFieldTypesDataProvider()) { TestTable table; try { - table = new TestTable( - getQueryRunner()::execute, + table = newTrinoTable( "test_set_field_type_", " AS SELECT CAST(row(" + setup.sourceValueLiteral + ") AS row(field " + setup.sourceColumnType + ")) AS col"); } @@ -3182,7 +3175,7 @@ public void testSetFieldTypeCaseSensitivity() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_case_", " AS SELECT CAST(row(1) AS row(\"UPPER\" integer)) col")) { + try (TestTable table = newTrinoTable("test_set_field_type_case_", " AS SELECT CAST(row(1) AS row(\"UPPER\" integer)) col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("row(UPPER integer)"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.upper SET DATA TYPE bigint"); @@ -3197,7 +3190,7 @@ public void testSetFieldTypeWithNotNull() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_null_", "(col row(field int) NOT NULL)")) { + try (TestTable table = newTrinoTable("test_set_field_type_null_", "(col row(field int) NOT NULL)")) { assertThat(columnIsNullable(table.getName(), "col")).isFalse(); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.field SET DATA TYPE bigint"); @@ -3210,7 +3203,7 @@ public void testSetFieldTypeWithComment() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_comment_", "(col row(field int) COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_set_field_type_comment_", "(col row(field int) COMMENT 'test comment')")) { assertThat(getColumnComment(table.getName(), "col")).isEqualTo("test comment"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.field SET DATA TYPE bigint"); @@ -3223,8 +3216,7 @@ public void testSetFieldIncompatibleType() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_set_invalid_field_type_", "(row_col row(field varchar), nested_col row(field row(nested int)))")) { assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN row_col.field SET DATA TYPE row(nested integer)")) @@ -3241,8 +3233,7 @@ public void testSetFieldOutOfRangeType() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_set_field_type_invalid_range_", "AS SELECT CAST(row(9223372036854775807) AS row(field bigint)) AS col")) { assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.field SET DATA TYPE integer")) @@ -3256,7 +3247,7 @@ public void testSetFieldTypeInArray() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ARRAY) && hasBehavior(SUPPORTS_ROW_TYPE)); if (!hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_ARRAY)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_array_", "(col array(row(field int)))")) { + try (TestTable table = newTrinoTable("test_set_field_type_in_array_", "(col array(row(field int)))")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ALTER COLUMN col.element.field SET DATA TYPE bigint", ".*does not support.*"); @@ -3264,7 +3255,7 @@ public void testSetFieldTypeInArray() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_array_", "AS SELECT CAST(array[row(123)] AS array(row(field integer))) AS col")) { + try (TestTable table = newTrinoTable("test_set_field_type_in_array_", "AS SELECT CAST(array[row(123)] AS array(row(field integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(row(field integer))"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.element.field SET DATA TYPE bigint"); @@ -3281,7 +3272,7 @@ public void testSetFieldTypeInNestedArray() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_ARRAY) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ARRAY) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_nested_array_", "AS SELECT CAST(array[array[row(array[row(123)])]] AS array(array(row(field array(row(a integer)))))) AS col")) { + try (TestTable table = newTrinoTable("test_set_field_type_in_nested_array_", "AS SELECT CAST(array[array[row(array[row(123)])]] AS array(array(row(field array(row(a integer)))))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(array(row(field array(row(a integer)))))"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.element.element.field.element.a SET DATA TYPE bigint"); @@ -3304,13 +3295,13 @@ public void testSetFieldMapKeyType() String tableDefinition = "AS SELECT CAST(map(array[row(1)], array[2]) AS map(row(field integer), integer)) AS col"; if (!hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_MAP)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_map", tableDefinition)) { + try (TestTable table = newTrinoTable("test_set_field_type_in_map", tableDefinition)) { assertQueryFails("ALTER TABLE " + table.getName() + " ALTER COLUMN col.key.field SET DATA TYPE bigint", ".*does not support.*"); } return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_map", tableDefinition)) { + try (TestTable table = newTrinoTable("test_set_field_type_in_map", tableDefinition)) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("map(row(field integer), integer)"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.key.field SET DATA TYPE bigint"); @@ -3328,13 +3319,13 @@ public void testSetFieldMapValueType() String tableDefinition = "AS SELECT CAST(map(array[1], array[row(2)]) AS map(integer, row(field integer))) AS col"; if (!hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_MAP)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_map", tableDefinition)) { + try (TestTable table = newTrinoTable("test_set_field_type_in_map", tableDefinition)) { assertQueryFails("ALTER TABLE " + table.getName() + " ALTER COLUMN col.value.field SET DATA TYPE bigint", ".*does not support.*"); } return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_field_type_in_map", tableDefinition)) { + try (TestTable table = newTrinoTable("test_set_field_type_in_map", tableDefinition)) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("map(integer, row(field integer))"); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col.value.field SET DATA TYPE bigint"); @@ -3350,8 +3341,7 @@ public void testSetNestedFieldMapKeyType() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_ARRAY) && hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_MAP) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ARRAY) && hasBehavior(SUPPORTS_MAP_TYPE) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_set_nested_field_type_in_map", "AS SELECT CAST(array[map(array[row(1)], array[2])] AS array(map(row(field integer), integer))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(map(row(field integer), integer))"); @@ -3369,8 +3359,7 @@ public void testSetNestedFieldMapValueType() { skipTestUnless(hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_ARRAY) && hasBehavior(SUPPORTS_SET_FIELD_TYPE_IN_MAP) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ARRAY) && hasBehavior(SUPPORTS_MAP_TYPE) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_set_nested_field_type_in_map", "AS SELECT CAST(array[map(array[1], array[row(2)])] AS array(map(integer, row(field integer)))) AS col")) { assertThat(getColumnType(table.getName(), "col")).isEqualTo("array(map(integer, row(field integer)))"); @@ -3401,7 +3390,7 @@ public void testDropNotNullConstraint() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); if (!hasBehavior(SUPPORTS_DROP_NOT_NULL_CONSTRAINT)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_not_null_", "(col integer NOT NULL)")) { + try (TestTable table = newTrinoTable("test_drop_not_null_", "(col integer NOT NULL)")) { assertQueryFails( "ALTER TABLE " + table.getName() + " ALTER COLUMN col DROP NOT NULL", "This connector does not support dropping a not null constraint"); @@ -3409,7 +3398,7 @@ public void testDropNotNullConstraint() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_not_null_", "(col integer NOT NULL)")) { + try (TestTable table = newTrinoTable("test_drop_not_null_", "(col integer NOT NULL)")) { assertThat(columnIsNullable(table.getName(), "col")).isFalse(); assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col DROP NOT NULL"); @@ -3426,7 +3415,7 @@ public void testDropNotNullConstraintWithColumnComment() skipTestUnless(hasBehavior(SUPPORTS_DROP_NOT_NULL_CONSTRAINT) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); // Verify DROP NOT NULL preserves the existing column comment - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_drop_not_null_", "(col integer NOT NULL COMMENT 'test comment')")) { + try (TestTable table = newTrinoTable("test_drop_not_null_", "(col integer NOT NULL COMMENT 'test comment')")) { assertThat(getColumnComment(table.getName(), "col")).isEqualTo("test comment"); assertThat(columnIsNullable(table.getName(), "col")).isFalse(); @@ -3557,7 +3546,7 @@ public void testCreateOrReplaceTableWhenTableAlreadyExistsSameSchema() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", "AS SELECT CAST(1 AS BIGINT) AS nationkey, 'test' AS name, CAST(2 AS BIGINT) AS regionkey FROM nation LIMIT 1")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", "AS SELECT CAST(1 AS BIGINT) AS nationkey, 'test' AS name, CAST(2 AS BIGINT) AS regionkey FROM nation LIMIT 1")) { @Language("SQL") String query = "SELECT nationkey, name, regionkey FROM nation"; @Language("SQL") String rowCountQuery = "SELECT count(*) FROM nation"; assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS " + query, rowCountQuery); @@ -3574,7 +3563,7 @@ public void testCreateOrReplaceTableWhenTableAlreadyExistsSameSchemaNoData() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " AS SELECT nationkey, name, regionkey FROM nation")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " AS SELECT nationkey, name, regionkey FROM nation")) { assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS SELECT nationkey, name, regionkey FROM nation WITH NO DATA", 0L); assertQueryReturnsEmptyResult("SELECT * FROM " + table.getName()); } @@ -3589,7 +3578,7 @@ public void testCreateOrReplaceTableWithNewColumnNames() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " AS SELECT nationkey, name, regionkey FROM nation")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " AS SELECT nationkey, name, regionkey FROM nation")) { assertTableColumnNames(table.getName(), "nationkey", "name", "regionkey"); @Language("SQL") String query = "SELECT nationkey AS nationkey_new, name AS name_new_2, regionkey AS region_key_new FROM nation"; @Language("SQL") String rowCountQuery = "SELECT count(*) FROM nation"; @@ -3608,7 +3597,7 @@ public void testCreateOrReplaceTableWithDifferentDataType() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_", " AS SELECT nationkey, name FROM nation")) { + try (TestTable table = newTrinoTable("test_create_or_replace_", " AS SELECT nationkey, name FROM nation")) { @Language("SQL") String query = "SELECT name AS nationkey, nationkey AS name FROM nation"; @Language("SQL") String rowCountQuery = "SELECT count(*) FROM nation"; assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " AS " + query, rowCountQuery); @@ -4218,7 +4207,7 @@ public void testCommentTable() String catalogName = getSession().getCatalog().orElseThrow(); String schemaName = getSession().getSchema().orElseThrow(); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_", "(a integer)")) { + try (TestTable table = newTrinoTable("test_comment_", "(a integer)")) { // comment initially not set assertThat(getTableComment(catalogName, schemaName, table.getName())).isEqualTo(null); @@ -4311,7 +4300,7 @@ public void testCommentColumn() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column_", "(a integer)")) { + try (TestTable table = newTrinoTable("test_comment_column_", "(a integer)")) { // comment set assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS 'new comment'"); assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())).contains("COMMENT 'new comment'"); @@ -4346,7 +4335,7 @@ protected void testCommentColumnName(String columnName, boolean delimited) String nameInSql = toColumnNameInSql(columnName, delimited); // TODO (https://github.com/trinodb/trino/issues/5901) Use longer table name once Oracle version is updated - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column", "(" + nameInSql + " integer)")) { + try (TestTable table = newTrinoTable("test_comment_column", "(" + nameInSql + " integer)")) { assertUpdate("COMMENT ON COLUMN " + table.getName() + "." + nameInSql + " IS 'test comment'"); assertThat(getColumnComment(table.getName(), columnName.replace("'", "''").toLowerCase(ENGLISH))).isEqualTo("test comment"); } @@ -4416,7 +4405,7 @@ public void testInsert() String query = "SELECT name, nationkey, regionkey FROM nation"; - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", "AS " + query + " WITH NO DATA")) { + try (TestTable table = newTrinoTable("test_insert_", "AS " + query + " WITH NO DATA")) { assertQuery("SELECT count(*) FROM " + table.getName() + "", "SELECT 0"); assertUpdate("INSERT INTO " + table.getName() + " " + query, 25); @@ -4477,13 +4466,13 @@ public void testInsertUnicode() throw new AssertionError("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", "(test varchar(50))")) { + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50))")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'Hello', U&'hello\\6d4B\\8Bd5world\\7F16\\7801' ", 2); assertThat(computeActual("SELECT test FROM " + table.getName()).getOnlyColumnAsSet()) .containsExactlyInAnyOrder("Hello", "hello测试world编码"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", "(test varchar(50))")) { + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50))")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'aa', 'bé'", 2); assertQuery("SELECT test FROM " + table.getName(), "VALUES 'aa', 'bé'"); assertQuery("SELECT test FROM " + table.getName() + " WHERE test = 'aa'", "VALUES 'aa'"); @@ -4492,7 +4481,7 @@ public void testInsertUnicode() assertQueryReturnsEmptyResult("SELECT test FROM " + table.getName() + " WHERE test = 'ba'"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", "(test varchar(50))")) { + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50))")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'a', 'é'", 2); assertQuery("SELECT test FROM " + table.getName(), "VALUES 'a', 'é'"); assertQuery("SELECT test FROM " + table.getName() + " WHERE test = 'a'", "VALUES 'a'"); @@ -4510,7 +4499,7 @@ public void testInsertHighestUnicodeCharacter() throw new AssertionError("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_unicode_", "(test varchar(50))")) { + try (TestTable table = newTrinoTable("test_insert_unicode_", "(test varchar(50))")) { assertUpdate("INSERT INTO " + table.getName() + "(test) VALUES 'Hello', U&'hello\\6d4B\\8Bd5\\+10FFFFworld\\7F16\\7801' ", 2); assertThat(computeActual("SELECT test FROM " + table.getName()).getOnlyColumnAsSet()) .containsExactlyInAnyOrder("Hello", "hello测试􏿿world编码"); @@ -4533,7 +4522,7 @@ public void testInsertArray() abort("not supported"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_array_", "(a ARRAY, b ARRAY)")) { + try (TestTable table = newTrinoTable("test_insert_array_", "(a ARRAY, b ARRAY)")) { assertUpdate("INSERT INTO " + table.getName() + " (a) VALUES (ARRAY[null])", 1); assertUpdate("INSERT INTO " + table.getName() + " (a, b) VALUES (ARRAY[1.23E1], ARRAY[1.23E1])", 1); assertQuery("SELECT a[1], b[1] FROM " + table.getName(), "VALUES (null, null), (12.3, 12)"); @@ -4553,7 +4542,7 @@ public void testInsertMap() abort("not supported"); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_map_", "(col map(integer, integer))")) { + try (TestTable table = newTrinoTable("test_insert_map_", "(col map(integer, integer))")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES map(ARRAY[1], ARRAY[2])", 1); assertThat(query("SELECT * FROM " + table.getName())) .matches("VALUES map(ARRAY[1], ARRAY[2])"); @@ -4566,8 +4555,7 @@ public void testInsertSameValues() skipTestUnless(hasBehavior(SUPPORTS_INSERT)); skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "insert_same_values", "AS " + join(" UNION ALL ", nCopies(2, "SELECT * FROM region")))) { assertQuery("SELECT count(*) FROM " + table.getName(), "VALUES 10"); @@ -4585,13 +4573,13 @@ public void testInsertNegativeDate() throw new AssertionError("Cannot test INSERT negative dates without CREATE TABLE, the test needs to be implemented in a connector-specific way"); } if (!hasBehavior(SUPPORTS_NEGATIVE_DATE)) { - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_date", "(dt DATE)")) { + try (TestTable table = newTrinoTable("insert_date", "(dt DATE)")) { assertQueryFails(format("INSERT INTO %s VALUES (DATE '-0001-01-01')", table.getName()), errorMessageForInsertNegativeDate("-0001-01-01")); } return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_date", "(dt DATE)")) { + try (TestTable table = newTrinoTable("insert_date", "(dt DATE)")) { assertUpdate(format("INSERT INTO %s VALUES (DATE '-0001-01-01')", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES DATE '-0001-01-01'"); assertQuery(format("SELECT * FROM %s WHERE dt = DATE '-0001-01-01'", table.getName()), "VALUES DATE '-0001-01-01'"); @@ -4616,7 +4604,7 @@ public void testInsertIntoNotNullColumn() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { + try (TestTable table = newTrinoTable("insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); // The error message comes from remote databases when ConnectorMetadata.supportsMissingColumnsOnInsert is true @@ -4628,7 +4616,7 @@ public void testInsertIntoNotNullColumn() assertUpdate(format("INSERT INTO %s (nullable_col) SELECT nationkey FROM nation WHERE regionkey < 0", table.getName()), 0); } - try (TestTable table = new TestTable(getQueryRunner()::execute, "commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { + try (TestTable table = newTrinoTable("commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) { assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)"); // This is enforced by the engine and not the connector @@ -4669,8 +4657,7 @@ public void testInsertInTransaction() skipTestUnless(hasBehavior(SUPPORTS_INSERT)); skipTestUnless(hasBehavior(SUPPORTS_MULTI_STATEMENT_WRITES)); // covered by testWriteNotAllowedInTransaction - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_tx_insert", "(a bigint)")) { String tableName = table.getName(); @@ -4688,8 +4675,7 @@ public void testSelectAfterInsertInTransaction() return; } - try (TestTable table = new TestTable( - getQueryRunner()::execute, + try (TestTable table = newTrinoTable( "test_insert_select_", "AS SELECT nationkey, name, regionkey FROM nation WHERE nationkey = 1")) { String tableName = table.getName(); @@ -4733,7 +4719,7 @@ public void testDelete() skipTestUnless(hasBehavior(SUPPORTS_DELETE)); // delete successive parts of the table - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_", "AS SELECT * FROM orders")) { + try (TestTable table = newTrinoTable("test_delete_", "AS SELECT * FROM orders")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE custkey <= 100", "SELECT count(*) FROM orders WHERE custkey <= 100"); assertQuery("SELECT * FROM " + table.getName(), "SELECT * FROM orders WHERE custkey > 100"); @@ -4745,12 +4731,12 @@ public void testDelete() } // delete without matching any rows - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_", "AS SELECT * FROM orders")) { + try (TestTable table = newTrinoTable("test_delete_", "AS SELECT * FROM orders")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE orderkey < 0", 0); } // delete with a predicate that optimizes to false - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_", "AS SELECT * FROM orders")) { + try (TestTable table = newTrinoTable("test_delete_", "AS SELECT * FROM orders")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE orderkey > 5 AND orderkey < 4", 0); } @@ -4776,7 +4762,7 @@ public void testDeleteWithLike() { skipTestUnless(hasBehavior(SUPPORTS_DELETE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_with_like_", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_with_like_", "AS SELECT * FROM nation")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE name LIKE '%a%'", "VALUES 0"); assertUpdate("DELETE FROM " + table.getName() + " WHERE name LIKE '%A%'", "SELECT count(*) FROM nation WHERE name LIKE '%A%'"); } @@ -4904,7 +4890,7 @@ public void testDeleteWithVarcharPredicate() { skipTestUnless(hasBehavior(SUPPORTS_DELETE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_with_varchar_predicate_", "AS SELECT * FROM orders")) { + try (TestTable table = newTrinoTable("test_delete_with_varchar_predicate_", "AS SELECT * FROM orders")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE orderstatus = 'O'", "SELECT count(*) FROM orders WHERE orderstatus = 'O'"); assertQuery("SELECT * FROM " + table.getName(), "SELECT * FROM orders WHERE orderstatus <> 'O'"); } @@ -4919,7 +4905,7 @@ public void verifySupportsDeleteDeclaration() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_delete", "(regionkey int)")) { + try (TestTable table = newTrinoTable("test_supports_delete", "(regionkey int)")) { assertQueryFails("DELETE FROM " + table.getName(), MODIFYING_ROWS_MESSAGE); } } @@ -4933,7 +4919,7 @@ public void verifySupportsRowLevelDeleteDeclaration() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_row_level_delete", "(regionkey int)")) { + try (TestTable table = newTrinoTable("test_supports_row_level_delete", "(regionkey int)")) { assertQueryFails("DELETE FROM " + table.getName() + " WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -4942,7 +4928,7 @@ public void verifySupportsRowLevelDeleteDeclaration() public void testDeleteAllDataFromTable() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_DELETE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_delete_all_data", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_delete_all_data", "AS SELECT * FROM region")) { // not using assertUpdate as some connectors provide update count and some not getQueryRunner().execute("DELETE FROM " + table.getName()); assertQuery("SELECT count(*) FROM " + table.getName(), "VALUES 0"); @@ -4954,7 +4940,7 @@ public void testRowLevelDelete() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_LEVEL_DELETE)); // TODO (https://github.com/trinodb/trino/issues/5901) Use longer table name once Oracle version is updated - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_delete", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_row_delete", "AS SELECT * FROM region")) { assertUpdate("DELETE FROM " + table.getName() + " WHERE regionkey = 2", 1); assertQuery("SELECT count(*) FROM " + table.getName(), "VALUES 4"); } @@ -4969,7 +4955,7 @@ public void verifySupportsUpdateDeclaration() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_supports_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = 100 WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -4983,7 +4969,7 @@ public void verifySupportsRowLevelUpdateDeclaration() } skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_supports_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_supports_update", "AS SELECT * FROM nation")) { assertQueryFails("UPDATE " + table.getName() + " SET nationkey = nationkey * 100 WHERE regionkey = 2", MODIFYING_ROWS_MESSAGE); } } @@ -4996,7 +4982,7 @@ public void testUpdate() assertQueryFails("UPDATE nation SET nationkey = nationkey + regionkey WHERE regionkey < 1", MODIFYING_ROWS_MESSAGE); return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM nation")) { assertUpdate("UPDATE " + table.getName() + " SET nationkey = 100 WHERE regionkey = 2", 5); assertQuery("SELECT count(*) FROM " + table.getName() + " WHERE nationkey = 100", "VALUES 5"); } @@ -5007,7 +4993,7 @@ public void testUpdateMultipleCondition() { skipTestUnless(hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM (VALUES (1, 10), (1, 20), (2, 10)) AS t(a, b)")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM (VALUES (1, 10), (1, 20), (2, 10)) AS t(a, b)")) { assertUpdate("UPDATE " + table.getName() + " SET b = 100 WHERE a = 1 AND b = 10", 1); assertQuery("SELECT * FROM " + table.getName(), "VALUES (1, 100), (1, 20), (2, 10)"); } @@ -5043,7 +5029,7 @@ public void testUpdateCaseSensitivity() { skipTestUnless(hasBehavior(SUPPORTS_UPDATE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_row_update", "AS SELECT * FROM nation")) { + try (TestTable table = newTrinoTable("test_row_update", "AS SELECT * FROM nation")) { assertUpdate("UPDATE " + table.getName() + " SET NATIONKEY = 100 WHERE REGIONKEY = 2", 5); assertQuery("SELECT count(*) FROM " + table.getName() + " WHERE nationkey = 100", "VALUES 5"); } @@ -5336,7 +5322,7 @@ public void testCreateOrReplaceTableConcurrently() protected TestTable createTableWithOneIntegerColumn(String namePrefix) { - return new TestTable(getQueryRunner()::execute, namePrefix, "(col integer)"); + return newTrinoTable(namePrefix, "(col integer)"); } @Test @@ -5375,7 +5361,7 @@ public void testUpdateRowType() return; } - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_update_with_predicates_on_row_types", "(int_t INT, row_t ROW(f1 INT, f2 INT))")) { + try (TestTable table = newTrinoTable("test_update_with_predicates_on_row_types", "(int_t INT, row_t ROW(f1 INT, f2 INT))")) { String tableName = table.getName(); assertUpdate("INSERT INTO " + tableName + " VALUES (1, ROW(2, 3)), (11, ROW(12, 13)), (21, ROW(22, 23))", 3); assertUpdate("UPDATE " + tableName + " SET int_t = int_t - 1 WHERE row_t.f2 = 3", 1); @@ -5394,7 +5380,7 @@ public void testPredicateOnRowTypeField() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_INSERT) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_predicate_on_row_type_field", "(int_t INT, row_t row(varchar_t VARCHAR, int_t INT))")) { + try (TestTable table = newTrinoTable("test_predicate_on_row_type_field", "(int_t INT, row_t row(varchar_t VARCHAR, int_t INT))")) { assertUpdate("INSERT INTO " + table.getName() + " VALUES (2, row('first', 1)), (20, row('second', 10)), (200, row('third', 100))", 3); assertQuery("SELECT int_t FROM " + table.getName() + " WHERE row_t.int_t = 1", "VALUES 2"); assertQuery("SELECT int_t FROM " + table.getName() + " WHERE row_t.int_t > 1", "VALUES 20, 200"); @@ -5449,7 +5435,7 @@ public void testTruncateTable() skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_truncate", "AS SELECT * FROM region")) { + try (TestTable table = newTrinoTable("test_truncate", "AS SELECT * FROM region")) { assertUpdate("TRUNCATE TABLE " + table.getName()); assertQuery("SELECT count(*) FROM " + table.getName(), "VALUES 0"); } @@ -5798,7 +5784,7 @@ protected void testCreateTableWithTableCommentSpecialCharacter(String comment) { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_", "(a bigint) COMMENT " + varcharLiteral(comment))) { + try (TestTable table = newTrinoTable("test_create_", "(a bigint) COMMENT " + varcharLiteral(comment))) { assertThat(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), table.getName())).isEqualTo(comment); } } @@ -5821,7 +5807,7 @@ private void testCreateTableAsSelectWithTableCommentSpecialCharacter(String comm { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_", " COMMENT " + varcharLiteral(comment) + " AS SELECT 1 a")) { + try (TestTable table = newTrinoTable("test_create_", " COMMENT " + varcharLiteral(comment) + " AS SELECT 1 a")) { assertThat(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), table.getName())).isEqualTo(comment); } } @@ -5844,7 +5830,7 @@ private void testCreateTableWithColumnCommentSpecialCharacter(String comment) { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_", " (a bigint COMMENT " + varcharLiteral(comment) + ")")) { + try (TestTable table = newTrinoTable("test_create_", " (a bigint COMMENT " + varcharLiteral(comment) + ")")) { assertThat(getColumnComment(table.getName(), "a")).isEqualTo(comment); } } @@ -5867,7 +5853,7 @@ protected void testAddColumnWithCommentSpecialCharacter(String comment) { skipTestUnless(hasBehavior(SUPPORTS_ADD_COLUMN_WITH_COMMENT)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_col_", "(a_varchar varchar)")) { + try (TestTable table = newTrinoTable("test_add_col_", "(a_varchar varchar)")) { assertUpdate("ALTER TABLE " + table.getName() + " ADD COLUMN b_varchar varchar COMMENT " + varcharLiteral(comment)); assertThat(getColumnComment(table.getName(), "b_varchar")).isEqualTo(comment); } @@ -5891,7 +5877,7 @@ private void testCommentTableSpecialCharacter(String comment) { skipTestUnless(hasBehavior(SUPPORTS_COMMENT_ON_TABLE)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_table_", "(a integer)")) { + try (TestTable table = newTrinoTable("test_comment_table_", "(a integer)")) { assertUpdate("COMMENT ON TABLE " + table.getName() + " IS " + varcharLiteral(comment)); assertThat(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), table.getName())).isEqualTo(comment); } @@ -5915,7 +5901,7 @@ private void testCommentColumnSpecialCharacter(String comment) { skipTestUnless(hasBehavior(SUPPORTS_COMMENT_ON_COLUMN)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column_", "(a integer)")) { + try (TestTable table = newTrinoTable("test_comment_column_", "(a integer)")) { assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS " + varcharLiteral(comment)); assertThat(getColumnComment(table.getName(), "a")).isEqualTo(comment); } @@ -6081,8 +6067,7 @@ public void testTimestampWithTimeZoneCastToDatePredicate() TestTable table; try { - table = new TestTable( - getQueryRunner()::execute, + table = newTrinoTable( "timestamptz_to_date", // These to timestamps are same local time, but different point in times and also different date at UTC time zone """ @@ -6112,8 +6097,7 @@ public void testTimestampWithTimeZoneCastToTimestampPredicate() TestTable table; try { - table = new TestTable( - getQueryRunner()::execute, + table = newTrinoTable( "timestamptz_to_ts", // These to timestamps are same local time, but different point in times """ @@ -7038,8 +7022,7 @@ public void testProjectionPushdown() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_ROW_TYPE)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_", "(id BIGINT, root ROW(f1 BIGINT, f2 BIGINT))", ImmutableList.of("(1, ROW(1, 2))", "(2, NULl)", "(3, ROW(NULL, 4))"))) { @@ -7071,8 +7054,7 @@ public void testProjectionWithCaseSensitiveField() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_DEREFERENCE_PUSHDOWN)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_with_case_sensitive_field_", "(id BIGINT, a ROW(\"UPPER_CASE\" BIGINT, \"lower_case\" BIGINT, \"MiXeD_cAsE\" BIGINT))", ImmutableList.of("(1, ROW(2, 3, 4))", "(5, ROW(6, 7, 8))"))) { @@ -7094,8 +7076,7 @@ public void testProjectionPushdownMultipleRows() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_DEREFERENCE_PUSHDOWN)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_multiple_rows_", "(id BIGINT, nested1 ROW(child1 BIGINT, child2 VARCHAR, child3 BIGINT), nested2 ROW(child1 DOUBLE, child2 BOOLEAN, child3 DATE))", ImmutableList.of( @@ -7143,8 +7124,7 @@ public void testProjectionPushdownWithHighlyNestedData() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_DEREFERENCE_PUSHDOWN)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_highly_nested_data_", "(id INT, row1_t ROW(f1 INT, f2 INT, row2_t ROW (f1 INT, f2 INT, row3_t ROW(f1 INT, f2 INT))))", ImmutableList.of("(1, ROW(2, 3, ROW(4, 5, ROW(6, 7))))", @@ -7176,8 +7156,7 @@ public void testProjectionPushdownReadsLessData() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_DEREFERENCE_PUSHDOWN)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_reads_less_data_", "AS SELECT val AS id, CAST(ROW(val + 1, val + 2) AS ROW(leaf1 BIGINT, leaf2 BIGINT)) AS root FROM UNNEST(SEQUENCE(1, 10)) AS t(val)")) { MaterializedResult expectedResult = computeActual("SELECT val + 2 FROM UNNEST(SEQUENCE(1, 10)) AS t(val)"); @@ -7215,8 +7194,7 @@ public void testProjectionPushdownPhysicalInputSize() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA) && hasBehavior(SUPPORTS_DEREFERENCE_PUSHDOWN)); - try (TestTable testTable = new TestTable( - getQueryRunner()::execute, + try (TestTable testTable = newTrinoTable( "test_projection_pushdown_physical_input_size_", "AS SELECT val AS id, CAST(ROW(val + 1, val + 2) AS ROW(leaf1 BIGINT, leaf2 BIGINT)) AS root FROM UNNEST(SEQUENCE(1, 10)) AS t(val)")) { // Verify that the physical input size is smaller when reading the root.leaf1 field compared to reading the root field