Skip to content

Commit

Permalink
Add newTrinoTable method to AbstractTestQueryFramework
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Dec 28, 2024
1 parent 0c3330a commit 4d76f58
Show file tree
Hide file tree
Showing 54 changed files with 470 additions and 628 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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())
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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()) {
Expand Down Expand Up @@ -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)) {
Expand Down Expand Up @@ -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);
Expand All @@ -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);
Expand All @@ -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);
}
}
Expand All @@ -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);
}
}
Expand All @@ -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);
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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(),
Expand All @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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));
Expand Down Expand Up @@ -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()
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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'"))) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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");
Expand Down Expand Up @@ -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'");
Expand All @@ -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'");
Expand Down Expand Up @@ -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')");
Expand Down Expand Up @@ -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<DOUBLE>, b ARRAY<BIGINT>)")) {
try (TestTable table = newTrinoTable("test_insert_array_", "(a ARRAY<DOUBLE>, b ARRAY<BIGINT>)")) {
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)");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -738,7 +738,7 @@ public void testArray()
@Test
public void testArrayType()
{
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_array_", "(a BIGINT, b ARRAY<DOUBLE>, c ARRAY<BIGINT>)")) {
try (TestTable table = newTrinoTable("test_array_", "(a BIGINT, b ARRAY<DOUBLE>, c ARRAY<BIGINT>)")) {
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 " +
Expand All @@ -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");
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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");
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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')");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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));
Expand Down Expand Up @@ -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));
Expand Down
Loading

0 comments on commit 4d76f58

Please sign in to comment.