Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ public final class WindowAssertions
) AS orders (orderkey, orderstatus, orderdate)
""";

private WindowAssertions() { }
private WindowAssertions() {}

public static void assertWindowQuery(@Language("SQL") String sql, MaterializedResult expected, QueryRunner queryRunner)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,8 +218,6 @@ public void testSkipBroadcastSubtree()
true);
}



private Session getSession()
{
return Session.builder(getPlanTester().getDefaultSession())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ public void testColocatedJoinWhenNumberOfBucketsInTableScanIsNotSufficient()
WHERE
orders.column_a = t.column_a
AND orders.column_b = t.column_b
""",
""",
prepareSession(20, colocatedJoinEnabled),
anyTree(
anyTree(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ WITH t(a, b) AS (
WHEN a = 2 OR b THEN 'b'
ELSE 'c'
END = 'b'
FROM t
FROM t
"""))
.matches("VALUES false");
}
Expand Down
16 changes: 8 additions & 8 deletions core/trino-main/src/test/java/io/trino/sql/query/TestJoin.java
Original file line number Diff line number Diff line change
Expand Up @@ -317,18 +317,18 @@ void testFilterThatMayFail()
assertThat(assertions.query(
"""
WITH
t(x,y) AS (
VALUES
t(x,y) AS (
VALUES
('a', '1'),
('b', 'x'),
(null, 'y')
),
u(x,y) AS (
VALUES
),
u(x,y) AS (
VALUES
('a', '1'),
('c', 'x'),
(null, 'y')
)
)
SELECT *
FROM t JOIN u ON t.x = u.x
WHERE CAST(t.y AS int) = 1
Expand All @@ -341,8 +341,8 @@ WHERE CAST(t.y AS int) = 1
a(k, v) AS (VALUES if(random() >= 0, (1, CAST('10' AS varchar)))),
b(k, v) AS (VALUES if(random() >= 0, (1, CAST('foo' AS varchar)))),
t AS (
SELECT k, CAST(v AS BIGINT) v1, v
FROM a)
SELECT k, CAST(v AS BIGINT) v1, v
FROM a)
SELECT t.k, b.k
FROM t JOIN b ON t.k = b.k AND t.v1 = 10 AND t.v = b.v
"""))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,23 +40,23 @@ public void test()
assertThat(assertions.query(
"""
WITH t(a, b) AS (SELECT random(), random())
SELECT a = b
SELECT a = b
FROM t
"""))
.matches("VALUES false");

assertThat(assertions.query(
"""
WITH t(a, b) AS (VALUES (random(), random()))
SELECT a = b
SELECT a = b
FROM t
"""))
.matches("VALUES false");

assertThat(assertions.query(
"""
WITH t(a, b) AS (SELECT transform(array[1], x -> random())[1], transform(array[1], x -> random())[1])
SELECT a = b
SELECT a = b
FROM t
"""))
.matches("VALUES false");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -868,18 +868,18 @@ public void testIntersect()
assertStatement("SELECT 123 INTERSECT DISTINCT CORRESPONDING SELECT 123 INTERSECT ALL CORRESPONDING SELECT 123",
query(new Intersect(
ImmutableList.of(
new Intersect(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1 ,1), List.of()))),
new Intersect(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1, 1), List.of()))),
createSelect123()),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));

assertStatement("SELECT 123 INTERSECT DISTINCT CORRESPONDING BY (x) SELECT 123 INTERSECT ALL CORRESPONDING SELECT 123",
query(new Intersect(
ImmutableList.of(
new Intersect(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1 ,1), List.of(identifier("x"))))),
new Intersect(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1, 1), List.of(identifier("x"))))),
createSelect123()),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));
}

@Test
Expand All @@ -896,18 +896,18 @@ public void testUnion()
assertStatement("SELECT 123 UNION DISTINCT CORRESPONDING SELECT 123 UNION ALL CORRESPONDING SELECT 123",
query(new Union(
ImmutableList.of(
new Union(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1 ,1), List.of()))),
new Union(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1, 1), List.of()))),
createSelect123()),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));

assertStatement("SELECT 123 UNION DISTINCT CORRESPONDING BY (x) SELECT 123 UNION ALL CORRESPONDING SELECT 123",
query(new Union(
ImmutableList.of(
new Union(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1 ,1), List.of(identifier("x"))))),
new Union(ImmutableList.of(createSelect123(), createSelect123()), true, Optional.of(new Corresponding(location(1, 1), List.of(identifier("x"))))),
createSelect123()),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));
}

@Test
Expand All @@ -924,18 +924,18 @@ public void testExcept()
assertStatement("SELECT 123 EXCEPT DISTINCT CORRESPONDING SELECT 123 EXCEPT ALL CORRESPONDING SELECT 123",
query(new Except(
location(1, 1),
new Except(location(1, 1), createSelect123(), createSelect123(), true, Optional.of(new Corresponding(location(1 ,1), List.of()))),
new Except(location(1, 1), createSelect123(), createSelect123(), true, Optional.of(new Corresponding(location(1, 1), List.of()))),
createSelect123(),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));

assertStatement("SELECT 123 EXCEPT DISTINCT CORRESPONDING BY (x) SELECT 123 EXCEPT ALL CORRESPONDING SELECT 123",
query(new Except(
location(1, 1),
new Except(location(1, 1), createSelect123(), createSelect123(), true, Optional.of(new Corresponding(location(1 ,1), List.of(identifier("x"))))),
new Except(location(1, 1), createSelect123(), createSelect123(), true, Optional.of(new Corresponding(location(1, 1), List.of(identifier("x"))))),
createSelect123(),
false,
Optional.of(new Corresponding(location(1 ,1), List.of())))));
Optional.of(new Corresponding(location(1, 1), List.of())))));
}

private static QuerySpecification createSelect123()
Expand Down Expand Up @@ -3519,22 +3519,23 @@ THEN INSERT (part, qty) VALUES (c.part, c.qty)
ON true
WHEN MATCHED
THEN DELETE
""")).isEqualTo(new Merge(
location,
new AliasedRelation(
new Table(
location(1, 1),
QualifiedName.of(List.of(new Identifier(location(1, 12), "inventory", false))),
Optional.of(new Identifier(location(1, 24), "dev", false))),
new Identifier(location(1, 31), "i", false),
null),
new AliasedRelation(
location(2, 9),
new Table(location(2, 9), QualifiedName.of(List.of(new Identifier(location(2, 9), "changes", false)))),
new Identifier(location(2, 20), "c", false),
null),
new BooleanLiteral(location(3, 6), "true"),
ImmutableList.of(new MergeDelete(location(4, 1), Optional.empty()))));
"""))
.isEqualTo(new Merge(
location,
new AliasedRelation(
new Table(
location(1, 1),
QualifiedName.of(List.of(new Identifier(location(1, 12), "inventory", false))),
Optional.of(new Identifier(location(1, 24), "dev", false))),
new Identifier(location(1, 31), "i", false),
null),
new AliasedRelation(
location(2, 9),
new Table(location(2, 9), QualifiedName.of(List.of(new Identifier(location(2, 9), "changes", false)))),
new Identifier(location(2, 20), "c", false),
null),
new BooleanLiteral(location(3, 6), "true"),
ImmutableList.of(new MergeDelete(location(4, 1), Optional.empty()))));
}

@Test
Expand Down Expand Up @@ -6706,7 +6707,6 @@ public void testListagg()
new BooleanLiteral(location(1, 1), "true"),
new StringLiteral(location(1, 1), "..."),
new BooleanLiteral(location(1, 1), "false"))));

}

@Test
Expand Down Expand Up @@ -7139,7 +7139,7 @@ public void testSessionProperty()
Optional.empty(),
new QuerySpecification(
location(4, 1),
new Select(location(4, 1), false, ImmutableList.of(new SingleColumn(location(4, 8), new LongLiteral(location(4, 8),"1"), Optional.empty()))),
new Select(location(4, 1), false, ImmutableList.of(new SingleColumn(location(4, 8), new LongLiteral(location(4, 8), "1"), Optional.empty()))),
Optional.empty(),
Optional.empty(),
Optional.empty(),
Expand All @@ -7163,7 +7163,7 @@ public void testInvalidQueryScoped()

// Session after function
assertStatementIsInvalid("WITH FUNCTION abc() RETURNS int RETURN 42, SESSION query_max_memory = '1GB' SELECT 1")
.withMessage("line 1:44: mismatched input 'SESSION'. Expecting: 'FUNCTION'");
.withMessage("line 1:44: mismatched input 'SESSION'. Expecting: 'FUNCTION'");

// Session after function
assertStatementIsInvalid("WITH SESSION query_max_memory = '1GB', FUNCTION abc() RETURNS int RETURN 42, SESSION query_max_total_memory = '1GB' SELECT 1")
Expand Down Expand Up @@ -7203,7 +7203,7 @@ FUNCTION foo()
Optional.empty(),
new QuerySpecification(
location(7, 1),
new Select(location(7, 1), false, ImmutableList.of(new SingleColumn(location(7, 8), new LongLiteral(location(7, 8),"1"), Optional.empty()))),
new Select(location(7, 1), false, ImmutableList.of(new SingleColumn(location(7, 8), new LongLiteral(location(7, 8), "1"), Optional.empty()))),
Optional.empty(),
Optional.empty(),
Optional.empty(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,8 @@ public void testDeserializeSimpleFeature()
}

@Test
public void testDeserializeNullValues() throws IOException
public void testDeserializeNullValues()
throws IOException
{
String json = """
{
Expand Down Expand Up @@ -235,7 +236,8 @@ public void testUnsupportedAttributeTypes()
}

@Test
public void testDeserializeMissingColumns() throws IOException
public void testDeserializeMissingColumns()
throws IOException
{
String json = """
{
Expand All @@ -262,7 +264,9 @@ public void testDeserializeInvalidJson()
}

@Test
public void testDeserializeDateFormats() throws IOException {
public void testDeserializeDateFormats()
throws IOException
{
// Test valid epoch milliseconds (as number)
String jsonEpoch = """
{
Expand Down Expand Up @@ -320,7 +324,9 @@ public void testDeserializeDateFormats() throws IOException {
}

@Test
public void testDeserializeTimestampFormats() throws IOException {
public void testDeserializeTimestampFormats()
throws IOException
{
String[] validTimestampFormats = {
"2025-03-03 00:00:00.000", // with leading zeros
"2025-03-03 00:00:00",
Expand Down Expand Up @@ -387,7 +393,9 @@ public void testDeserializeTimestampFormats() throws IOException {
}

@Test
public void testDeserializeInvalidDateFormat() throws IOException {
public void testDeserializeInvalidDateFormat()
throws IOException
{
String json = """
{
"attributes": {
Expand All @@ -402,7 +410,9 @@ public void testDeserializeInvalidDateFormat() throws IOException {
}

@Test
public void testDeserializeInvalidTimestampFormat() throws IOException {
public void testDeserializeInvalidTimestampFormat()
throws IOException
{
String json = """
{
"attributes": {
Expand All @@ -417,7 +427,9 @@ public void testDeserializeInvalidTimestampFormat() throws IOException {
}

@Test
public void testDeserializeDateOutOfRange() throws IOException {
public void testDeserializeDateOutOfRange()
throws IOException
{
String json = """
{
"attributes": {
Expand Down Expand Up @@ -595,7 +607,8 @@ private static Page parse(String json)
}

private static Page parse(String json, List<Column> columns)
throws IOException {
throws IOException
{
JsonParser jsonParser = JSON_FACTORY.createParser(json);
assertThat(jsonParser.nextToken()).isEqualTo(START_OBJECT);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1930,7 +1930,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 = createTestTableForWrites( "test_delete_varchar", "(col varchar(1), pk INT)", ImmutableList.of("'a', 1", "'A', 2", "null, 3"), "pk")) {
try (TestTable table = createTestTableForWrites("test_delete_varchar", "(col varchar(1), pk INT)", ImmutableList.of("'a', 1", "'A', 2", "null, 3"), "pk")) {
if (!hasBehavior(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_EQUALITY) && !hasBehavior(SUPPORTS_ROW_LEVEL_UPDATE)) {
assertQueryFails("DELETE FROM " + table.getName() + " WHERE col = 'A'", MODIFYING_ROWS_MESSAGE);
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1233,7 +1233,7 @@ public void testExecuteProcedureWithInvalidQuery()
@Test
public void testLimitPushdownWithExternalTable()
{
String externalTableName = TEST_SCHEMA + ".region_external_table_" + randomNameSuffix();
String externalTableName = TEST_SCHEMA + ".region_external_table_" + randomNameSuffix();
onBigQuery("CREATE EXTERNAL TABLE " + externalTableName + " OPTIONS (format = 'CSV', uris = ['gs://" + gcpStorageBucket + "/tpch/tiny/region.csv'])");
try {
assertLimitPushdownOnRegionTable(getSession(), externalTableName);
Expand Down Expand Up @@ -1262,7 +1262,7 @@ public void testLimitPushdownWithView()
@Test
public void testLimitPushdownWithMaterializedView()
{
String mvName = TEST_SCHEMA + ".region_mv_" + randomNameSuffix();
String mvName = TEST_SCHEMA + ".region_mv_" + randomNameSuffix();
onBigQuery("CREATE MATERIALIZED VIEW " + mvName + " AS SELECT * FROM tpch.region");
try {
// materialized view with materialization uses storage api, with storage api limit pushdown is not supported
Expand Down Expand Up @@ -1531,5 +1531,4 @@ private void onBigQuery(@Language("SQL") String sql)
{
bigQuerySqlExecutor.execute(sql);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,7 @@ public class TestDeltaLakeBasic
new ResourceTable("person_without_last_checkpoint", "databricks73/person_without_last_checkpoint"),
new ResourceTable("person_without_old_jsons", "databricks73/person_without_old_jsons"),
new ResourceTable("person_without_checkpoints", "databricks73/person_without_checkpoints"));

private static final List<ResourceTable> OTHER_TABLES = ImmutableList.of(
new ResourceTable("allow_column_defaults", "deltalake/allow_column_defaults"),
new ResourceTable("stats_with_minmax_nulls", "deltalake/stats_with_minmax_nulls"),
Expand Down Expand Up @@ -1511,7 +1512,7 @@ private void testDeletionVectorsPages(boolean parquetUseColumnIndex)
assertQueryReturnsEmptyResult(session, "SELECT * FROM " + tableName + " WHERE id = 20001");
assertThat(query(session, "SELECT * FROM " + tableName + " WHERE id = 99999")).matches("VALUES 99999");

assertThat(query(session, "SELECT id, _change_type, _commit_version FROM TABLE(system.table_changes('tpch', '" +tableName+ "')) WHERE id = 20001"))
assertThat(query(session, "SELECT id, _change_type, _commit_version FROM TABLE(system.table_changes('tpch', '" + tableName + "')) WHERE id = 20001"))
.matches("VALUES (20001, VARCHAR 'insert', BIGINT '1'), (20001, VARCHAR 'update_preimage', BIGINT '2')");

assertUpdate("DROP TABLE " + tableName);
Expand Down Expand Up @@ -2723,7 +2724,8 @@ public static void updateClonedTableDeletionVectorPathPrefixAndSource(Path locat
if (!content.equals(newContent)) {
Files.write(file, newContent.getBytes(StandardCharsets.UTF_8), StandardOpenOption.TRUNCATE_EXISTING);
}
} catch (IOException e) {
}
catch (IOException e) {
throw new RuntimeException(e);
}
});
Expand Down
Loading
Loading